diff --git a/.asf.yaml b/.asf.yaml
index 3935a525ff3c4..296aaff5c4a2d 100644
--- a/.asf.yaml
+++ b/.asf.yaml
@@ -13,7 +13,7 @@
# See the License for the specific language governing permissions and
# limitations under the License.
-# https://cwiki.apache.org/confluence/display/INFRA/git+-+.asf.yaml+features
+# https://github.com/apache/infrastructure-asfyaml/blob/main/README.md
---
github:
description: "Apache Spark - A unified analytics engine for large-scale data processing"
diff --git a/.github/workflows/benchmark.yml b/.github/workflows/benchmark.yml
index 6b2e72b3f23be..3e90bb329be56 100644
--- a/.github/workflows/benchmark.yml
+++ b/.github/workflows/benchmark.yml
@@ -50,6 +50,11 @@ on:
description: 'Number of job splits'
required: true
default: '1'
+ create-commit:
+ type: boolean
+ description: 'Commit the benchmark results to the current branch'
+ required: true
+ default: false
jobs:
matrix-gen:
@@ -195,10 +200,31 @@ jobs:
# To keep the directory structure and file permissions, tar them
# See also https://github.com/actions/upload-artifact#maintaining-file-permissions-and-case-sensitive-files
echo "Preparing the benchmark results:"
- tar -cvf benchmark-results-${{ inputs.jdk }}-${{ inputs.scala }}.tar `git diff --name-only` `git ls-files --others --exclude=tpcds-sf-1 --exclude=tpcds-sf-1-text --exclude-standard`
+ tar -cvf target/benchmark-results-${{ inputs.jdk }}-${{ inputs.scala }}.tar `git diff --name-only` `git ls-files --others --exclude=tpcds-sf-1 --exclude=tpcds-sf-1-text --exclude-standard`
+ - name: Create a pull request with the results
+ if: ${{ inputs.create-commit && success() }}
+ run: |
+ git config --local user.name "${{ github.actor }}"
+ git config --local user.email "${{ github.event.pusher.email || format('{0}@users.noreply.github.com', github.actor) }}"
+ git add -A
+ git commit -m "Benchmark results for ${{ inputs.class }} (JDK ${{ inputs.jdk }}, Scala ${{ inputs.scala }}, split ${{ matrix.split }} of ${{ inputs.num-splits }})"
+ for i in {1..5}; do
+ echo "Attempt $i to push..."
+ git fetch origin ${{ github.ref_name }}
+ git rebase origin/${{ github.ref_name }}
+ if git push origin ${{ github.ref_name }}:${{ github.ref_name }}; then
+ echo "Push successful."
+ exit 0
+ else
+ echo "Push failed, retrying in 3 seconds..."
+ sleep 3
+ fi
+ done
+ echo "Error: Failed to push after 5 attempts."
+ exit 1
- name: Upload benchmark results
uses: actions/upload-artifact@v4
with:
name: benchmark-results-${{ inputs.jdk }}-${{ inputs.scala }}-${{ matrix.split }}
- path: benchmark-results-${{ inputs.jdk }}-${{ inputs.scala }}.tar
+ path: target/benchmark-results-${{ inputs.jdk }}-${{ inputs.scala }}.tar
diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml
index ff005103a2461..5e6d22256e2f3 100644
--- a/.github/workflows/build_and_test.yml
+++ b/.github/workflows/build_and_test.yml
@@ -112,7 +112,7 @@ jobs:
ui=false
docs=false
fi
- build=`./dev/is-changed.py -m "core,unsafe,kvstore,avro,utils,network-common,network-shuffle,repl,launcher,examples,sketch,variant,api,catalyst,hive-thriftserver,mllib-local,mllib,graphx,streaming,sql-kafka-0-10,streaming-kafka-0-10,streaming-kinesis-asl,kubernetes,hadoop-cloud,spark-ganglia-lgpl,profiler,protobuf,yarn,connect,sql,hive,pipelines"`
+ build=`./dev/is-changed.py -m "core,unsafe,kvstore,avro,utils,utils-java,network-common,network-shuffle,repl,launcher,examples,sketch,variant,api,catalyst,hive-thriftserver,mllib-local,mllib,graphx,streaming,sql-kafka-0-10,streaming-kafka-0-10,streaming-kinesis-asl,kubernetes,hadoop-cloud,spark-ganglia-lgpl,profiler,protobuf,yarn,connect,sql,hive,pipelines"`
precondition="
{
\"build\": \"$build\",
@@ -122,6 +122,8 @@ jobs:
\"tpcds-1g\": \"$tpcds\",
\"docker-integration-tests\": \"$docker\",
\"lint\" : \"true\",
+ \"java17\" : \"$build\",
+ \"java25\" : \"$build\",
\"docs\" : \"$docs\",
\"yarn\" : \"$yarn\",
\"k8s-integration-tests\" : \"$kubernetes\",
@@ -240,7 +242,7 @@ jobs:
# Note that the modules below are from sparktestsupport/modules.py.
modules:
- >-
- core, unsafe, kvstore, avro, utils,
+ core, unsafe, kvstore, avro, utils, utils-java,
network-common, network-shuffle, repl, launcher,
examples, sketch, variant
- >-
@@ -360,7 +362,7 @@ jobs:
- name: Install Python packages (Python 3.11)
if: (contains(matrix.modules, 'sql') && !contains(matrix.modules, 'sql-')) || contains(matrix.modules, 'connect') || contains(matrix.modules, 'yarn')
run: |
- python3.11 -m pip install 'numpy>=1.20.0' pyarrow pandas scipy unittest-xml-reporting 'lxml==4.9.4' 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.1'
+ python3.11 -m pip install 'numpy>=1.22' pyarrow pandas scipy unittest-xml-reporting 'lxml==4.9.4' 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.1'
python3.11 -m pip list
# Run the tests.
- name: Run tests
@@ -519,13 +521,9 @@ jobs:
- >-
pyspark-pandas-slow
- >-
- pyspark-pandas-connect-part0
+ pyspark-pandas-connect-part0, pyspark-pandas-connect-part3
- >-
- pyspark-pandas-connect-part1
- - >-
- pyspark-pandas-connect-part2
- - >-
- pyspark-pandas-connect-part3
+ pyspark-pandas-connect-part1, pyspark-pandas-connect-part2
exclude:
# Always run if pyspark == 'true', even infra-image is skip (such as non-master job)
# In practice, the build will run in individual PR, but not against the individual commit
@@ -605,8 +603,9 @@ jobs:
run: |
for py in $(echo $PYTHON_TO_TEST | tr "," "\n")
do
- echo $py
+ $py --version
$py -m pip list
+ echo ""
done
- name: Install Conda for pip packaging test
if: contains(matrix.modules, 'pyspark-errors')
@@ -919,6 +918,42 @@ jobs:
- name: R linter
run: ./dev/lint-r
+ java17:
+ needs: [precondition]
+ if: fromJson(needs.precondition.outputs.required).java17 == 'true'
+ name: Java 17 build with Maven
+ runs-on: ubuntu-latest
+ timeout-minutes: 120
+ steps:
+ - uses: actions/checkout@v4
+ - uses: actions/setup-java@v4
+ with:
+ distribution: zulu
+ java-version: 17
+ - name: Build with Maven
+ run: |
+ export MAVEN_OPTS="-Xss64m -Xmx4g -Xms4g -XX:ReservedCodeCacheSize=128m -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN"
+ export MAVEN_CLI_OPTS="--no-transfer-progress"
+ ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pkubernetes -Pvolcano -Phive -Phive-thriftserver -Phadoop-cloud -Pjvm-profiler -Pspark-ganglia-lgpl -Pkinesis-asl clean install
+
+ java25:
+ needs: [precondition]
+ if: fromJson(needs.precondition.outputs.required).java25 == 'true'
+ name: Java 25 build with Maven
+ runs-on: ubuntu-latest
+ timeout-minutes: 120
+ steps:
+ - uses: actions/checkout@v4
+ - uses: actions/setup-java@v4
+ with:
+ distribution: zulu
+ java-version: 25-ea
+ - name: Build with Maven
+ run: |
+ export MAVEN_OPTS="-Xss64m -Xmx4g -Xms4g -XX:ReservedCodeCacheSize=128m -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN"
+ export MAVEN_CLI_OPTS="--no-transfer-progress"
+ ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pkubernetes -Pvolcano -Phive -Phive-thriftserver -Phadoop-cloud -Pjvm-profiler -Pspark-ganglia-lgpl -Pkinesis-asl clean install
+
# Documentation build
docs:
needs: [precondition, infra-image]
@@ -998,10 +1033,14 @@ jobs:
# Should unpin 'sphinxcontrib-*' after upgrading sphinx>5
python3.9 -m pip install 'sphinx==4.5.0' mkdocs 'pydata_sphinx_theme>=0.13' sphinx-copybutton nbsphinx numpydoc jinja2 markupsafe 'pyzmq<24.0.0' 'sphinxcontrib-applehelp==1.0.4' 'sphinxcontrib-devhelp==1.0.2' 'sphinxcontrib-htmlhelp==2.0.1' 'sphinxcontrib-qthelp==1.0.3' 'sphinxcontrib-serializinghtml==1.1.5'
python3.9 -m pip install ipython_genutils # See SPARK-38517
- python3.9 -m pip install sphinx_plotly_directive 'numpy>=1.20.0' pyarrow pandas 'plotly<6.0.0'
+ python3.9 -m pip install sphinx_plotly_directive 'numpy>=1.22' pyarrow pandas 'plotly<6.0.0'
python3.9 -m pip install 'docutils<0.18.0' # See SPARK-39421
- - name: List Python packages
+ - name: List Python packages for branch-3.5 and branch-4.0
+ if: inputs.branch == 'branch-3.5' || inputs.branch == 'branch-4.0'
run: python3.9 -m pip list
+ - name: List Python packages
+ if: inputs.branch != 'branch-3.5' && inputs.branch != 'branch-4.0'
+ run: python3.11 -m pip list
- name: Install dependencies for documentation generation
run: |
# Keep the version of Bundler here in sync with the following locations:
@@ -1010,7 +1049,8 @@ jobs:
gem install bundler -v 2.4.22
cd docs
bundle install --retry=100
- - name: Run documentation build
+ - name: Run documentation build for branch-3.5 and branch-4.0
+ if: inputs.branch == 'branch-3.5' || inputs.branch == 'branch-4.0'
run: |
# We need this link to make sure `python3` points to `python3.9` which contains the prerequisite packages.
ln -s "$(which python3.9)" "/usr/local/bin/python3"
@@ -1031,6 +1071,30 @@ jobs:
echo "SKIP_SQLDOC: $SKIP_SQLDOC"
cd docs
bundle exec jekyll build
+ - name: Run documentation build
+ if: inputs.branch != 'branch-3.5' && inputs.branch != 'branch-4.0'
+ run: |
+ # We need this link to make sure `python3` points to `python3.11` which contains the prerequisite packages.
+ ln -s "$(which python3.11)" "/usr/local/bin/python3"
+ # Build docs first with SKIP_API to ensure they are buildable without requiring any
+ # language docs to be built beforehand.
+ cd docs; SKIP_ERRORDOC=1 SKIP_API=1 bundle exec jekyll build; cd ..
+ if [ -f "./dev/is-changed.py" ]; then
+ # Skip PySpark and SparkR docs while keeping Scala/Java/SQL docs
+ pyspark_modules=`cd dev && python3.11 -c "import sparktestsupport.modules as m; print(','.join(m.name for m in m.all_modules if m.name.startswith('pyspark')))"`
+ if [ `./dev/is-changed.py -m $pyspark_modules` = false ]; then export SKIP_PYTHONDOC=1; fi
+ if [ `./dev/is-changed.py -m sparkr` = false ]; then export SKIP_RDOC=1; fi
+ fi
+ export PYSPARK_DRIVER_PYTHON=python3.11
+ export PYSPARK_PYTHON=python3.11
+ # Print the values of environment variables `SKIP_ERRORDOC`, `SKIP_SCALADOC`, `SKIP_PYTHONDOC`, `SKIP_RDOC` and `SKIP_SQLDOC`
+ echo "SKIP_ERRORDOC: $SKIP_ERRORDOC"
+ echo "SKIP_SCALADOC: $SKIP_SCALADOC"
+ echo "SKIP_PYTHONDOC: $SKIP_PYTHONDOC"
+ echo "SKIP_RDOC: $SKIP_RDOC"
+ echo "SKIP_SQLDOC: $SKIP_SQLDOC"
+ cd docs
+ bundle exec jekyll build
- name: Tar documentation
if: github.repository != 'apache/spark'
run: tar cjf site.tar.bz2 docs/_site
@@ -1279,8 +1343,10 @@ jobs:
kubectl create clusterrolebinding serviceaccounts-cluster-admin --clusterrole=cluster-admin --group=system:serviceaccounts || true
if [[ "${{ inputs.branch }}" == 'branch-3.5' ]]; then
kubectl apply -f https://raw.githubusercontent.com/volcano-sh/volcano/v1.7.0/installer/volcano-development.yaml || true
- else
+ elif [[ "${{ inputs.branch }}" == 'branch-4.0' ]]; then
kubectl apply -f https://raw.githubusercontent.com/volcano-sh/volcano/v1.11.0/installer/volcano-development.yaml || true
+ else
+ kubectl apply -f https://raw.githubusercontent.com/volcano-sh/volcano/v1.12.2/installer/volcano-development.yaml || true
fi
eval $(minikube docker-env)
build/sbt -Phadoop-3 -Psparkr -Pkubernetes -Pvolcano -Pkubernetes-integration-tests -Dspark.kubernetes.test.volcanoMaxConcurrencyJobNum=1 -Dtest.exclude.tags=local "kubernetes-integration-tests/test"
diff --git a/.github/workflows/build_infra_images_cache.yml b/.github/workflows/build_infra_images_cache.yml
index ccd47826ff099..9ec93a4af52c1 100644
--- a/.github/workflows/build_infra_images_cache.yml
+++ b/.github/workflows/build_infra_images_cache.yml
@@ -33,7 +33,6 @@ on:
- 'dev/spark-test-image/python-minimum/Dockerfile'
- 'dev/spark-test-image/python-ps-minimum/Dockerfile'
- 'dev/spark-test-image/pypy-310/Dockerfile'
- - 'dev/spark-test-image/python-309/Dockerfile'
- 'dev/spark-test-image/python-310/Dockerfile'
- 'dev/spark-test-image/python-311/Dockerfile'
- 'dev/spark-test-image/python-311-classic-only/Dockerfile'
@@ -153,19 +152,6 @@ jobs:
- name: Image digest (PySpark with PyPy 3.10)
if: hashFiles('dev/spark-test-image/pypy-310/Dockerfile') != ''
run: echo ${{ steps.docker_build_pyspark_pypy_310.outputs.digest }}
- - name: Build and push (PySpark with Python 3.9)
- if: hashFiles('dev/spark-test-image/python-309/Dockerfile') != ''
- id: docker_build_pyspark_python_309
- uses: docker/build-push-action@v6
- with:
- context: ./dev/spark-test-image/python-309/
- push: true
- tags: ghcr.io/apache/spark/apache-spark-github-action-image-pyspark-python-309-cache:${{ github.ref_name }}-static
- cache-from: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-pyspark-python-309-cache:${{ github.ref_name }}
- cache-to: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-pyspark-python-309-cache:${{ github.ref_name }},mode=max
- - name: Image digest (PySpark with Python 3.9)
- if: hashFiles('dev/spark-test-image/python-309/Dockerfile') != ''
- run: echo ${{ steps.docker_build_pyspark_python_309.outputs.digest }}
- name: Build and push (PySpark with Python 3.10)
if: hashFiles('dev/spark-test-image/python-310/Dockerfile') != ''
id: docker_build_pyspark_python_310
diff --git a/.github/workflows/build_maven_java21_arm.yml b/.github/workflows/build_maven_java21_arm.yml
index 505bdd63189c0..16417bb1c5f22 100644
--- a/.github/workflows/build_maven_java21_arm.yml
+++ b/.github/workflows/build_maven_java21_arm.yml
@@ -21,7 +21,7 @@ name: "Build / Maven (master, Scala 2.13, Hadoop 3, JDK 21, ARM)"
on:
schedule:
- - cron: '0 15 * * *'
+ - cron: '0 15 */2 * *'
workflow_dispatch:
jobs:
diff --git a/.github/workflows/build_non_ansi.yml b/.github/workflows/build_non_ansi.yml
index 547a227e61d7e..debdaf4f8709d 100644
--- a/.github/workflows/build_non_ansi.yml
+++ b/.github/workflows/build_non_ansi.yml
@@ -40,6 +40,7 @@ jobs:
"PYSPARK_IMAGE_TO_TEST": "python-311",
"PYTHON_TO_TEST": "python3.11",
"SPARK_ANSI_SQL_MODE": "false",
+ "SPARK_TEST_SPARK_BLOOM_FILTER_SUITE_ENABLED": "true"
}
jobs: >-
{
diff --git a/.github/workflows/build_python_3.9.yml b/.github/workflows/build_python_3.9.yml
deleted file mode 100644
index 0df17699140ed..0000000000000
--- a/.github/workflows/build_python_3.9.yml
+++ /dev/null
@@ -1,47 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing,
-# software distributed under the License is distributed on an
-# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-# KIND, either express or implied. See the License for the
-# specific language governing permissions and limitations
-# under the License.
-#
-
-name: "Build / Python-only (master, Python 3.9)"
-
-on:
- schedule:
- - cron: '0 21 * * *'
- workflow_dispatch:
-
-jobs:
- run-build:
- permissions:
- packages: write
- name: Run
- uses: ./.github/workflows/build_and_test.yml
- if: github.repository == 'apache/spark'
- with:
- java: 17
- branch: master
- hadoop: hadoop3
- envs: >-
- {
- "PYSPARK_IMAGE_TO_TEST": "python-309",
- "PYTHON_TO_TEST": "python3.9"
- }
- jobs: >-
- {
- "pyspark": "true",
- "pyspark-pandas": "true"
- }
diff --git a/.github/workflows/build_python_connect.yml b/.github/workflows/build_python_connect.yml
index 8d82ff192ab07..bf247db613dba 100644
--- a/.github/workflows/build_python_connect.yml
+++ b/.github/workflows/build_python_connect.yml
@@ -72,7 +72,7 @@ jobs:
python packaging/client/setup.py sdist
cd dist
pip install pyspark*client-*.tar.gz
- pip install 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.1' 'googleapis-common-protos==1.65.0' 'graphviz==0.20.3' 'six==1.16.0' 'pandas==2.2.3' scipy 'plotly<6.0.0' 'mlflow>=2.8.1' coverage matplotlib openpyxl 'memory-profiler>=0.61.0' 'scikit-learn>=1.3.2' 'graphviz==0.20.3' 'torch<2.6.0' torchvision torcheval deepspeed unittest-xml-reporting
+ pip install 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.1' 'googleapis-common-protos==1.65.0' 'graphviz==0.20.3' 'six==1.16.0' 'pandas==2.3.2' scipy 'plotly<6.0.0' 'mlflow>=2.8.1' coverage matplotlib openpyxl 'memory-profiler>=0.61.0' 'scikit-learn>=1.3.2' 'graphviz==0.20.3' 'torch<2.6.0' torchvision torcheval deepspeed unittest-xml-reporting
- name: List Python packages
run: python -m pip list
- name: Run tests (local)
diff --git a/.github/workflows/build_python_connect35.yml b/.github/workflows/build_python_connect35.yml
index e68f288f0184f..6c37091afcb4f 100644
--- a/.github/workflows/build_python_connect35.yml
+++ b/.github/workflows/build_python_connect35.yml
@@ -68,7 +68,7 @@ jobs:
./build/sbt -Phive Test/package
- name: Install Python dependencies
run: |
- pip install 'numpy==1.25.1' 'pyarrow==12.0.1' 'pandas<=2.0.3' scipy unittest-xml-reporting 'plotly<6.0.0' 'mlflow>=2.3.1' coverage 'matplotlib==3.7.2' openpyxl 'memory-profiler==0.60.0' 'scikit-learn==1.1.*'
+ pip install 'numpy==1.25.1' 'pyarrow>=18.0.0' 'pandas<=2.0.3' scipy unittest-xml-reporting 'plotly<6.0.0' 'mlflow>=2.3.1' coverage 'matplotlib==3.7.2' openpyxl 'memory-profiler==0.60.0' 'scikit-learn==1.1.*'
# Add Python deps for Spark Connect.
pip install 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.1' 'googleapis-common-protos==1.65.0' 'graphviz==0.20.3'
diff --git a/.github/workflows/build_python_minimum.yml b/.github/workflows/build_python_minimum.yml
index 4e65503006489..3514a82f6217c 100644
--- a/.github/workflows/build_python_minimum.yml
+++ b/.github/workflows/build_python_minimum.yml
@@ -38,7 +38,7 @@ jobs:
envs: >-
{
"PYSPARK_IMAGE_TO_TEST": "python-minimum",
- "PYTHON_TO_TEST": "python3.9"
+ "PYTHON_TO_TEST": "python3.10"
}
jobs: >-
{
diff --git a/.github/workflows/build_python_ps_minimum.yml b/.github/workflows/build_python_ps_minimum.yml
index 3aa83ff06a996..ed80a904ebd7f 100644
--- a/.github/workflows/build_python_ps_minimum.yml
+++ b/.github/workflows/build_python_ps_minimum.yml
@@ -38,7 +38,7 @@ jobs:
envs: >-
{
"PYSPARK_IMAGE_TO_TEST": "python-ps-minimum",
- "PYTHON_TO_TEST": "python3.9"
+ "PYTHON_TO_TEST": "python3.10"
}
jobs: >-
{
diff --git a/.github/workflows/build_sparkr_window.yml b/.github/workflows/build_sparkr_window.yml
index e3ef9d7ba0752..8bbcdf79bd58b 100644
--- a/.github/workflows/build_sparkr_window.yml
+++ b/.github/workflows/build_sparkr_window.yml
@@ -16,7 +16,7 @@
# specific language governing permissions and limitations
# under the License.
#
-name: "Build / SparkR-only (master, 4.4.3, windows-2022)"
+name: "Build / SparkR-only (master, 4.4.3, windows-2025)"
on:
schedule:
@@ -26,7 +26,7 @@ on:
jobs:
build:
name: "Build module: sparkr"
- runs-on: windows-2022
+ runs-on: windows-2025
timeout-minutes: 120
if: github.repository == 'apache/spark'
steps:
diff --git a/.github/workflows/maven_test.yml b/.github/workflows/maven_test.yml
index e0a5e411571a0..5b1d3b47858d6 100644
--- a/.github/workflows/maven_test.yml
+++ b/.github/workflows/maven_test.yml
@@ -67,7 +67,7 @@ jobs:
- hive2.3
modules:
- >-
- core,launcher,common#unsafe,common#kvstore,common#network-common,common#network-shuffle,common#sketch,common#utils,common#variant
+ core,launcher,common#unsafe,common#kvstore,common#network-common,common#network-shuffle,common#sketch,common#utils,common#utils-java,common#variant
- >-
graphx,streaming,hadoop-cloud
- >-
@@ -181,7 +181,7 @@ jobs:
- name: Install Python packages (Python 3.11)
if: contains(matrix.modules, 'resource-managers#yarn') || (contains(matrix.modules, 'sql#core')) || contains(matrix.modules, 'connect')
run: |
- python3.11 -m pip install 'numpy>=1.20.0' pyarrow pandas scipy unittest-xml-reporting 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.1'
+ python3.11 -m pip install 'numpy>=1.22' pyarrow pandas scipy unittest-xml-reporting 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.1'
python3.11 -m pip list
# Run the tests.
- name: Run tests
@@ -190,6 +190,7 @@ jobs:
export MAVEN_OPTS="-Xss64m -Xmx4g -Xms4g -XX:ReservedCodeCacheSize=128m -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN"
export MAVEN_CLI_OPTS="--no-transfer-progress"
export JAVA_VERSION=${{ matrix.java }}
+ export INPUT_BRANCH=${{ inputs.branch }}
export ENABLE_KINESIS_TESTS=0
# Replace with the real module name, for example, connector#kafka-0-10 -> connector/kafka-0-10
export TEST_MODULES=`echo "$MODULES_TO_TEST" | sed -e "s%#%/%g"`
@@ -209,6 +210,14 @@ jobs:
elif [[ "$MODULES_TO_TEST" == *"sql#hive-thriftserver"* ]]; then
# To avoid a compilation loop, for the `sql/hive-thriftserver` module, run `clean install` instead
./build/mvn $MAVEN_CLI_OPTS -pl "$TEST_MODULES" -Pyarn -Pkubernetes -Pvolcano -Phive -Phive-thriftserver -Phadoop-cloud -Pjvm-profiler -Pspark-ganglia-lgpl -Pkinesis-asl -Djava.version=${JAVA_VERSION/-ea} clean install -fae
+ elif [[ "$MODULES_TO_TEST" == *"sql#pipelines"* && "$INPUT_BRANCH" == "branch-4.0" ]]; then
+ # SPARK-52441: Remove sql/pipelines from TEST_MODULES for branch-4.0, this branch can be deleted after the EOL of branch-4.0.
+ TEST_MODULES=${TEST_MODULES/,sql\/pipelines/}
+ ./build/mvn $MAVEN_CLI_OPTS -pl "$TEST_MODULES" -Pyarn -Pkubernetes -Pvolcano -Phive -Phive-thriftserver -Pspark-ganglia-lgpl -Phadoop-cloud -Pjvm-profiler -Pkinesis-asl -Djava.version=${JAVA_VERSION/-ea} test -fae
+ elif [[ "$MODULES_TO_TEST" == *"common#utils-java"* && "$INPUT_BRANCH" == "branch-4.0" ]]; then
+ # SPARK-53138: Remove common/utils-java from TEST_MODULES for branch-4.0, this branch can be deleted after the EOL of branch-4.0.
+ TEST_MODULES=${TEST_MODULES/,common\/utils-java/}
+ ./build/mvn $MAVEN_CLI_OPTS -pl "$TEST_MODULES" -Pyarn -Pkubernetes -Pvolcano -Phive -Phive-thriftserver -Pspark-ganglia-lgpl -Phadoop-cloud -Pjvm-profiler -Pkinesis-asl -Djava.version=${JAVA_VERSION/-ea} test -fae
else
./build/mvn $MAVEN_CLI_OPTS -pl "$TEST_MODULES" -Pyarn -Pkubernetes -Pvolcano -Phive -Phive-thriftserver -Pspark-ganglia-lgpl -Phadoop-cloud -Pjvm-profiler -Pkinesis-asl -Djava.version=${JAVA_VERSION/-ea} test -fae
fi
diff --git a/.github/workflows/pages.yml b/.github/workflows/pages.yml
index 4bcc275064d3c..8293c2429c9d1 100644
--- a/.github/workflows/pages.yml
+++ b/.github/workflows/pages.yml
@@ -52,16 +52,16 @@ jobs:
with:
distribution: zulu
java-version: 17
- - name: Install Python 3.9
+ - name: Install Python 3.11
uses: actions/setup-python@v5
with:
- python-version: '3.9'
+ python-version: '3.11'
architecture: x64
cache: 'pip'
- name: Install Python dependencies
run: |
pip install 'sphinx==4.5.0' mkdocs 'pydata_sphinx_theme>=0.13' sphinx-copybutton nbsphinx numpydoc jinja2 markupsafe 'pyzmq<24.0.0' \
- ipython ipython_genutils sphinx_plotly_directive 'numpy>=1.20.0' pyarrow 'pandas==2.2.3' 'plotly>=4.8' 'docutils<0.18.0' \
+ ipython ipython_genutils sphinx_plotly_directive 'numpy>=1.22' pyarrow 'pandas==2.3.2' 'plotly>=4.8' 'docutils<0.18.0' \
'flake8==3.9.0' 'mypy==1.8.0' 'pytest==7.1.3' 'pytest-mypy-plugins==1.9.3' 'black==23.12.1' \
'pandas-stubs==1.2.0.53' 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.1' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \
'sphinxcontrib-applehelp==1.0.4' 'sphinxcontrib-devhelp==1.0.2' 'sphinxcontrib-htmlhelp==2.0.1' 'sphinxcontrib-qthelp==1.0.3' 'sphinxcontrib-serializinghtml==1.1.5'
diff --git a/.github/workflows/python_hosted_runner_test.yml b/.github/workflows/python_hosted_runner_test.yml
index 9a87c4f7061b0..afc016186be52 100644
--- a/.github/workflows/python_hosted_runner_test.yml
+++ b/.github/workflows/python_hosted_runner_test.yml
@@ -149,7 +149,7 @@ jobs:
run: |
python${{matrix.python}} -m pip install --ignore-installed 'blinker>=1.6.2'
python${{matrix.python}} -m pip install --ignore-installed 'six==1.16.0'
- python${{matrix.python}} -m pip install numpy 'pyarrow>=19.0.0' 'six==1.16.0' 'pandas==2.2.3' scipy 'plotly<6.0.0' 'mlflow>=2.8.1' coverage matplotlib openpyxl 'memory-profiler>=0.61.0' 'scikit-learn>=1.3.2' unittest-xml-reporting && \
+ python${{matrix.python}} -m pip install numpy 'pyarrow>=21.0.0' 'six==1.16.0' 'pandas==2.3.2' scipy 'plotly<6.0.0' 'mlflow>=2.8.1' coverage matplotlib openpyxl 'memory-profiler>=0.61.0' 'scikit-learn>=1.3.2' unittest-xml-reporting && \
python${{matrix.python}} -m pip install 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.1' 'googleapis-common-protos==1.65.0' 'graphviz==0.20.3' && \
python${{matrix.python}} -m pip cache purge
- name: List Python packages
diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml
index 976aaf616295c..a6fa04b120de5 100644
--- a/.github/workflows/release.yml
+++ b/.github/workflows/release.yml
@@ -74,7 +74,8 @@ on:
inputs:
branch:
description: 'Branch to release. Leave it empty to launch a dryrun. Dispatch this workflow only in the forked repository.'
- required: false
+ required: true
+ default: master
release-version:
description: 'Release version. Leave it empty to launch a dryrun.'
required: false
@@ -90,10 +91,23 @@ jobs:
release:
name: Release Apache Spark
runs-on: ubuntu-latest
- # Do not allow dispatching this workflow manually in the main repo.
- # and skip this workflow in forked repository when running as a
- # scheduled job (dryrun).
- if: ${{ (github.repository == 'apache/spark') != (inputs.branch != '' && inputs.release-version != '') }}
+ # Allow workflow to run only in the following cases:
+ # 1. In the apache/spark repository:
+ # - Only allow dry runs (i.e., both 'branch' and 'release-version' inputs are empty).
+ # 2. In forked repositories:
+ # - Allow real runs when both 'branch' and 'release-version' are provided.
+ # - Allow dry runs only if manually dispatched (not on a schedule).
+ if: |
+ (
+ github.repository == 'apache/spark' &&
+ inputs.branch == '' &&
+ inputs.release-version == ''
+ ) || (
+ github.repository != 'apache/spark' &&
+ (
+ (inputs.branch != '' && inputs.release-version != '') || github.event_name == 'workflow_dispatch'
+ )
+ )
steps:
- name: Checkout Spark repository
uses: actions/checkout@v4
@@ -132,22 +146,12 @@ jobs:
sleep 60
fi
- empty_count=0
- non_empty_count=0
- for val in "$GIT_BRANCH" "$RELEASE_VERSION" "$SPARK_RC_COUNT"; do
- if [ -z "$val" ]; then
- empty_count=$((empty_count+1))
- else
- non_empty_count=$((non_empty_count+1))
- fi
- done
-
- if [ "$empty_count" -gt 0 ] && [ "$non_empty_count" -gt 0 ]; then
- echo "Error: Either provide all inputs or leave them all empty for a dryrun."
+ if { [ -n "$RELEASE_VERSION" ] && [ -z "$SPARK_RC_COUNT" ]; } || { [ -z "$RELEASE_VERSION" ] && [ -n "$SPARK_RC_COUNT" ]; }; then
+ echo "Error: Either provide both 'Release version' and 'RC number', or leave both empty for a dryrun."
exit 1
fi
- if [ "$empty_count" -eq 3 ]; then
+ if [ -z "$RELEASE_VERSION" ] && [ -z "$SPARK_RC_COUNT" ]; then
echo "Dry run mode enabled"
export DRYRUN_MODE=1
ASF_PASSWORD="not_used"
@@ -155,7 +159,6 @@ jobs:
GPG_PASSPHRASE="not_used"
ASF_USERNAME="gurwls223"
export SKIP_TAG=1
- unset GIT_BRANCH
unset RELEASE_VERSION
else
echo "Full release mode enabled"
@@ -163,7 +166,7 @@ jobs:
fi
export ASF_PASSWORD GPG_PRIVATE_KEY GPG_PASSPHRASE ASF_USERNAME
- [ -n "$GIT_BRANCH" ] && export GIT_BRANCH
+ export GIT_BRANCH="${GIT_BRANCH:-master}"
[ -n "$RELEASE_VERSION" ] && export RELEASE_VERSION
if [ "$DRYRUN_MODE" = "1" ]; then
@@ -237,9 +240,18 @@ jobs:
cp "$file" "$file.bak"
for pattern in "${PATTERNS[@]}"; do
[ -n "$pattern" ] || continue # Skip empty patterns
- escaped_pattern=$(printf '%s\n' "$pattern" | sed 's/[\/&]/\\&/g')
- sed -i "s/${escaped_pattern}/***/g" "$file"
+
+ # Safely escape special characters for sed
+ escaped_pattern=${pattern//\\/\\\\} # Escape backslashes
+ escaped_pattern=${escaped_pattern//\//\\/} # Escape forward slashes
+ escaped_pattern=${escaped_pattern//&/\\&} # Escape &
+ escaped_pattern=${escaped_pattern//$'\n'/} # Remove newlines
+ escaped_pattern=${escaped_pattern//$'\r'/} # Remove carriage returns (optional)
+
+ # Redact the pattern
+ sed -i.bak "s/${escaped_pattern}/***/g" "$file"
done
+ rm -f "$file.bak"
done
# Zip logs/output
diff --git a/.gitignore b/.gitignore
index b6a1e63c41920..bbf02496498c1 100644
--- a/.gitignore
+++ b/.gitignore
@@ -125,3 +125,7 @@ sql/api/gen/
sql/api/src/main/gen/
sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.tokens
sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/gen/
+
+tpcds-sf-1/
+tpcds-sf-1-text/
+tpcds-kit/
diff --git a/connect-examples/server-library-example/client/src/main/resources/log4j2.xml b/.mvn/extensions.xml
similarity index 72%
rename from connect-examples/server-library-example/client/src/main/resources/log4j2.xml
rename to .mvn/extensions.xml
index 21b0d9719193e..da2aee8827a54 100644
--- a/connect-examples/server-library-example/client/src/main/resources/log4j2.xml
+++ b/.mvn/extensions.xml
@@ -15,16 +15,10 @@
~ See the License for the specific language governing permissions and
~ limitations under the License.
-->
-
-
-
-
-
-
-
-
-
-
-
-
-
+
+
+ eu.maveniverse.maven.nisse
+ extension
+ 0.4.6
+
+
diff --git a/.mvn/jvm.config b/.mvn/jvm.config
index 81b88d8173419..b41dca0e56a6b 100644
--- a/.mvn/jvm.config
+++ b/.mvn/jvm.config
@@ -1 +1,4 @@
+-XX:+IgnoreUnrecognizedVMOptions
+-XX:+UnlockDiagnosticVMOptions
+-XX:GCLockerRetryAllocationCount=100
--enable-native-access=ALL-UNNAMED
diff --git a/.mvn/maven.config b/.mvn/maven.config
new file mode 100644
index 0000000000000..e61f1a94abdef
--- /dev/null
+++ b/.mvn/maven.config
@@ -0,0 +1 @@
+-Dnisse.compat.osDetector
diff --git a/LICENSE b/LICENSE
index 9b1e96a44a58c..ef8192ec38d05 100644
--- a/LICENSE
+++ b/LICENSE
@@ -217,6 +217,7 @@ core/src/main/resources/org/apache/spark/ui/static/vis*
connector/spark-ganglia-lgpl/src/main/java/com/codahale/metrics/ganglia/GangliaReporter.java
core/src/main/resources/org/apache/spark/ui/static/d3-flamegraph.min.js
core/src/main/resources/org/apache/spark/ui/static/d3-flamegraph.css
+mllib-local/src/main/scala/scala/collection/compat/package.scala
Python Software Foundation License
----------------------------------
diff --git a/LICENSE-binary b/LICENSE-binary
index 0c3c7aecb71ac..6f6bf9db5448a 100644
--- a/LICENSE-binary
+++ b/LICENSE-binary
@@ -365,8 +365,6 @@ org.apache.xbean:xbean-asm9-shaded
org.apache.yetus:audience-annotations
org.apache.zookeeper:zookeeper
org.apache.zookeeper:zookeeper-jute
-org.codehaus.jackson:jackson-core-asl
-org.codehaus.jackson:jackson-mapper-asl
org.datanucleus:datanucleus-api-jdo
org.datanucleus:datanucleus-core
org.datanucleus:datanucleus-rdbms
@@ -388,7 +386,6 @@ org.glassfish.jersey.core:jersey-common
org.glassfish.jersey.core:jersey-server
org.glassfish.jersey.inject:jersey-hk2
org.javassist:javassist
-org.jetbrains:annotations
org.json4s:json4s-ast_2.13
org.json4s:json4s-core_2.13
org.json4s:json4s-jackson-core_2.13
@@ -401,7 +398,6 @@ org.rocksdb:rocksdbjni
org.scala-lang:scala-compiler
org.scala-lang:scala-library
org.scala-lang:scala-reflect
-org.scala-lang.modules:scala-collection-compat_2.13
org.scala-lang.modules:scala-parallel-collections_2.13
org.scala-lang.modules:scala-parser-combinators_2.13
org.scala-lang.modules:scala-xml_2.13
@@ -442,7 +438,6 @@ com.github.luben:zstd-jni
com.github.wendykierp:JTransforms
javolution:javolution
jline:jline
-org.jodd:jodd-core
pl.edu.icm:JLargeArrays
python/pyspark/errors/exceptions/tblib.py
@@ -465,6 +460,7 @@ org.codehaus.janino:janino
org.fusesource.leveldbjni:leveldbjni-all
org.jline:jline
org.jpmml:pmml-model
+org.locationtech.jts:jts-core
org.threeten:threeten-extra
python/lib/py4j-*-src.zip
diff --git a/README.md b/README.md
index 0f0bf039550d7..17344563d90ac 100644
--- a/README.md
+++ b/README.md
@@ -36,7 +36,6 @@ This README file only contains basic setup instructions.
| | [](https://github.com/apache/spark/actions/workflows/build_maven_java21_arm.yml) |
| | [](https://github.com/apache/spark/actions/workflows/build_coverage.yml) |
| | [](https://github.com/apache/spark/actions/workflows/build_python_pypy3.10.yml) |
-| | [](https://github.com/apache/spark/actions/workflows/build_python_3.9.yml) |
| | [](https://github.com/apache/spark/actions/workflows/build_python_3.10.yml) |
| | [](https://github.com/apache/spark/actions/workflows/build_python_3.11_classic_only.yml) |
| | [](https://github.com/apache/spark/actions/workflows/build_python_3.11_arm.yml) |
diff --git a/assembly/pom.xml b/assembly/pom.xml
index a85ac5d9bc837..3a6b73b4522db 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -306,13 +306,6 @@
providedprovided
- provided
-
-
-
- hive-jackson-provided
-
- provided
diff --git a/bin/spark-pipelines b/bin/spark-pipelines
index 52baeeafab08a..2174df7bed699 100755
--- a/bin/spark-pipelines
+++ b/bin/spark-pipelines
@@ -30,4 +30,4 @@ fi
export PYTHONPATH="${SPARK_HOME}/python/:$PYTHONPATH"
export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.9.9-src.zip:$PYTHONPATH"
-$PYSPARK_PYTHON "${SPARK_HOME}"/python/pyspark/pipelines/cli.py "$@"
+exec "${SPARK_HOME}"/bin/spark-class org.apache.spark.deploy.SparkPipelines "$@"
diff --git a/build/sbt b/build/sbt
index db9d3b345ff6f..7644c143ad423 100755
--- a/build/sbt
+++ b/build/sbt
@@ -36,7 +36,7 @@ fi
declare -r noshare_opts="-Dsbt.global.base=project/.sbtboot -Dsbt.boot.directory=project/.boot -Dsbt.ivy.home=project/.ivy"
declare -r sbt_opts_file=".sbtopts"
declare -r etc_sbt_opts_file="/etc/sbt/sbtopts"
-declare -r default_sbt_opts="-Xss64m"
+declare -r default_sbt_opts="-Xss64m -XX:+IgnoreUnrecognizedVMOptions -XX:+UnlockDiagnosticVMOptions -XX:GCLockerRetryAllocationCount=100"
usage() {
cat <slf4j-api
-
- commons-io
- commons-io
- test
-
-
- org.apache.commons
- commons-lang3
- test
-
-
org.apache.logging.log4jlog4j-api
diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/ArrayWrappers.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/ArrayWrappers.java
index 5265881e990e9..a9d6784805f6d 100644
--- a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/ArrayWrappers.java
+++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/ArrayWrappers.java
@@ -19,7 +19,7 @@
import java.util.Arrays;
-import com.google.common.base.Preconditions;
+import org.apache.spark.network.util.JavaUtils;
/**
* A factory for array wrappers so that arrays can be used as keys in a map, sorted or not.
@@ -38,7 +38,7 @@ class ArrayWrappers {
@SuppressWarnings("unchecked")
public static Comparable
-
- org.apache.commons
- commons-lang3
- ${leveldbjni.group}leveldbjni-all
@@ -174,7 +170,7 @@
org.apache.spark
- spark-common-utils_${scala.binary.version}
+ spark-common-utils-java_${scala.binary.version}${project.version}
diff --git a/common/network-common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java b/common/network-common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java
index dd7c2061ec95b..eed43a8d28d90 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java
@@ -26,11 +26,8 @@
import java.nio.channels.FileChannel;
import java.nio.file.StandardOpenOption;
-import com.google.common.io.ByteStreams;
import io.netty.channel.DefaultFileRegion;
import io.netty.handler.stream.ChunkedStream;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
import org.apache.spark.network.util.JavaUtils;
import org.apache.spark.network.util.LimitedInputStream;
@@ -100,7 +97,7 @@ public InputStream createInputStream() throws IOException {
boolean shouldClose = true;
try {
is = new FileInputStream(file);
- ByteStreams.skipFully(is, offset);
+ is.skipNBytes(offset);
InputStream r = new LimitedInputStream(is, length);
shouldClose = false;
return r;
@@ -152,10 +149,7 @@ public Object convertToNettyForSsl() throws IOException {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("file", file)
- .append("offset", offset)
- .append("length", length)
- .toString();
+ return "FileSegmentManagedBuffer[file=" + file + ",offset=" + offset +
+ ",length=" + length + "]";
}
}
diff --git a/common/network-common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java b/common/network-common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java
index a40cfc8bc04b1..e7b8bafa92f4b 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java
@@ -23,8 +23,6 @@
import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufInputStream;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
/**
* A {@link ManagedBuffer} backed by a Netty {@link ByteBuf}.
@@ -75,8 +73,6 @@ public Object convertToNettyForSsl() throws IOException {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("buf", buf)
- .toString();
+ return "NettyManagedBuffer[buf=" + buf + "]";
}
}
diff --git a/common/network-common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java b/common/network-common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java
index 6eb8d4e2c731c..d97f853c58f1f 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java
@@ -23,8 +23,6 @@
import io.netty.buffer.ByteBufInputStream;
import io.netty.buffer.Unpooled;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
/**
* A {@link ManagedBuffer} backed by {@link ByteBuffer}.
@@ -73,9 +71,7 @@ public Object convertToNettyForSsl() throws IOException {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("buf", buf)
- .toString();
+ return "NioManagedBuffer[buf=" + buf + "]";
}
}
diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java
index a9df47645d36f..f02f2c63ecd4c 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java
@@ -21,20 +21,17 @@
import java.io.IOException;
import java.net.SocketAddress;
import java.nio.ByteBuffer;
+import java.util.Objects;
import java.util.UUID;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import javax.annotation.Nullable;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Throwables;
import com.google.common.util.concurrent.SettableFuture;
import io.netty.channel.Channel;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.GenericFutureListener;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;
@@ -43,6 +40,7 @@
import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.buffer.NioManagedBuffer;
import org.apache.spark.network.protocol.*;
+import org.apache.spark.network.util.JavaUtils;
import static org.apache.spark.network.util.NettyUtils.getRemoteAddress;
@@ -81,8 +79,8 @@ public class TransportClient implements Closeable {
private volatile boolean timedOut;
public TransportClient(Channel channel, TransportResponseHandler handler) {
- this.channel = Preconditions.checkNotNull(channel);
- this.handler = Preconditions.checkNotNull(handler);
+ this.channel = Objects.requireNonNull(channel);
+ this.handler = Objects.requireNonNull(handler);
this.timedOut = false;
}
@@ -113,7 +111,7 @@ public String getClientId() {
* Trying to set a different client ID after it's been set will result in an exception.
*/
public void setClientId(String id) {
- Preconditions.checkState(clientId == null, "Client ID has already been set.");
+ JavaUtils.checkState(clientId == null, "Client ID has already been set.");
this.clientId = id;
}
@@ -290,10 +288,9 @@ public void onFailure(Throwable e) {
try {
return result.get(timeoutMs, TimeUnit.MILLISECONDS);
} catch (ExecutionException e) {
- Throwables.throwIfUnchecked(e.getCause());
throw new RuntimeException(e.getCause());
} catch (Exception e) {
- Throwables.throwIfUnchecked(e);
+ if (e instanceof RuntimeException re) throw re;
throw new RuntimeException(e);
}
}
@@ -338,11 +335,8 @@ public void close() {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("remoteAddress", channel.remoteAddress())
- .append("clientId", clientId)
- .append("isActive", isActive())
- .toString();
+ return "TransportClient[remoteAddress=" + channel.remoteAddress() + "clientId=" + clientId +
+ ",isActive=" + isActive() + "]";
}
private static long requestId() {
@@ -369,8 +363,8 @@ public void operationComplete(Future super Void> future) throws Exception {
}
} else {
logger.error("Failed to send RPC {} to {}", future.cause(),
- MDC.of(LogKeys.REQUEST_ID$.MODULE$, requestId),
- MDC.of(LogKeys.HOST_PORT$.MODULE$, getRemoteAddress(channel)));
+ MDC.of(LogKeys.REQUEST_ID, requestId),
+ MDC.of(LogKeys.HOST_PORT, getRemoteAddress(channel)));
channel.close();
try {
String errorMsg = String.format("Failed to send RPC %s to %s: %s", requestId,
diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java
index d64b8c8f838e9..2137b5f3136ef 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java
@@ -21,16 +21,15 @@
import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.SocketAddress;
+import java.util.ArrayList;
import java.util.List;
+import java.util.Objects;
import java.util.Random;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicReference;
import com.codahale.metrics.MetricSet;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Throwables;
-import com.google.common.collect.Lists;
import io.netty.bootstrap.Bootstrap;
import io.netty.buffer.PooledByteBufAllocator;
import io.netty.channel.Channel;
@@ -100,9 +99,9 @@ private static class ClientPool {
public TransportClientFactory(
TransportContext context,
List clientBootstraps) {
- this.context = Preconditions.checkNotNull(context);
+ this.context = Objects.requireNonNull(context);
this.conf = context.getConf();
- this.clientBootstraps = Lists.newArrayList(Preconditions.checkNotNull(clientBootstraps));
+ this.clientBootstraps = new ArrayList<>(Objects.requireNonNull(clientBootstraps));
this.connectionPool = new ConcurrentHashMap<>();
this.numConnectionsPerPeer = conf.numConnectionsPerPeer();
this.rand = new Random();
@@ -193,9 +192,9 @@ public TransportClient createClient(String remoteHost, int remotePort, boolean f
final String resolvMsg = resolvedAddress.isUnresolved() ? "failed" : "succeed";
if (hostResolveTimeMs > 2000) {
logger.warn("DNS resolution {} for {} took {} ms",
- MDC.of(LogKeys.STATUS$.MODULE$, resolvMsg),
- MDC.of(LogKeys.HOST_PORT$.MODULE$, resolvedAddress),
- MDC.of(LogKeys.TIME$.MODULE$, hostResolveTimeMs));
+ MDC.of(LogKeys.STATUS, resolvMsg),
+ MDC.of(LogKeys.HOST_PORT, resolvedAddress),
+ MDC.of(LogKeys.TIME, hostResolveTimeMs));
} else {
logger.trace("DNS resolution {} for {} took {} ms",
resolvMsg, resolvedAddress, hostResolveTimeMs);
@@ -210,7 +209,7 @@ public TransportClient createClient(String remoteHost, int remotePort, boolean f
return cachedClient;
} else {
logger.info("Found inactive connection to {}, creating a new one.",
- MDC.of(LogKeys.HOST_PORT$.MODULE$, resolvedAddress));
+ MDC.of(LogKeys.HOST_PORT, resolvedAddress));
}
}
// If this connection should fast fail when last connection failed in last fast fail time
@@ -314,7 +313,7 @@ public void operationComplete(final Future handshakeFuture) {
logger.debug("{} successfully completed TLS handshake to ", address);
} else {
logger.info("failed to complete TLS handshake to {}", handshakeFuture.cause(),
- MDC.of(LogKeys.HOST_PORT$.MODULE$, address));
+ MDC.of(LogKeys.HOST_PORT, address));
cf.channel().close();
}
}
@@ -340,17 +339,17 @@ public void operationComplete(final Future handshakeFuture) {
} catch (Exception e) { // catch non-RuntimeExceptions too as bootstrap may be written in Scala
long bootstrapTimeMs = (System.nanoTime() - preBootstrap) / 1000000;
logger.error("Exception while bootstrapping client after {} ms", e,
- MDC.of(LogKeys.BOOTSTRAP_TIME$.MODULE$, bootstrapTimeMs));
+ MDC.of(LogKeys.BOOTSTRAP_TIME, bootstrapTimeMs));
client.close();
- Throwables.throwIfUnchecked(e);
+ if (e instanceof RuntimeException re) throw re;
throw new RuntimeException(e);
}
long postBootstrap = System.nanoTime();
logger.info("Successfully created connection to {} after {} ms ({} ms spent in bootstraps)",
- MDC.of(LogKeys.HOST_PORT$.MODULE$, address),
- MDC.of(LogKeys.ELAPSED_TIME$.MODULE$, (postBootstrap - preConnect) / 1000000),
- MDC.of(LogKeys.BOOTSTRAP_TIME$.MODULE$, (postBootstrap - preBootstrap) / 1000000));
+ MDC.of(LogKeys.HOST_PORT, address),
+ MDC.of(LogKeys.ELAPSED_TIME, (postBootstrap - preConnect) / 1000000),
+ MDC.of(LogKeys.BOOTSTRAP_TIME, (postBootstrap - preBootstrap) / 1000000));
return client;
}
diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java
index be4cf4a58abeb..d27fa08d829bb 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java
@@ -26,8 +26,6 @@
import com.google.common.annotations.VisibleForTesting;
import io.netty.channel.Channel;
-import org.apache.commons.lang3.tuple.ImmutablePair;
-import org.apache.commons.lang3.tuple.Pair;
import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;
@@ -45,6 +43,7 @@
import org.apache.spark.network.server.MessageHandler;
import static org.apache.spark.network.util.NettyUtils.getRemoteAddress;
import org.apache.spark.network.util.TransportFrameDecoder;
+import org.apache.spark.util.Pair;
/**
* Handler that processes server responses, in response to requests issued from a
@@ -96,7 +95,7 @@ public void removeRpcRequest(long requestId) {
public void addStreamCallback(String streamId, StreamCallback callback) {
updateTimeOfLastRequest();
- streamCallbacks.offer(ImmutablePair.of(streamId, callback));
+ streamCallbacks.offer(Pair.of(streamId, callback));
}
@VisibleForTesting
@@ -125,7 +124,7 @@ private void failOutstandingRequests(Throwable cause) {
}
for (Pair entry : streamCallbacks) {
try {
- entry.getValue().onFailure(entry.getKey(), cause);
+ entry.getRight().onFailure(entry.getLeft(), cause);
} catch (Exception e) {
logger.warn("StreamCallback.onFailure throws exception", e);
}
@@ -146,8 +145,8 @@ public void channelInactive() {
if (hasOutstandingRequests()) {
String remoteAddress = getRemoteAddress(channel);
logger.error("Still have {} requests outstanding when connection from {} is closed",
- MDC.of(LogKeys.COUNT$.MODULE$, numOutstandingRequests()),
- MDC.of(LogKeys.HOST_PORT$.MODULE$, remoteAddress));
+ MDC.of(LogKeys.COUNT, numOutstandingRequests()),
+ MDC.of(LogKeys.HOST_PORT, remoteAddress));
failOutstandingRequests(new IOException("Connection from " + remoteAddress + " closed"));
}
}
@@ -157,8 +156,8 @@ public void exceptionCaught(Throwable cause) {
if (hasOutstandingRequests()) {
String remoteAddress = getRemoteAddress(channel);
logger.error("Still have {} requests outstanding when connection from {} is closed",
- MDC.of(LogKeys.COUNT$.MODULE$, numOutstandingRequests()),
- MDC.of(LogKeys.HOST_PORT$.MODULE$, remoteAddress));
+ MDC.of(LogKeys.COUNT, numOutstandingRequests()),
+ MDC.of(LogKeys.HOST_PORT, remoteAddress));
failOutstandingRequests(cause);
}
}
@@ -169,8 +168,8 @@ public void handle(ResponseMessage message) throws Exception {
ChunkReceivedCallback listener = outstandingFetches.get(resp.streamChunkId);
if (listener == null) {
logger.warn("Ignoring response for block {} from {} since it is not outstanding",
- MDC.of(LogKeys.STREAM_CHUNK_ID$.MODULE$, resp.streamChunkId),
- MDC.of(LogKeys.HOST_PORT$.MODULE$, getRemoteAddress(channel)));
+ MDC.of(LogKeys.STREAM_CHUNK_ID, resp.streamChunkId),
+ MDC.of(LogKeys.HOST_PORT, getRemoteAddress(channel)));
resp.body().release();
} else {
outstandingFetches.remove(resp.streamChunkId);
@@ -181,9 +180,9 @@ public void handle(ResponseMessage message) throws Exception {
ChunkReceivedCallback listener = outstandingFetches.get(resp.streamChunkId);
if (listener == null) {
logger.warn("Ignoring response for block {} from {} ({}) since it is not outstanding",
- MDC.of(LogKeys.STREAM_CHUNK_ID$.MODULE$, resp.streamChunkId),
- MDC.of(LogKeys.HOST_PORT$.MODULE$, getRemoteAddress(channel)),
- MDC.of(LogKeys.ERROR$.MODULE$, resp.errorString));
+ MDC.of(LogKeys.STREAM_CHUNK_ID, resp.streamChunkId),
+ MDC.of(LogKeys.HOST_PORT, getRemoteAddress(channel)),
+ MDC.of(LogKeys.ERROR, resp.errorString));
} else {
outstandingFetches.remove(resp.streamChunkId);
listener.onFailure(resp.streamChunkId.chunkIndex(), new ChunkFetchFailureException(
@@ -193,9 +192,9 @@ public void handle(ResponseMessage message) throws Exception {
RpcResponseCallback listener = (RpcResponseCallback) outstandingRpcs.get(resp.requestId);
if (listener == null) {
logger.warn("Ignoring response for RPC {} from {} ({} bytes) since it is not outstanding",
- MDC.of(LogKeys.REQUEST_ID$.MODULE$, resp.requestId),
- MDC.of(LogKeys.HOST_PORT$.MODULE$, getRemoteAddress(channel)),
- MDC.of(LogKeys.RESPONSE_BODY_SIZE$.MODULE$, resp.body().size()));
+ MDC.of(LogKeys.REQUEST_ID, resp.requestId),
+ MDC.of(LogKeys.HOST_PORT, getRemoteAddress(channel)),
+ MDC.of(LogKeys.RESPONSE_BODY_SIZE, resp.body().size()));
resp.body().release();
} else {
outstandingRpcs.remove(resp.requestId);
@@ -209,9 +208,9 @@ public void handle(ResponseMessage message) throws Exception {
BaseResponseCallback listener = outstandingRpcs.get(resp.requestId);
if (listener == null) {
logger.warn("Ignoring response for RPC {} from {} ({}) since it is not outstanding",
- MDC.of(LogKeys.REQUEST_ID$.MODULE$, resp.requestId),
- MDC.of(LogKeys.HOST_PORT$.MODULE$, getRemoteAddress(channel)),
- MDC.of(LogKeys.ERROR$.MODULE$, resp.errorString));
+ MDC.of(LogKeys.REQUEST_ID, resp.requestId),
+ MDC.of(LogKeys.HOST_PORT, getRemoteAddress(channel)),
+ MDC.of(LogKeys.ERROR, resp.errorString));
} else {
outstandingRpcs.remove(resp.requestId);
listener.onFailure(new RuntimeException(resp.errorString));
@@ -223,9 +222,9 @@ public void handle(ResponseMessage message) throws Exception {
if (listener == null) {
logger.warn("Ignoring response for MergedBlockMetaRequest {} from {} ({} bytes) since "
+ "it is not outstanding",
- MDC.of(LogKeys.REQUEST_ID$.MODULE$, resp.requestId),
- MDC.of(LogKeys.HOST_PORT$.MODULE$, getRemoteAddress(channel)),
- MDC.of(LogKeys.RESPONSE_BODY_SIZE$.MODULE$, resp.body().size()));
+ MDC.of(LogKeys.REQUEST_ID, resp.requestId),
+ MDC.of(LogKeys.HOST_PORT, getRemoteAddress(channel)),
+ MDC.of(LogKeys.RESPONSE_BODY_SIZE, resp.body().size()));
} else {
outstandingRpcs.remove(resp.requestId);
listener.onSuccess(resp.getNumChunks(), resp.body());
@@ -236,7 +235,7 @@ public void handle(ResponseMessage message) throws Exception {
} else if (message instanceof StreamResponse resp) {
Pair entry = streamCallbacks.poll();
if (entry != null) {
- StreamCallback callback = entry.getValue();
+ StreamCallback callback = entry.getRight();
if (resp.byteCount > 0) {
StreamInterceptor interceptor = new StreamInterceptor<>(
this, resp.streamId, resp.byteCount, callback);
@@ -262,7 +261,7 @@ public void handle(ResponseMessage message) throws Exception {
} else if (message instanceof StreamFailure resp) {
Pair entry = streamCallbacks.poll();
if (entry != null) {
- StreamCallback callback = entry.getValue();
+ StreamCallback callback = entry.getRight();
try {
callback.onFailure(resp.streamId, new RuntimeException(resp.error));
} catch (IOException ioe) {
@@ -270,7 +269,7 @@ public void handle(ResponseMessage message) throws Exception {
}
} else {
logger.warn("Stream failure with unknown callback: {}",
- MDC.of(LogKeys.ERROR$.MODULE$, resp.error));
+ MDC.of(LogKeys.ERROR, resp.error));
}
} else {
throw new IllegalStateException("Unknown response type: " + message.type());
diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java
index 8449a774a404a..f02fbc3aa26c7 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java
@@ -21,10 +21,10 @@
import java.io.Closeable;
import java.security.GeneralSecurityException;
import java.util.Arrays;
+import java.util.Objects;
import java.util.Properties;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
import com.google.common.primitives.Bytes;
import com.google.crypto.tink.subtle.AesGcmJce;
import com.google.crypto.tink.subtle.Hkdf;
@@ -33,6 +33,8 @@
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import static java.nio.charset.StandardCharsets.UTF_8;
+
+import org.apache.spark.network.util.JavaUtils;
import org.apache.spark.network.util.TransportConf;
/**
@@ -61,10 +63,8 @@ class AuthEngine implements Closeable {
private TransportCipher sessionCipher;
AuthEngine(String appId, String preSharedSecret, TransportConf conf) {
- Preconditions.checkNotNull(appId);
- Preconditions.checkNotNull(preSharedSecret);
- this.appId = appId;
- this.preSharedSecret = preSharedSecret.getBytes(UTF_8);
+ this.appId = Objects.requireNonNull(appId);
+ this.preSharedSecret = Objects.requireNonNull(preSharedSecret).getBytes(UTF_8);
this.conf = conf;
this.cryptoConf = conf.cryptoConf();
// This is for backward compatibility with version 1.0 of this protocol,
@@ -126,7 +126,7 @@ private AuthMessage encryptEphemeralPublicKey(
private byte[] decryptEphemeralPublicKey(
AuthMessage encryptedPublicKey,
byte[] transcript) throws GeneralSecurityException {
- Preconditions.checkArgument(appId.equals(encryptedPublicKey.appId()));
+ JavaUtils.checkArgument(appId.equals(encryptedPublicKey.appId()), "appID is different.");
// Mix in the app ID, salt, and transcript into HKDF and use it as AES-GCM AAD
byte[] aadState = Bytes.concat(appId.getBytes(UTF_8), encryptedPublicKey.salt(), transcript);
// Use HKDF to derive an AES_GCM key from the pre-shared key, non-secret salt, and AAD state
@@ -162,7 +162,7 @@ AuthMessage challenge() throws GeneralSecurityException {
* @return An encrypted server ephemeral public key to be sent to the client.
*/
AuthMessage response(AuthMessage encryptedClientPublicKey) throws GeneralSecurityException {
- Preconditions.checkArgument(appId.equals(encryptedClientPublicKey.appId()));
+ JavaUtils.checkArgument(appId.equals(encryptedClientPublicKey.appId()), "appId is different.");
// Compute a shared secret given the client public key and the server private key
byte[] clientPublicKey =
decryptEphemeralPublicKey(encryptedClientPublicKey, EMPTY_TRANSCRIPT);
@@ -190,8 +190,7 @@ AuthMessage response(AuthMessage encryptedClientPublicKey) throws GeneralSecurit
*/
void deriveSessionCipher(AuthMessage encryptedClientPublicKey,
AuthMessage encryptedServerPublicKey) throws GeneralSecurityException {
- Preconditions.checkArgument(appId.equals(encryptedClientPublicKey.appId()));
- Preconditions.checkArgument(appId.equals(encryptedServerPublicKey.appId()));
+ JavaUtils.checkArgument(appId.equals(encryptedClientPublicKey.appId()), "appId is different.");
// Compute a shared secret given the server public key and the client private key,
// mixing in the protocol transcript.
byte[] serverPublicKey = decryptEphemeralPublicKey(
@@ -252,7 +251,7 @@ private byte[] getTranscript(AuthMessage... encryptedPublicKeys) {
}
TransportCipher sessionCipher() {
- Preconditions.checkState(sessionCipher != null);
+ JavaUtils.checkState(sessionCipher != null, "sessionCipher is null.");
return sessionCipher;
}
diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java
index 087e3d21e22bb..8ce4680f32437 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java
@@ -20,8 +20,6 @@
import java.nio.ByteBuffer;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Throwables;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.channel.Channel;
@@ -36,6 +34,7 @@
import org.apache.spark.network.sasl.SaslRpcHandler;
import org.apache.spark.network.server.AbstractAuthRpcHandler;
import org.apache.spark.network.server.RpcHandler;
+import org.apache.spark.network.util.JavaUtils;
import org.apache.spark.network.util.TransportConf;
/**
@@ -93,7 +92,7 @@ protected boolean doAuthChallenge(
} catch (RuntimeException e) {
if (conf.saslFallback()) {
LOG.warn("Failed to parse new auth challenge, reverting to SASL for client {}.",
- MDC.of(LogKeys.HOST_PORT$.MODULE$, channel.remoteAddress()));
+ MDC.of(LogKeys.HOST_PORT, channel.remoteAddress()));
saslHandler = new SaslRpcHandler(conf, channel, null, secretKeyHolder);
message.position(position);
message.limit(limit);
@@ -111,7 +110,7 @@ protected boolean doAuthChallenge(
AuthEngine engine = null;
try {
String secret = secretKeyHolder.getSecretKey(challenge.appId());
- Preconditions.checkState(secret != null,
+ JavaUtils.checkState(secret != null,
"Trying to authenticate non-registered app %s.", challenge.appId());
LOG.debug("Authenticating challenge for app {}.", challenge.appId());
engine = new AuthEngine(challenge.appId(), secret, conf);
@@ -132,7 +131,7 @@ protected boolean doAuthChallenge(
try {
engine.close();
} catch (Exception e) {
- Throwables.throwIfUnchecked(e);
+ if (e instanceof RuntimeException re) throw re;
throw new RuntimeException(e);
}
}
diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/CtrTransportCipher.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/CtrTransportCipher.java
index 85b893751b39c..de7d1ae5753d9 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/crypto/CtrTransportCipher.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/CtrTransportCipher.java
@@ -27,7 +27,6 @@
import javax.crypto.spec.IvParameterSpec;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.channel.*;
@@ -37,6 +36,7 @@
import org.apache.spark.network.util.AbstractFileRegion;
import org.apache.spark.network.util.ByteArrayReadableChannel;
import org.apache.spark.network.util.ByteArrayWritableChannel;
+import org.apache.spark.network.util.JavaUtils;
/**
* Cipher for encryption and decryption.
@@ -239,7 +239,7 @@ static class EncryptedMessage extends AbstractFileRegion {
Object msg,
ByteArrayWritableChannel byteEncChannel,
ByteArrayWritableChannel byteRawChannel) {
- Preconditions.checkArgument(msg instanceof ByteBuf || msg instanceof FileRegion,
+ JavaUtils.checkArgument(msg instanceof ByteBuf || msg instanceof FileRegion,
"Unrecognized message type: %s", msg.getClass().getName());
this.handler = handler;
this.isByteBuf = msg instanceof ByteBuf;
@@ -304,7 +304,7 @@ public boolean release(int decrement) {
@Override
public long transferTo(WritableByteChannel target, long position) throws IOException {
- Preconditions.checkArgument(position == transferred(), "Invalid position.");
+ JavaUtils.checkArgument(position == transferred(), "Invalid position.");
if (transferred == count) {
return 0;
diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/GcmTransportCipher.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/GcmTransportCipher.java
index c3540838bef09..e1cf22a612ea4 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/crypto/GcmTransportCipher.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/GcmTransportCipher.java
@@ -18,15 +18,16 @@
package org.apache.spark.network.crypto;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
import com.google.common.primitives.Longs;
import com.google.crypto.tink.subtle.*;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.channel.*;
import io.netty.util.ReferenceCounted;
+
import org.apache.spark.network.util.AbstractFileRegion;
import org.apache.spark.network.util.ByteBufferWriteableChannel;
+import org.apache.spark.network.util.JavaUtils;
import javax.crypto.spec.SecretKeySpec;
import java.io.IOException;
@@ -118,7 +119,7 @@ static class GcmEncryptedMessage extends AbstractFileRegion {
Object plaintextMessage,
ByteBuffer plaintextBuffer,
ByteBuffer ciphertextBuffer) throws GeneralSecurityException {
- Preconditions.checkArgument(
+ JavaUtils.checkArgument(
plaintextMessage instanceof ByteBuf || plaintextMessage instanceof FileRegion,
"Unrecognized message type: %s", plaintextMessage.getClass().getName());
this.plaintextMessage = plaintextMessage;
@@ -221,10 +222,12 @@ public long transferTo(WritableByteChannel target, long position) throws IOExcep
int readLimit =
(int) Math.min(readableBytes, plaintextBuffer.remaining());
if (plaintextMessage instanceof ByteBuf byteBuf) {
- Preconditions.checkState(0 == plaintextBuffer.position());
+ JavaUtils.checkState(0 == plaintextBuffer.position(),
+ "plaintextBuffer.position is not 0");
plaintextBuffer.limit(readLimit);
byteBuf.readBytes(plaintextBuffer);
- Preconditions.checkState(readLimit == plaintextBuffer.position());
+ JavaUtils.checkState(readLimit == plaintextBuffer.position(),
+ "plaintextBuffer.position should be equal to readLimit.");
} else if (plaintextMessage instanceof FileRegion fileRegion) {
ByteBufferWriteableChannel plaintextChannel =
new ByteBufferWriteableChannel(plaintextBuffer);
@@ -347,7 +350,7 @@ private boolean initalizeDecrypter(ByteBuf ciphertextNettyBuf)
@Override
public void channelRead(ChannelHandlerContext ctx, Object ciphertextMessage)
throws GeneralSecurityException {
- Preconditions.checkArgument(ciphertextMessage instanceof ByteBuf,
+ JavaUtils.checkArgument(ciphertextMessage instanceof ByteBuf,
"Unrecognized message type: %s",
ciphertextMessage.getClass().getName());
ByteBuf ciphertextNettyBuf = (ByteBuf) ciphertextMessage;
diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/AbstractMessage.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/AbstractMessage.java
index 2924218c2f08b..1170fd3f1ab33 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/protocol/AbstractMessage.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/AbstractMessage.java
@@ -17,7 +17,7 @@
package org.apache.spark.network.protocol;
-import com.google.common.base.Objects;
+import java.util.Objects;
import org.apache.spark.network.buffer.ManagedBuffer;
@@ -48,7 +48,7 @@ public boolean isBodyInFrame() {
}
protected boolean equals(AbstractMessage other) {
- return isBodyInFrame == other.isBodyInFrame && Objects.equal(body, other.body);
+ return isBodyInFrame == other.isBodyInFrame && Objects.equals(body, other.body);
}
}
diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java
index cbad4c61b9b4a..736d8e6f5eea2 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java
@@ -20,8 +20,6 @@
import java.util.Objects;
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
/**
* Response to {@link ChunkFetchRequest} when there is an error fetching the chunk.
@@ -70,9 +68,6 @@ public boolean equals(Object other) {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("streamChunkId", streamChunkId)
- .append("errorString", errorString)
- .toString();
+ return "ChunkFetchFailure[streamChunkId=" + streamChunkId + ",errorString=" + errorString + "]";
}
}
diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java
index 2865388b3297c..cc042fdf76b77 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java
@@ -18,8 +18,6 @@
package org.apache.spark.network.protocol;
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
/**
* Request to fetch a sequence of a single chunk of a stream. This will correspond to a single
@@ -64,8 +62,6 @@ public boolean equals(Object other) {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("streamChunkId", streamChunkId)
- .toString();
+ return "ChunkFetchRequest[streamChunkId=" + streamChunkId + "]";
}
}
diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java
index aa89b2062f626..948190e7a2d53 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java
@@ -20,8 +20,6 @@
import java.util.Objects;
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.buffer.NettyManagedBuffer;
@@ -83,9 +81,6 @@ public boolean equals(Object other) {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("streamChunkId", streamChunkId)
- .append("buffer", body())
- .toString();
+ return "ChunkFetchSuccess[streamChunkId=" + streamChunkId + ",body=" + body() + "]";
}
}
diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/EncryptedMessageWithHeader.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/EncryptedMessageWithHeader.java
index 321ac13881c2a..84917eca17190 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/protocol/EncryptedMessageWithHeader.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/EncryptedMessageWithHeader.java
@@ -21,7 +21,6 @@
import java.io.InputStream;
import javax.annotation.Nullable;
-import com.google.common.base.Preconditions;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufAllocator;
import io.netty.channel.ChannelHandlerContext;
@@ -29,6 +28,7 @@
import io.netty.handler.stream.ChunkedInput;
import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.util.JavaUtils;
/**
* A wrapper message that holds two separate pieces (a header and a body).
@@ -60,7 +60,7 @@ public class EncryptedMessageWithHeader implements ChunkedInput {
public EncryptedMessageWithHeader(
@Nullable ManagedBuffer managedBuffer, ByteBuf header, Object body, long bodyLength) {
- Preconditions.checkArgument(body instanceof InputStream || body instanceof ChunkedStream,
+ JavaUtils.checkArgument(body instanceof InputStream || body instanceof ChunkedStream,
"Body must be an InputStream or a ChunkedStream.");
this.managedBuffer = managedBuffer;
this.header = header;
diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/MergedBlockMetaRequest.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/MergedBlockMetaRequest.java
index 3723730ebc06c..0b1476664f651 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/protocol/MergedBlockMetaRequest.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/MergedBlockMetaRequest.java
@@ -17,10 +17,9 @@
package org.apache.spark.network.protocol;
-import com.google.common.base.Objects;
+import java.util.Objects;
+
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
/**
* Request to find the meta information for the specified merged block. The meta information
@@ -79,7 +78,7 @@ public static MergedBlockMetaRequest decode(ByteBuf buf) {
@Override
public int hashCode() {
- return Objects.hashCode(requestId, appId, shuffleId, shuffleMergeId, reduceId);
+ return Objects.hash(requestId, appId, shuffleId, shuffleMergeId, reduceId);
}
@Override
@@ -87,19 +86,14 @@ public boolean equals(Object other) {
if (other instanceof MergedBlockMetaRequest o) {
return requestId == o.requestId && shuffleId == o.shuffleId &&
shuffleMergeId == o.shuffleMergeId && reduceId == o.reduceId &&
- Objects.equal(appId, o.appId);
+ Objects.equals(appId, o.appId);
}
return false;
}
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("requestId", requestId)
- .append("appId", appId)
- .append("shuffleId", shuffleId)
- .append("shuffleMergeId", shuffleMergeId)
- .append("reduceId", reduceId)
- .toString();
+ return "MergedBlockMetaRequest[requestId=" + requestId + ",appId=" + appId + ",shuffleId=" +
+ shuffleId + ",shuffleMergeId=" + shuffleMergeId + ",reduceId=" + reduceId + "]";
}
}
diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/MergedBlockMetaSuccess.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/MergedBlockMetaSuccess.java
index d2edaf4532e11..255174e34600c 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/protocol/MergedBlockMetaSuccess.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/MergedBlockMetaSuccess.java
@@ -17,10 +17,9 @@
package org.apache.spark.network.protocol;
-import com.google.common.base.Objects;
+import java.util.Objects;
+
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.buffer.NettyManagedBuffer;
@@ -51,13 +50,12 @@ public Type type() {
@Override
public int hashCode() {
- return Objects.hashCode(requestId, numChunks);
+ return Objects.hash(requestId, numChunks);
}
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("requestId", requestId).append("numChunks", numChunks).toString();
+ return "MergedBlockMetaSuccess[requestId=" + requestId + ",numChunks=" + numChunks + "]";
}
@Override
diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java
index ab20fb908eb42..3f23c17939e6c 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java
@@ -66,8 +66,8 @@ public void encode(ChannelHandlerContext ctx, Message in, List out) thro
// Re-encode this message as a failure response.
String error = e.getMessage() != null ? e.getMessage() : "null";
logger.error("Error processing {} for client {}", e,
- MDC.of(LogKeys.MESSAGE$.MODULE$, in),
- MDC.of(LogKeys.HOST_PORT$.MODULE$, ctx.channel().remoteAddress()));
+ MDC.of(LogKeys.MESSAGE, in),
+ MDC.of(LogKeys.HOST_PORT, ctx.channel().remoteAddress()));
encode(ctx, resp.createFailureResponse(error), out);
} else {
throw e;
diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java
index e8eb83e7577bf..993ce2381caa5 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java
@@ -22,13 +22,13 @@
import java.nio.channels.WritableByteChannel;
import javax.annotation.Nullable;
-import com.google.common.base.Preconditions;
import io.netty.buffer.ByteBuf;
import io.netty.channel.FileRegion;
import io.netty.util.ReferenceCountUtil;
import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.util.AbstractFileRegion;
+import org.apache.spark.network.util.JavaUtils;
/**
* A wrapper message that holds two separate pieces (a header and a body).
@@ -72,7 +72,7 @@ public class MessageWithHeader extends AbstractFileRegion {
ByteBuf header,
Object body,
long bodyLength) {
- Preconditions.checkArgument(body instanceof ByteBuf || body instanceof FileRegion,
+ JavaUtils.checkArgument(body instanceof ByteBuf || body instanceof FileRegion,
"Body must be a ByteBuf or a FileRegion.");
this.managedBuffer = managedBuffer;
this.header = header;
@@ -105,7 +105,7 @@ public long transferred() {
*/
@Override
public long transferTo(final WritableByteChannel target, final long position) throws IOException {
- Preconditions.checkArgument(position == totalBytesTransferred, "Invalid position.");
+ JavaUtils.checkArgument(position == totalBytesTransferred, "Invalid position.");
// Bytes written for header in this call.
long writtenHeader = 0;
if (header.readableBytes() > 0) {
diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/OneWayMessage.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/OneWayMessage.java
index 91c818f3612a9..de1f91bc8d318 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/protocol/OneWayMessage.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/OneWayMessage.java
@@ -20,8 +20,6 @@
import java.util.Objects;
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.buffer.NettyManagedBuffer;
@@ -74,8 +72,6 @@ public boolean equals(Object other) {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("body", body())
- .toString();
+ return "OneWayMessage[body=" + body() + "]";
}
}
diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java
index 02a45d68c650e..f48264a494f0e 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java
@@ -20,8 +20,6 @@
import java.util.Objects;
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
/** Response to {@link RpcRequest} for a failed RPC. */
public final class RpcFailure extends AbstractMessage implements ResponseMessage {
@@ -68,9 +66,6 @@ public boolean equals(Object other) {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("requestId", requestId)
- .append("errorString", errorString)
- .toString();
+ return "RpcFailure[requestId=" + requestId + ",errorString=" + errorString + "]";
}
}
diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java
index a7dbe1283b314..2619b176e331e 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java
@@ -20,8 +20,6 @@
import java.util.Objects;
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.buffer.NettyManagedBuffer;
@@ -80,9 +78,6 @@ public boolean equals(Object other) {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("requestId", requestId)
- .append("body", body())
- .toString();
+ return "RpcRequest[requestId=" + requestId + ",body=" + body() + "]";
}
}
diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java
index 85709e36f83ee..a9805bcf686b4 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java
@@ -20,8 +20,6 @@
import java.util.Objects;
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.buffer.NettyManagedBuffer;
@@ -80,9 +78,6 @@ public boolean equals(Object other) {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("requestId", requestId)
- .append("body", body())
- .toString();
+ return "RpcResponse[requestId=" + requestId + ",body=" + body() + "]";
}
}
diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/SslMessageEncoder.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/SslMessageEncoder.java
index abe6ccca7bfd6..083e45b3d6bdb 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/protocol/SslMessageEncoder.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/SslMessageEncoder.java
@@ -71,8 +71,8 @@ public void encode(ChannelHandlerContext ctx, Message in, List out) thro
// Re-encode this message as a failure response.
String error = e.getMessage() != null ? e.getMessage() : "null";
logger.error("Error processing {} for client {}", e,
- MDC.of(LogKeys.MESSAGE$.MODULE$, in),
- MDC.of(LogKeys.HOST_PORT$.MODULE$, ctx.channel().remoteAddress()));
+ MDC.of(LogKeys.MESSAGE, in),
+ MDC.of(LogKeys.HOST_PORT, ctx.channel().remoteAddress()));
encode(ctx, resp.createFailureResponse(error), out);
} else {
throw e;
diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamChunkId.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamChunkId.java
index c3b715009dffe..61aae3e36eceb 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamChunkId.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamChunkId.java
@@ -20,8 +20,6 @@
import java.util.Objects;
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
/**
* Encapsulates a request for a particular chunk of a stream.
@@ -61,9 +59,6 @@ public boolean equals(Object other) {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("streamId", streamId)
- .append("chunkIndex", chunkIndex)
- .toString();
+ return "StreamChunkId[streamId=" + streamId + ",chunkIndex=" + chunkIndex + "]";
}
}
diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamFailure.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamFailure.java
index 9a7bf2f65af3a..50cc25a4919a5 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamFailure.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamFailure.java
@@ -20,8 +20,6 @@
import java.util.Objects;
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
/**
* Message indicating an error when transferring a stream.
@@ -70,10 +68,7 @@ public boolean equals(Object other) {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("streamId", streamId)
- .append("error", error)
- .toString();
+ return "StreamFailure[streamId=" + streamId + ",error=" + error + "]";
}
}
diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamRequest.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamRequest.java
index 5906b4d380d6e..45ca2578b01a4 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamRequest.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamRequest.java
@@ -20,8 +20,6 @@
import java.util.Objects;
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
/**
* Request to stream data from the remote end.
@@ -69,9 +67,7 @@ public boolean equals(Object other) {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("streamId", streamId)
- .toString();
+ return "StreamRequest[streamId=" + streamId + "]";
}
}
diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamResponse.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamResponse.java
index 0c0aa5c9a635b..d7c304e5c5b34 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamResponse.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamResponse.java
@@ -20,8 +20,6 @@
import java.util.Objects;
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
import org.apache.spark.network.buffer.ManagedBuffer;
@@ -83,11 +81,8 @@ public boolean equals(Object other) {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("streamId", streamId)
- .append("byteCount", byteCount)
- .append("body", body())
- .toString();
+ return "StreamResponse[streamId=" + streamId + ",byteCount=" + byteCount +
+ ",body=" + body() + "]";
}
}
diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/UploadStream.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/UploadStream.java
index 4722f39dfa9db..09baaf60c3f92 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/protocol/UploadStream.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/UploadStream.java
@@ -21,8 +21,6 @@
import java.nio.ByteBuffer;
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.buffer.NettyManagedBuffer;
@@ -99,9 +97,6 @@ public boolean equals(Object other) {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("requestId", requestId)
- .append("body", body())
- .toString();
+ return "UploadStream[requestId=" + requestId + ",body=" + body() + "]";
}
}
diff --git a/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java b/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java
index e1275689ae6a0..1cdb951d2d04e 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java
@@ -23,7 +23,6 @@
import java.util.List;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.channel.Channel;
@@ -35,6 +34,7 @@
import org.apache.spark.network.util.AbstractFileRegion;
import org.apache.spark.network.util.ByteArrayWritableChannel;
+import org.apache.spark.network.util.JavaUtils;
import org.apache.spark.network.util.NettyUtils;
/**
@@ -152,7 +152,7 @@ static class EncryptedMessage extends AbstractFileRegion {
private long transferred;
EncryptedMessage(SaslEncryptionBackend backend, Object msg, int maxOutboundBlockSize) {
- Preconditions.checkArgument(msg instanceof ByteBuf || msg instanceof FileRegion,
+ JavaUtils.checkArgument(msg instanceof ByteBuf || msg instanceof FileRegion,
"Unrecognized message type: %s", msg.getClass().getName());
this.backend = backend;
this.isByteBuf = msg instanceof ByteBuf;
@@ -241,7 +241,7 @@ public boolean release(int decrement) {
public long transferTo(final WritableByteChannel target, final long position)
throws IOException {
- Preconditions.checkArgument(position == transferred(), "Invalid position.");
+ JavaUtils.checkArgument(position == transferred(), "Invalid position.");
long reportedWritten = 0L;
long actuallyWritten = 0L;
diff --git a/common/network-common/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java b/common/network-common/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java
index f32fd5145c7c5..24e01c924ef3a 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java
@@ -29,8 +29,8 @@
import javax.security.sasl.SaslServer;
import java.nio.charset.StandardCharsets;
import java.util.Map;
+import java.util.Objects;
-import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableMap;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
@@ -182,13 +182,13 @@ public void handle(Callback[] callbacks) throws UnsupportedCallbackException {
/* Encode a byte[] identifier as a Base64-encoded string. */
public static String encodeIdentifier(String identifier) {
- Preconditions.checkNotNull(identifier, "User cannot be null if SASL is enabled");
+ Objects.requireNonNull(identifier, "User cannot be null if SASL is enabled");
return getBase64EncodedString(identifier);
}
/** Encode a password as a base64-encoded char[] array. */
public static char[] encodePassword(String password) {
- Preconditions.checkNotNull(password, "Password cannot be null if SASL is enabled");
+ Objects.requireNonNull(password, "Password cannot be null if SASL is enabled");
return getBase64EncodedString(password).toCharArray();
}
diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/BlockPushNonFatalFailure.java b/common/network-common/src/main/java/org/apache/spark/network/server/BlockPushNonFatalFailure.java
index f60a74670d149..a0e9305265385 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/server/BlockPushNonFatalFailure.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/server/BlockPushNonFatalFailure.java
@@ -18,8 +18,9 @@
package org.apache.spark.network.server;
import java.nio.ByteBuffer;
+import java.util.Objects;
-import com.google.common.base.Preconditions;
+import org.apache.spark.network.util.JavaUtils;
/**
* A special RuntimeException thrown when shuffle service experiences a non-fatal failure
@@ -101,14 +102,12 @@ public synchronized Throwable fillInStackTrace() {
public ByteBuffer getResponse() {
// Ensure we do not invoke this method if response is not set
- Preconditions.checkNotNull(response);
- return response;
+ return Objects.requireNonNull(response);
}
public ReturnCode getReturnCode() {
// Ensure we do not invoke this method if returnCode is not set
- Preconditions.checkNotNull(returnCode);
- return returnCode;
+ return Objects.requireNonNull(returnCode);
}
public enum ReturnCode {
@@ -171,7 +170,7 @@ public static boolean shouldNotRetryErrorCode(ReturnCode returnCode) {
}
public static String getErrorMsg(String blockId, ReturnCode errorCode) {
- Preconditions.checkArgument(errorCode != ReturnCode.SUCCESS);
+ JavaUtils.checkArgument(errorCode != ReturnCode.SUCCESS, "errorCode should not be SUCCESS.");
return "Block " + blockId + errorCode.errorMsgSuffix;
}
}
diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/ChunkFetchRequestHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/ChunkFetchRequestHandler.java
index cc0bed7ed5b6d..c7d4d671dec7d 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/server/ChunkFetchRequestHandler.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/server/ChunkFetchRequestHandler.java
@@ -19,7 +19,6 @@
import java.net.SocketAddress;
-import com.google.common.base.Throwables;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelFutureListener;
@@ -36,6 +35,7 @@
import org.apache.spark.network.protocol.ChunkFetchRequest;
import org.apache.spark.network.protocol.ChunkFetchSuccess;
import org.apache.spark.network.protocol.Encodable;
+import org.apache.spark.network.util.JavaUtils;
import static org.apache.spark.network.util.NettyUtils.*;
@@ -74,7 +74,7 @@ public ChunkFetchRequestHandler(
@Override
public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
logger.warn("Exception in connection from {}", cause,
- MDC.of(LogKeys.HOST_PORT$.MODULE$, getRemoteAddress(ctx.channel())));
+ MDC.of(LogKeys.HOST_PORT, getRemoteAddress(ctx.channel())));
ctx.close();
}
@@ -96,8 +96,8 @@ public void processFetchRequest(
long chunksBeingTransferred = streamManager.chunksBeingTransferred();
if (chunksBeingTransferred >= maxChunksBeingTransferred) {
logger.warn("The number of chunks being transferred {} is above {}, close the connection.",
- MDC.of(LogKeys.NUM_CHUNKS$.MODULE$, chunksBeingTransferred),
- MDC.of(LogKeys.MAX_NUM_CHUNKS$.MODULE$, maxChunksBeingTransferred));
+ MDC.of(LogKeys.NUM_CHUNKS, chunksBeingTransferred),
+ MDC.of(LogKeys.MAX_NUM_CHUNKS, maxChunksBeingTransferred));
channel.close();
return;
}
@@ -111,10 +111,10 @@ public void processFetchRequest(
}
} catch (Exception e) {
logger.error("Error opening block {} for request from {}", e,
- MDC.of(LogKeys.STREAM_CHUNK_ID$.MODULE$, msg.streamChunkId),
- MDC.of(LogKeys.HOST_PORT$.MODULE$, getRemoteAddress(channel)));
+ MDC.of(LogKeys.STREAM_CHUNK_ID, msg.streamChunkId),
+ MDC.of(LogKeys.HOST_PORT, getRemoteAddress(channel)));
respond(channel, new ChunkFetchFailure(msg.streamChunkId,
- Throwables.getStackTraceAsString(e)));
+ JavaUtils.stackTraceToString(e)));
return;
}
@@ -153,8 +153,8 @@ private ChannelFuture respond(
} else {
logger.error("Error sending result {} to {}; closing connection",
future.cause(),
- MDC.of(LogKeys.RESULT$.MODULE$, result),
- MDC.of(LogKeys.HOST_PORT$.MODULE$, remoteAddress));
+ MDC.of(LogKeys.RESULT, result),
+ MDC.of(LogKeys.HOST_PORT, remoteAddress));
channel.close();
}
});
diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java b/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java
index f322293782dee..cb53d565e7e87 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java
@@ -19,20 +19,20 @@
import java.util.Iterator;
import java.util.Map;
+import java.util.Objects;
import java.util.Random;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicLong;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
import io.netty.channel.Channel;
-import org.apache.commons.lang3.tuple.ImmutablePair;
-import org.apache.commons.lang3.tuple.Pair;
import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;
import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.client.TransportClient;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.util.Pair;
/**
* StreamManager which allows registration of an Iterator<ManagedBuffer>, which are
@@ -72,7 +72,7 @@ private static class StreamState {
Channel channel,
boolean isBufferMaterializedOnNext) {
this.appId = appId;
- this.buffers = Preconditions.checkNotNull(buffers);
+ this.buffers = Objects.requireNonNull(buffers);
this.associatedChannel = channel;
this.isBufferMaterializedOnNext = isBufferMaterializedOnNext;
}
@@ -127,7 +127,7 @@ public static Pair parseStreamChunkId(String streamChunkId) {
"Stream id and chunk index should be specified.";
long streamId = Long.valueOf(array[0]);
int chunkIndex = Integer.valueOf(array[1]);
- return ImmutablePair.of(streamId, chunkIndex);
+ return Pair.of(streamId, chunkIndex);
}
@Override
@@ -167,7 +167,7 @@ public void connectionTerminated(Channel channel) {
public void checkAuthorization(TransportClient client, long streamId) {
if (client.getClientId() != null) {
StreamState state = streams.get(streamId);
- Preconditions.checkArgument(state != null, "Unknown stream ID.");
+ JavaUtils.checkArgument(state != null, "Unknown stream ID.");
if (!client.getClientId().equals(state.appId)) {
throw new SecurityException(String.format(
"Client %s not authorized to read stream %d (app %s).",
diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java
index 283f0f0a431fd..d0df24873cbce 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java
@@ -88,7 +88,7 @@ public TransportClient getClient() {
@Override
public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
logger.warn("Exception in connection from {}", cause,
- MDC.of(LogKeys.HOST_PORT$.MODULE$, getRemoteAddress(ctx.channel())));
+ MDC.of(LogKeys.HOST_PORT, getRemoteAddress(ctx.channel())));
requestHandler.exceptionCaught(cause);
responseHandler.exceptionCaught(cause);
ctx.close();
@@ -168,9 +168,9 @@ public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exc
logger.error("Connection to {} has been quiet for {} ms while there are outstanding " +
"requests. Assuming connection is dead; please adjust" +
" spark.{}.io.connectionTimeout if this is wrong.",
- MDC.of(LogKeys.HOST_PORT$.MODULE$, address),
- MDC.of(LogKeys.TIMEOUT$.MODULE$, requestTimeoutNs / 1000 / 1000),
- MDC.of(LogKeys.MODULE_NAME$.MODULE$, transportContext.getConf().getModuleName()));
+ MDC.of(LogKeys.HOST_PORT, address),
+ MDC.of(LogKeys.TIMEOUT, requestTimeoutNs / 1000 / 1000),
+ MDC.of(LogKeys.MODULE_NAME, transportContext.getConf().getModuleName()));
client.timeOut();
ctx.close();
} else if (closeIdleConnections) {
diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java
index 2727051894b7a..464d4d9eb378f 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java
@@ -21,7 +21,6 @@
import java.net.SocketAddress;
import java.nio.ByteBuffer;
-import com.google.common.base.Throwables;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
@@ -33,6 +32,7 @@
import org.apache.spark.network.buffer.NioManagedBuffer;
import org.apache.spark.network.client.*;
import org.apache.spark.network.protocol.*;
+import org.apache.spark.network.util.JavaUtils;
import org.apache.spark.network.util.TransportFrameDecoder;
import static org.apache.spark.network.util.NettyUtils.getRemoteAddress;
@@ -132,8 +132,8 @@ private void processStreamRequest(final StreamRequest req) {
long chunksBeingTransferred = streamManager.chunksBeingTransferred();
if (chunksBeingTransferred >= maxChunksBeingTransferred) {
logger.warn("The number of chunks being transferred {} is above {}, close the connection.",
- MDC.of(LogKeys.NUM_CHUNKS$.MODULE$, chunksBeingTransferred),
- MDC.of(LogKeys.MAX_NUM_CHUNKS$.MODULE$, maxChunksBeingTransferred));
+ MDC.of(LogKeys.NUM_CHUNKS, chunksBeingTransferred),
+ MDC.of(LogKeys.MAX_NUM_CHUNKS, maxChunksBeingTransferred));
channel.close();
return;
}
@@ -143,9 +143,9 @@ private void processStreamRequest(final StreamRequest req) {
buf = streamManager.openStream(req.streamId);
} catch (Exception e) {
logger.error("Error opening stream {} for request from {}", e,
- MDC.of(LogKeys.STREAM_ID$.MODULE$, req.streamId),
- MDC.of(LogKeys.HOST_PORT$.MODULE$, getRemoteAddress(channel)));
- respond(new StreamFailure(req.streamId, Throwables.getStackTraceAsString(e)));
+ MDC.of(LogKeys.STREAM_ID, req.streamId),
+ MDC.of(LogKeys.HOST_PORT, getRemoteAddress(channel)));
+ respond(new StreamFailure(req.streamId, JavaUtils.stackTraceToString(e)));
return;
}
@@ -172,14 +172,14 @@ public void onSuccess(ByteBuffer response) {
@Override
public void onFailure(Throwable e) {
- respond(new RpcFailure(req.requestId, Throwables.getStackTraceAsString(e)));
+ respond(new RpcFailure(req.requestId, JavaUtils.stackTraceToString(e)));
}
});
} catch (Exception e) {
logger.error("Error while invoking RpcHandler#receive() on RPC id {} from {}", e,
- MDC.of(LogKeys.REQUEST_ID$.MODULE$, req.requestId),
- MDC.of(LogKeys.HOST_PORT$.MODULE$, getRemoteAddress(channel)));
- respond(new RpcFailure(req.requestId, Throwables.getStackTraceAsString(e)));
+ MDC.of(LogKeys.REQUEST_ID, req.requestId),
+ MDC.of(LogKeys.HOST_PORT, getRemoteAddress(channel)));
+ respond(new RpcFailure(req.requestId, JavaUtils.stackTraceToString(e)));
} finally {
req.body().release();
}
@@ -199,7 +199,7 @@ public void onSuccess(ByteBuffer response) {
@Override
public void onFailure(Throwable e) {
- respond(new RpcFailure(req.requestId, Throwables.getStackTraceAsString(e)));
+ respond(new RpcFailure(req.requestId, JavaUtils.stackTraceToString(e)));
}
};
TransportFrameDecoder frameDecoder = (TransportFrameDecoder)
@@ -264,9 +264,9 @@ public String getID() {
new NioManagedBuffer(blockPushNonFatalFailure.getResponse())));
} else {
logger.error("Error while invoking RpcHandler#receive() on RPC id {} from {}", e,
- MDC.of(LogKeys.REQUEST_ID$.MODULE$, req.requestId),
- MDC.of(LogKeys.HOST_PORT$.MODULE$, getRemoteAddress(channel)));
- respond(new RpcFailure(req.requestId, Throwables.getStackTraceAsString(e)));
+ MDC.of(LogKeys.REQUEST_ID, req.requestId),
+ MDC.of(LogKeys.HOST_PORT, getRemoteAddress(channel)));
+ respond(new RpcFailure(req.requestId, JavaUtils.stackTraceToString(e)));
}
// We choose to totally fail the channel, rather than trying to recover as we do in other
// cases. We don't know how many bytes of the stream the client has already sent for the
@@ -282,7 +282,7 @@ private void processOneWayMessage(OneWayMessage req) {
rpcHandler.receive(reverseClient, req.body().nioByteBuffer());
} catch (Exception e) {
logger.error("Error while invoking RpcHandler#receive() for one-way message from {}.", e,
- MDC.of(LogKeys.HOST_PORT$.MODULE$, getRemoteAddress(channel)));
+ MDC.of(LogKeys.HOST_PORT, getRemoteAddress(channel)));
} finally {
req.body().release();
}
@@ -302,16 +302,16 @@ public void onSuccess(int numChunks, ManagedBuffer buffer) {
@Override
public void onFailure(Throwable e) {
logger.trace("Failed to send meta for {}", req);
- respond(new RpcFailure(req.requestId, Throwables.getStackTraceAsString(e)));
+ respond(new RpcFailure(req.requestId, JavaUtils.stackTraceToString(e)));
}
});
} catch (Exception e) {
logger.error("Error while invoking receiveMergeBlockMetaReq() for appId {} shuffleId {} "
- + "reduceId {} from {}", e, MDC.of(LogKeys.APP_ID$.MODULE$, req.appId),
- MDC.of(LogKeys.SHUFFLE_ID$.MODULE$, req.shuffleId),
- MDC.of(LogKeys.REDUCE_ID$.MODULE$, req.reduceId),
- MDC.of(LogKeys.HOST_PORT$.MODULE$, getRemoteAddress(channel)));
- respond(new RpcFailure(req.requestId, Throwables.getStackTraceAsString(e)));
+ + "reduceId {} from {}", e, MDC.of(LogKeys.APP_ID, req.appId),
+ MDC.of(LogKeys.SHUFFLE_ID, req.shuffleId),
+ MDC.of(LogKeys.REDUCE_ID, req.reduceId),
+ MDC.of(LogKeys.HOST_PORT, getRemoteAddress(channel)));
+ respond(new RpcFailure(req.requestId, JavaUtils.stackTraceToString(e)));
}
}
@@ -326,8 +326,8 @@ private ChannelFuture respond(Encodable result) {
logger.trace("Sent result {} to client {}", result, remoteAddress);
} else {
logger.error("Error sending result {} to {}; closing connection", future.cause(),
- MDC.of(LogKeys.RESULT$.MODULE$, result),
- MDC.of(LogKeys.HOST_PORT$.MODULE$, remoteAddress));
+ MDC.of(LogKeys.RESULT, result),
+ MDC.of(LogKeys.HOST_PORT, remoteAddress));
channel.close();
}
});
diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java
index d1a19652f5649..be5d9e03c45c1 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java
@@ -19,13 +19,13 @@
import java.io.Closeable;
import java.net.InetSocketAddress;
+import java.util.ArrayList;
import java.util.List;
+import java.util.Objects;
import java.util.concurrent.TimeUnit;
import com.codahale.metrics.Counter;
import com.codahale.metrics.MetricSet;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.buffer.PooledByteBufAllocator;
import io.netty.channel.ChannelFuture;
@@ -33,7 +33,6 @@
import io.netty.channel.ChannelOption;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.socket.SocketChannel;
-import org.apache.commons.lang3.SystemUtils;
import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;
@@ -77,7 +76,7 @@ public TransportServer(
this.pooledAllocator = NettyUtils.createPooledByteBufAllocator(
conf.preferDirectBufs(), true /* allowCache */, conf.serverThreads());
}
- this.bootstraps = Lists.newArrayList(Preconditions.checkNotNull(bootstraps));
+ this.bootstraps = new ArrayList<>(Objects.requireNonNull(bootstraps));
boolean shouldClose = true;
try {
@@ -105,11 +104,13 @@ private void init(String hostToBind, int portToBind) {
EventLoopGroup workerGroup = NettyUtils.createEventLoop(ioMode, conf.serverThreads(),
conf.getModuleName() + "-server");
+ String name = System.getProperty("os.name");
+ boolean isNotWindows = !name.regionMatches(true, 0, "Windows", 0, 7);
bootstrap = new ServerBootstrap()
.group(bossGroup, workerGroup)
.channel(NettyUtils.getServerChannelClass(ioMode))
.option(ChannelOption.ALLOCATOR, pooledAllocator)
- .option(ChannelOption.SO_REUSEADDR, !SystemUtils.IS_OS_WINDOWS)
+ .option(ChannelOption.SO_REUSEADDR, isNotWindows)
.childOption(ChannelOption.ALLOCATOR, pooledAllocator);
this.metrics = new NettyMemoryMetrics(
diff --git a/common/network-common/src/main/java/org/apache/spark/network/ssl/SSLFactory.java b/common/network-common/src/main/java/org/apache/spark/network/ssl/SSLFactory.java
index a2e42e3eb39f6..f4b245ca7b128 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/ssl/SSLFactory.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/ssl/SSLFactory.java
@@ -20,6 +20,7 @@
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStream;
+import java.nio.file.Files;
import java.security.GeneralSecurityException;
import java.security.KeyStore;
import java.security.KeyStoreException;
@@ -41,8 +42,6 @@
import javax.net.ssl.TrustManagerFactory;
import javax.net.ssl.X509TrustManager;
-import com.google.common.io.Files;
-
import io.netty.buffer.ByteBufAllocator;
import io.netty.handler.ssl.OpenSsl;
import io.netty.handler.ssl.SslContext;
@@ -378,7 +377,7 @@ private static TrustManager[] trustStoreManagers(
private static TrustManager[] defaultTrustManagers(File trustStore, String trustStorePassword)
throws IOException, KeyStoreException, CertificateException, NoSuchAlgorithmException {
- try (InputStream input = Files.asByteSource(trustStore).openStream()) {
+ try (InputStream input = Files.newInputStream(trustStore.toPath())) {
KeyStore ks = KeyStore.getInstance(KeyStore.getDefaultType());
char[] passwordCharacters = trustStorePassword != null?
trustStorePassword.toCharArray() : null;
diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/LevelDBProvider.java b/common/network-common/src/main/java/org/apache/spark/network/util/LevelDBProvider.java
index 391931961a474..ec3e032102e4f 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/util/LevelDBProvider.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/util/LevelDBProvider.java
@@ -50,7 +50,7 @@ public static DB initLevelDB(File dbFile, StoreVersion version, ObjectMapper map
tmpDb = JniDBFactory.factory.open(dbFile, options);
} catch (NativeDB.DBException e) {
if (e.isNotFound() || e.getMessage().contains(" does not exist ")) {
- logger.info("Creating state database at {}", MDC.of(LogKeys.PATH$.MODULE$, dbFile));
+ logger.info("Creating state database at {}", MDC.of(LogKeys.PATH, dbFile));
options.createIfMissing(true);
try {
tmpDb = JniDBFactory.factory.open(dbFile, options);
@@ -61,16 +61,16 @@ public static DB initLevelDB(File dbFile, StoreVersion version, ObjectMapper map
// the leveldb file seems to be corrupt somehow. Lets just blow it away and create a new
// one, so we can keep processing new apps
logger.error("error opening leveldb file {}. Creating new file, will not be able to " +
- "recover state for existing applications", e, MDC.of(LogKeys.PATH$.MODULE$, dbFile));
+ "recover state for existing applications", e, MDC.of(LogKeys.PATH, dbFile));
if (dbFile.isDirectory()) {
for (File f : dbFile.listFiles()) {
if (!f.delete()) {
- logger.warn("error deleting {}", MDC.of(LogKeys.PATH$.MODULE$, f.getPath()));
+ logger.warn("error deleting {}", MDC.of(LogKeys.PATH, f.getPath()));
}
}
}
if (!dbFile.delete()) {
- logger.warn("error deleting {}", MDC.of(LogKeys.PATH$.MODULE$, dbFile.getPath()));
+ logger.warn("error deleting {}", MDC.of(LogKeys.PATH, dbFile.getPath()));
}
options.createIfMissing(true);
try {
diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/LimitedInputStream.java b/common/network-common/src/main/java/org/apache/spark/network/util/LimitedInputStream.java
index e6cf02a590e29..79cf0eb7c6153 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/util/LimitedInputStream.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/util/LimitedInputStream.java
@@ -21,8 +21,7 @@
import java.io.FilterInputStream;
import java.io.IOException;
import java.io.InputStream;
-
-import com.google.common.base.Preconditions;
+import java.util.Objects;
/**
* Wraps a {@link InputStream}, limiting the number of bytes which can be read.
@@ -50,10 +49,9 @@ public LimitedInputStream(InputStream in, long limit) {
* @param closeWrappedStream whether to close {@code in} when {@link #close} is called
*/
public LimitedInputStream(InputStream in, long limit, boolean closeWrappedStream) {
- super(in);
+ super(Objects.requireNonNull(in));
this.closeWrappedStream = closeWrappedStream;
- Preconditions.checkNotNull(in);
- Preconditions.checkArgument(limit >= 0, "limit must be non-negative");
+ JavaUtils.checkArgument(limit >= 0, "limit must be non-negative");
left = limit;
}
@Override public int available() throws IOException {
diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/RocksDBProvider.java b/common/network-common/src/main/java/org/apache/spark/network/util/RocksDBProvider.java
index 1753c124c9935..cea9207d3470a 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/util/RocksDBProvider.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/util/RocksDBProvider.java
@@ -67,7 +67,7 @@ public static RocksDB initRockDB(File dbFile, StoreVersion version, ObjectMapper
tmpDb = RocksDB.open(dbOptions, dbFile.toString());
} catch (RocksDBException e) {
if (e.getStatus().getCode() == Status.Code.NotFound) {
- logger.info("Creating state database at {}", MDC.of(LogKeys.PATH$.MODULE$, dbFile));
+ logger.info("Creating state database at {}", MDC.of(LogKeys.PATH, dbFile));
dbOptions.setCreateIfMissing(true);
try {
tmpDb = RocksDB.open(dbOptions, dbFile.toString());
@@ -78,16 +78,16 @@ public static RocksDB initRockDB(File dbFile, StoreVersion version, ObjectMapper
// the RocksDB file seems to be corrupt somehow. Let's just blow it away and create
// a new one, so we can keep processing new apps
logger.error("error opening rocksdb file {}. Creating new file, will not be able to " +
- "recover state for existing applications", e, MDC.of(LogKeys.PATH$.MODULE$, dbFile));
+ "recover state for existing applications", e, MDC.of(LogKeys.PATH, dbFile));
if (dbFile.isDirectory()) {
for (File f : Objects.requireNonNull(dbFile.listFiles())) {
if (!f.delete()) {
- logger.warn("error deleting {}", MDC.of(LogKeys.PATH$.MODULE$, f.getPath()));
+ logger.warn("error deleting {}", MDC.of(LogKeys.PATH, f.getPath()));
}
}
}
if (!dbFile.delete()) {
- logger.warn("error deleting {}", MDC.of(LogKeys.PATH$.MODULE$, dbFile.getPath()));
+ logger.warn("error deleting {}", MDC.of(LogKeys.PATH, dbFile.getPath()));
}
dbOptions.setCreateIfMissing(true);
try {
diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java
index 822b8aa310a22..003e72edf29ee 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java
@@ -21,8 +21,6 @@
import java.util.Locale;
import java.util.Properties;
import java.util.concurrent.TimeUnit;
-import com.google.common.base.Preconditions;
-import com.google.common.primitives.Ints;
import io.netty.util.NettyRuntime;
/**
@@ -171,7 +169,7 @@ public int ioRetryWaitTimeMs() {
* memory mapping has high overhead for blocks close to or below the page size of the OS.
*/
public int memoryMapBytes() {
- return Ints.checkedCast(JavaUtils.byteStringAsBytes(
+ return JavaUtils.checkedCast(JavaUtils.byteStringAsBytes(
conf.get("spark.storage.memoryMapThreshold", "2m")));
}
@@ -248,7 +246,7 @@ public boolean saslEncryption() {
* Maximum number of bytes to be encrypted at a time when SASL encryption is used.
*/
public int maxSaslEncryptedBlockSize() {
- return Ints.checkedCast(JavaUtils.byteStringAsBytes(
+ return JavaUtils.checkedCast(JavaUtils.byteStringAsBytes(
conf.get("spark.network.sasl.maxEncryptedBlockSize", "64k")));
}
@@ -263,7 +261,7 @@ public boolean saslServerAlwaysEncrypt() {
* When Secure (SSL/TLS) Shuffle is enabled, the Chunk size to use for shuffling files.
*/
public int sslShuffleChunkSize() {
- return Ints.checkedCast(JavaUtils.byteStringAsBytes(
+ return JavaUtils.checkedCast(JavaUtils.byteStringAsBytes(
conf.get("spark.network.ssl.maxEncryptedBlockSize", "64k")));
}
@@ -504,7 +502,7 @@ public int finalizeShuffleMergeHandlerThreads() {
if (!this.getModuleName().equalsIgnoreCase("shuffle")) {
return 0;
}
- Preconditions.checkArgument(separateFinalizeShuffleMerge(),
+ JavaUtils.checkArgument(separateFinalizeShuffleMerge(),
"Please set spark.shuffle.server.finalizeShuffleMergeThreadsPercent to a positive value");
int finalizeShuffleMergeThreadsPercent =
Integer.parseInt(conf.get("spark.shuffle.server.finalizeShuffleMergeThreadsPercent"));
@@ -567,7 +565,7 @@ public String mergedShuffleFileManagerImpl() {
* service unnecessarily.
*/
public int minChunkSizeInMergedShuffleFile() {
- return Ints.checkedCast(JavaUtils.byteStringAsBytes(
+ return JavaUtils.checkedCast(JavaUtils.byteStringAsBytes(
conf.get("spark.shuffle.push.server.minChunkSizeInMergedShuffleFile", "2m")));
}
diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java
index cef0e415aa40a..6b490068507aa 100644
--- a/common/network-common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java
+++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java
@@ -20,7 +20,6 @@
import java.util.LinkedList;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.CompositeByteBuf;
import io.netty.buffer.Unpooled;
@@ -145,9 +144,9 @@ private ByteBuf decodeNext() {
}
if (frameBuf == null) {
- Preconditions.checkArgument(frameSize < MAX_FRAME_SIZE,
+ JavaUtils.checkArgument(frameSize < MAX_FRAME_SIZE,
"Too large frame: %s", frameSize);
- Preconditions.checkArgument(frameSize > 0,
+ JavaUtils.checkArgument(frameSize > 0,
"Frame length should be positive: %s", frameSize);
frameRemainingBytes = (int) frameSize;
@@ -252,7 +251,7 @@ public void handlerRemoved(ChannelHandlerContext ctx) throws Exception {
}
public void setInterceptor(Interceptor interceptor) {
- Preconditions.checkState(this.interceptor == null, "Already have an interceptor.");
+ JavaUtils.checkState(this.interceptor == null, "Already have an interceptor.");
this.interceptor = interceptor;
}
diff --git a/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java b/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java
index 576a106934fda..75ccd8d5789d4 100644
--- a/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java
+++ b/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java
@@ -30,7 +30,6 @@
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
-import com.google.common.collect.Sets;
import com.google.common.io.Closeables;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.BeforeAll;
@@ -188,7 +187,7 @@ public void onFailure(int chunkIndex, Throwable e) {
@Test
public void fetchBufferChunk() throws Exception {
FetchResult res = fetchChunks(Arrays.asList(BUFFER_CHUNK_INDEX));
- assertEquals(Sets.newHashSet(BUFFER_CHUNK_INDEX), res.successChunks);
+ assertEquals(Set.of(BUFFER_CHUNK_INDEX), res.successChunks);
assertTrue(res.failedChunks.isEmpty());
assertBufferListsEqual(Arrays.asList(bufferChunk), res.buffers);
res.releaseBuffers();
@@ -197,7 +196,7 @@ public void fetchBufferChunk() throws Exception {
@Test
public void fetchFileChunk() throws Exception {
FetchResult res = fetchChunks(Arrays.asList(FILE_CHUNK_INDEX));
- assertEquals(Sets.newHashSet(FILE_CHUNK_INDEX), res.successChunks);
+ assertEquals(Set.of(FILE_CHUNK_INDEX), res.successChunks);
assertTrue(res.failedChunks.isEmpty());
assertBufferListsEqual(Arrays.asList(fileChunk), res.buffers);
res.releaseBuffers();
@@ -207,14 +206,14 @@ public void fetchFileChunk() throws Exception {
public void fetchNonExistentChunk() throws Exception {
FetchResult res = fetchChunks(Arrays.asList(12345));
assertTrue(res.successChunks.isEmpty());
- assertEquals(Sets.newHashSet(12345), res.failedChunks);
+ assertEquals(Set.of(12345), res.failedChunks);
assertTrue(res.buffers.isEmpty());
}
@Test
public void fetchBothChunks() throws Exception {
FetchResult res = fetchChunks(Arrays.asList(BUFFER_CHUNK_INDEX, FILE_CHUNK_INDEX));
- assertEquals(Sets.newHashSet(BUFFER_CHUNK_INDEX, FILE_CHUNK_INDEX), res.successChunks);
+ assertEquals(Set.of(BUFFER_CHUNK_INDEX, FILE_CHUNK_INDEX), res.successChunks);
assertTrue(res.failedChunks.isEmpty());
assertBufferListsEqual(Arrays.asList(bufferChunk, fileChunk), res.buffers);
res.releaseBuffers();
@@ -223,8 +222,8 @@ public void fetchBothChunks() throws Exception {
@Test
public void fetchChunkAndNonExistent() throws Exception {
FetchResult res = fetchChunks(Arrays.asList(BUFFER_CHUNK_INDEX, 12345));
- assertEquals(Sets.newHashSet(BUFFER_CHUNK_INDEX), res.successChunks);
- assertEquals(Sets.newHashSet(12345), res.failedChunks);
+ assertEquals(Set.of(BUFFER_CHUNK_INDEX), res.successChunks);
+ assertEquals(Set.of(12345), res.failedChunks);
assertBufferListsEqual(Arrays.asList(bufferChunk), res.buffers);
res.releaseBuffers();
}
diff --git a/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchRequestHandlerSuite.java b/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchRequestHandlerSuite.java
index 74dffd87dcf30..e9a4c355ebe48 100644
--- a/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchRequestHandlerSuite.java
+++ b/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchRequestHandlerSuite.java
@@ -27,8 +27,6 @@
import static org.mockito.Mockito.*;
-import org.apache.commons.lang3.tuple.ImmutablePair;
-import org.apache.commons.lang3.tuple.Pair;
import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.client.TransportClient;
import org.apache.spark.network.protocol.*;
@@ -36,6 +34,7 @@
import org.apache.spark.network.server.NoOpRpcHandler;
import org.apache.spark.network.server.OneForOneStreamManager;
import org.apache.spark.network.server.RpcHandler;
+import org.apache.spark.util.Pair;
public class ChunkFetchRequestHandlerSuite {
@@ -54,7 +53,7 @@ public void handleChunkFetchRequest() throws Exception {
.thenAnswer(invocationOnMock0 -> {
Object response = invocationOnMock0.getArguments()[0];
ExtendedChannelPromise channelFuture = new ExtendedChannelPromise(channel);
- responseAndPromisePairs.add(ImmutablePair.of(response, channelFuture));
+ responseAndPromisePairs.add(Pair.of(response, channelFuture));
return channelFuture;
});
diff --git a/common/network-common/src/test/java/org/apache/spark/network/ProtocolSuite.java b/common/network-common/src/test/java/org/apache/spark/network/ProtocolSuite.java
index 8c1299ebcd836..500d91868bbbd 100644
--- a/common/network-common/src/test/java/org/apache/spark/network/ProtocolSuite.java
+++ b/common/network-common/src/test/java/org/apache/spark/network/ProtocolSuite.java
@@ -19,7 +19,6 @@
import java.util.List;
-import com.google.common.primitives.Ints;
import io.netty.buffer.Unpooled;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.FileRegion;
@@ -44,6 +43,7 @@
import org.apache.spark.network.protocol.StreamRequest;
import org.apache.spark.network.protocol.StreamResponse;
import org.apache.spark.network.util.ByteArrayWritableChannel;
+import org.apache.spark.network.util.JavaUtils;
import org.apache.spark.network.util.NettyUtils;
public class ProtocolSuite {
@@ -115,7 +115,8 @@ private static class FileRegionEncoder extends MessageToMessageEncoder out)
throws Exception {
- ByteArrayWritableChannel channel = new ByteArrayWritableChannel(Ints.checkedCast(in.count()));
+ ByteArrayWritableChannel channel =
+ new ByteArrayWritableChannel(JavaUtils.checkedCast(in.count()));
while (in.transferred() < in.count()) {
in.transferTo(channel, in.transferred());
}
diff --git a/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java b/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java
index 40495d6912c91..e229e32e91717 100644
--- a/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java
+++ b/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java
@@ -24,10 +24,6 @@
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
-import com.google.common.collect.Sets;
-import com.google.common.io.Files;
-import org.apache.commons.lang3.tuple.ImmutablePair;
-import org.apache.commons.lang3.tuple.Pair;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
@@ -41,6 +37,7 @@
import org.apache.spark.network.util.JavaUtils;
import org.apache.spark.network.util.MapConfigProvider;
import org.apache.spark.network.util.TransportConf;
+import org.apache.spark.util.Pair;
public class RpcIntegrationSuite {
static TransportConf conf;
@@ -248,14 +245,14 @@ public void onFailure(Throwable e) {
@Test
public void singleRPC() throws Exception {
RpcResult res = sendRPC("hello/Aaron");
- assertEquals(Sets.newHashSet("Hello, Aaron!"), res.successMessages);
+ assertEquals(Set.of("Hello, Aaron!"), res.successMessages);
assertTrue(res.errorMessages.isEmpty());
}
@Test
public void doubleRPC() throws Exception {
RpcResult res = sendRPC("hello/Aaron", "hello/Reynold");
- assertEquals(Sets.newHashSet("Hello, Aaron!", "Hello, Reynold!"), res.successMessages);
+ assertEquals(Set.of("Hello, Aaron!", "Hello, Reynold!"), res.successMessages);
assertTrue(res.errorMessages.isEmpty());
}
@@ -263,28 +260,28 @@ public void doubleRPC() throws Exception {
public void returnErrorRPC() throws Exception {
RpcResult res = sendRPC("return error/OK");
assertTrue(res.successMessages.isEmpty());
- assertErrorsContain(res.errorMessages, Sets.newHashSet("Returned: OK"));
+ assertErrorsContain(res.errorMessages, Set.of("Returned: OK"));
}
@Test
public void throwErrorRPC() throws Exception {
RpcResult res = sendRPC("throw error/uh-oh");
assertTrue(res.successMessages.isEmpty());
- assertErrorsContain(res.errorMessages, Sets.newHashSet("Thrown: uh-oh"));
+ assertErrorsContain(res.errorMessages, Set.of("Thrown: uh-oh"));
}
@Test
public void doubleTrouble() throws Exception {
RpcResult res = sendRPC("return error/OK", "throw error/uh-oh");
assertTrue(res.successMessages.isEmpty());
- assertErrorsContain(res.errorMessages, Sets.newHashSet("Returned: OK", "Thrown: uh-oh"));
+ assertErrorsContain(res.errorMessages, Set.of("Returned: OK", "Thrown: uh-oh"));
}
@Test
public void sendSuccessAndFailure() throws Exception {
RpcResult res = sendRPC("hello/Bob", "throw error/the", "hello/Builder", "return error/!");
- assertEquals(Sets.newHashSet("Hello, Bob!", "Hello, Builder!"), res.successMessages);
- assertErrorsContain(res.errorMessages, Sets.newHashSet("Thrown: the", "Returned: !"));
+ assertEquals(Set.of("Hello, Bob!", "Hello, Builder!"), res.successMessages);
+ assertErrorsContain(res.errorMessages, Set.of("Thrown: the", "Returned: !"));
}
@Test
@@ -311,7 +308,7 @@ public void sendRpcWithStreamOneAtATime() throws Exception {
for (String stream : StreamTestHelper.STREAMS) {
RpcResult res = sendRpcWithStream(stream);
assertTrue(res.errorMessages.isEmpty(), "there were error messages!" + res.errorMessages);
- assertEquals(Sets.newHashSet(stream), res.successMessages);
+ assertEquals(Set.of(stream), res.successMessages);
}
}
@@ -322,7 +319,7 @@ public void sendRpcWithStreamConcurrently() throws Exception {
streams[i] = StreamTestHelper.STREAMS[i % StreamTestHelper.STREAMS.length];
}
RpcResult res = sendRpcWithStream(streams);
- assertEquals(Sets.newHashSet(StreamTestHelper.STREAMS), res.successMessages);
+ assertEquals(Set.of(StreamTestHelper.STREAMS), res.successMessages);
assertTrue(res.errorMessages.isEmpty());
}
@@ -342,8 +339,8 @@ public void sendRpcWithStreamFailures() throws Exception {
RpcResult exceptionInOnComplete =
sendRpcWithStream("fail/exception-oncomplete/smallBuffer", "smallBuffer");
assertErrorsContain(exceptionInOnComplete.errorMessages,
- Sets.newHashSet("Failure post-processing"));
- assertEquals(Sets.newHashSet("smallBuffer"), exceptionInOnComplete.successMessages);
+ Set.of("Failure post-processing"));
+ assertEquals(Set.of("smallBuffer"), exceptionInOnComplete.successMessages);
}
private void assertErrorsContain(Set errors, Set contains) {
@@ -365,14 +362,14 @@ private void assertErrorAndClosed(RpcResult result, String expectedError) {
// We expect 1 additional error due to closed connection and here are possible keywords in the
// error message.
- Set possibleClosedErrors = Sets.newHashSet(
+ Set possibleClosedErrors = Set.of(
"closed",
"Connection reset",
"java.nio.channels.ClosedChannelException",
"io.netty.channel.StacklessClosedChannelException",
"java.io.IOException: Broken pipe"
);
- Set containsAndClosed = Sets.newHashSet(expectedError);
+ Set containsAndClosed = new HashSet<>(Set.of(expectedError));
containsAndClosed.addAll(possibleClosedErrors);
Pair, Set> r = checkErrorsContain(errors, containsAndClosed);
@@ -392,8 +389,8 @@ private void assertErrorAndClosed(RpcResult result, String expectedError) {
private Pair, Set> checkErrorsContain(
Set errors,
Set contains) {
- Set remainingErrors = Sets.newHashSet(errors);
- Set notFound = Sets.newHashSet();
+ Set remainingErrors = new HashSet<>(errors);
+ Set notFound = new HashSet<>();
for (String contain : contains) {
Iterator it = remainingErrors.iterator();
boolean foundMatch = false;
@@ -408,7 +405,7 @@ private Pair, Set> checkErrorsContain(
notFound.add(contain);
}
}
- return new ImmutablePair<>(remainingErrors, notFound);
+ return new Pair<>(remainingErrors, notFound);
}
private static class VerifyingStreamCallback implements StreamCallbackWithID {
@@ -431,7 +428,8 @@ private static class VerifyingStreamCallback implements StreamCallbackWithID {
void verify() throws IOException {
if (streamId.equals("file")) {
- assertTrue(Files.equal(testData.testFile, outFile), "File stream did not match.");
+ assertTrue(JavaUtils.contentEquals(testData.testFile, outFile),
+ "File stream did not match.");
} else {
byte[] result = ((ByteArrayOutputStream)out).toByteArray();
ByteBuffer srcBuffer = testData.srcBuffer(streamId);
diff --git a/common/network-common/src/test/java/org/apache/spark/network/StreamSuite.java b/common/network-common/src/test/java/org/apache/spark/network/StreamSuite.java
index 4f4637e302b94..496af96cb1cac 100644
--- a/common/network-common/src/test/java/org/apache/spark/network/StreamSuite.java
+++ b/common/network-common/src/test/java/org/apache/spark/network/StreamSuite.java
@@ -29,7 +29,6 @@
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
-import com.google.common.io.Files;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
@@ -43,6 +42,7 @@
import org.apache.spark.network.server.RpcHandler;
import org.apache.spark.network.server.StreamManager;
import org.apache.spark.network.server.TransportServer;
+import org.apache.spark.network.util.JavaUtils;
import org.apache.spark.network.util.MapConfigProvider;
import org.apache.spark.network.util.TransportConf;
@@ -212,7 +212,8 @@ public void run() {
callback.waitForCompletion(timeoutMs);
if (srcBuffer == null) {
- assertTrue(Files.equal(testData.testFile, outFile), "File stream did not match.");
+ assertTrue(JavaUtils.contentEquals(testData.testFile, outFile),
+ "File stream did not match.");
} else {
ByteBuffer base;
synchronized (srcBuffer) {
diff --git a/common/network-common/src/test/java/org/apache/spark/network/TestManagedBuffer.java b/common/network-common/src/test/java/org/apache/spark/network/TestManagedBuffer.java
index d1e93e3cb5845..828d995ba444b 100644
--- a/common/network-common/src/test/java/org/apache/spark/network/TestManagedBuffer.java
+++ b/common/network-common/src/test/java/org/apache/spark/network/TestManagedBuffer.java
@@ -21,11 +21,11 @@
import java.io.InputStream;
import java.nio.ByteBuffer;
-import com.google.common.base.Preconditions;
import io.netty.buffer.Unpooled;
import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.buffer.NettyManagedBuffer;
+import org.apache.spark.network.util.JavaUtils;
/**
* A ManagedBuffer implementation that contains 0, 1, 2, 3, ..., (len-1).
@@ -38,7 +38,7 @@ public class TestManagedBuffer extends ManagedBuffer {
private NettyManagedBuffer underlying;
public TestManagedBuffer(int len) {
- Preconditions.checkArgument(len <= Byte.MAX_VALUE);
+ JavaUtils.checkArgument(len <= Byte.MAX_VALUE, "length exceeds limit " + Byte.MAX_VALUE);
this.len = len;
byte[] byteArray = new byte[len];
for (int i = 0; i < len; i ++) {
diff --git a/common/network-common/src/test/java/org/apache/spark/network/TransportRequestHandlerSuite.java b/common/network-common/src/test/java/org/apache/spark/network/TransportRequestHandlerSuite.java
index d643fb4f662e3..9433e274b507f 100644
--- a/common/network-common/src/test/java/org/apache/spark/network/TransportRequestHandlerSuite.java
+++ b/common/network-common/src/test/java/org/apache/spark/network/TransportRequestHandlerSuite.java
@@ -28,8 +28,6 @@
import static org.junit.jupiter.api.Assertions.*;
import static org.mockito.Mockito.*;
-import org.apache.commons.lang3.tuple.ImmutablePair;
-import org.apache.commons.lang3.tuple.Pair;
import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.client.RpcResponseCallback;
import org.apache.spark.network.client.TransportClient;
@@ -39,6 +37,7 @@
import org.apache.spark.network.server.RpcHandler;
import org.apache.spark.network.server.StreamManager;
import org.apache.spark.network.server.TransportRequestHandler;
+import org.apache.spark.util.Pair;
public class TransportRequestHandlerSuite {
@@ -53,7 +52,7 @@ public void handleStreamRequest() throws Exception {
.thenAnswer(invocationOnMock0 -> {
Object response = invocationOnMock0.getArguments()[0];
ExtendedChannelPromise channelFuture = new ExtendedChannelPromise(channel);
- responseAndPromisePairs.add(ImmutablePair.of(response, channelFuture));
+ responseAndPromisePairs.add(Pair.of(response, channelFuture));
return channelFuture;
});
@@ -145,7 +144,7 @@ public MergedBlockMetaReqHandler getMergedBlockMetaReqHandler() {
when(channel.writeAndFlush(any())).thenAnswer(invocationOnMock0 -> {
Object response = invocationOnMock0.getArguments()[0];
ExtendedChannelPromise channelFuture = new ExtendedChannelPromise(channel);
- responseAndPromisePairs.add(ImmutablePair.of(response, channelFuture));
+ responseAndPromisePairs.add(Pair.of(response, channelFuture));
return channelFuture;
});
diff --git a/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java
index 628de9e780337..904deabba5867 100644
--- a/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java
+++ b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java
@@ -21,7 +21,6 @@
import java.security.GeneralSecurityException;
import java.util.Map;
-import com.google.common.collect.ImmutableMap;
import com.google.crypto.tink.subtle.Hex;
import org.apache.spark.network.util.*;
@@ -49,7 +48,7 @@ abstract class AuthEngineSuite {
static TransportConf getConf(int authEngineVerison, boolean useCtr) {
String authEngineVersion = (authEngineVerison == 1) ? "1" : "2";
String mode = useCtr ? "AES/CTR/NoPadding" : "AES/GCM/NoPadding";
- Map confMap = ImmutableMap.of(
+ Map confMap = Map.of(
"spark.network.crypto.enabled", "true",
"spark.network.crypto.authEngineVersion", authEngineVersion,
"spark.network.crypto.cipher", mode
diff --git a/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthIntegrationSuite.java b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthIntegrationSuite.java
index cb5929f7c65b4..8d1773316878b 100644
--- a/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthIntegrationSuite.java
+++ b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthIntegrationSuite.java
@@ -22,7 +22,6 @@
import java.util.List;
import java.util.Map;
-import com.google.common.collect.ImmutableMap;
import io.netty.channel.Channel;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Test;
@@ -214,7 +213,7 @@ private static class AuthTestCtx {
}
AuthTestCtx(RpcHandler rpcHandler, String mode) throws Exception {
- Map testConf = ImmutableMap.of(
+ Map testConf = Map.of(
"spark.network.crypto.enabled", "true",
"spark.network.crypto.cipher", mode);
this.conf = new TransportConf("rpc", new MapConfigProvider(testConf));
diff --git a/common/network-common/src/test/java/org/apache/spark/network/protocol/MergedBlockMetaSuccessSuite.java b/common/network-common/src/test/java/org/apache/spark/network/protocol/MergedBlockMetaSuccessSuite.java
index a3750ce11172b..41b84f3895876 100644
--- a/common/network-common/src/test/java/org/apache/spark/network/protocol/MergedBlockMetaSuccessSuite.java
+++ b/common/network-common/src/test/java/org/apache/spark/network/protocol/MergedBlockMetaSuccessSuite.java
@@ -21,9 +21,9 @@
import java.io.File;
import java.io.FileOutputStream;
import java.nio.file.Files;
+import java.util.ArrayList;
import java.util.List;
-import com.google.common.collect.Lists;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufAllocator;
import io.netty.buffer.Unpooled;
@@ -65,7 +65,7 @@ public void testMergedBlocksMetaEncodeDecode() throws Exception {
MergedBlockMetaSuccess expectedMeta = new MergedBlockMetaSuccess(requestId, 2,
new FileSegmentManagedBuffer(conf, chunkMetaFile, 0, chunkMetaFile.length()));
- List out = Lists.newArrayList();
+ List out = new ArrayList<>();
ChannelHandlerContext context = mock(ChannelHandlerContext.class);
when(context.alloc()).thenReturn(ByteBufAllocator.DEFAULT);
diff --git a/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java b/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java
index bf0424a1506a3..4feaf5cef3f2b 100644
--- a/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java
+++ b/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java
@@ -23,6 +23,7 @@
import java.io.File;
import java.lang.reflect.Method;
import java.nio.ByteBuffer;
+import java.nio.file.Files;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
@@ -36,8 +37,6 @@
import javax.security.sasl.SaslException;
import com.google.common.collect.ImmutableMap;
-import com.google.common.io.ByteStreams;
-import com.google.common.io.Files;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.channel.Channel;
@@ -221,7 +220,7 @@ public void testEncryptedMessageChunking() throws Exception {
byte[] data = new byte[8 * 1024];
new Random().nextBytes(data);
- Files.write(data, file);
+ Files.write(file.toPath(), data);
SaslEncryptionBackend backend = mock(SaslEncryptionBackend.class);
// It doesn't really matter what we return here, as long as it's not null.
@@ -245,7 +244,7 @@ public void testEncryptedMessageChunking() throws Exception {
@Test
public void testFileRegionEncryption() throws Exception {
- Map testConf = ImmutableMap.of(
+ Map testConf = Map.of(
"spark.network.sasl.maxEncryptedBlockSize", "1k");
AtomicReference response = new AtomicReference<>();
@@ -262,7 +261,7 @@ public void testFileRegionEncryption() throws Exception {
byte[] data = new byte[8 * 1024];
new Random().nextBytes(data);
- Files.write(data, file);
+ Files.write(file.toPath(), data);
ctx = new SaslTestCtx(rpcHandler, true, false, testConf);
@@ -282,7 +281,7 @@ public void testFileRegionEncryption() throws Exception {
verify(callback, times(1)).onSuccess(anyInt(), any(ManagedBuffer.class));
verify(callback, never()).onFailure(anyInt(), any(Throwable.class));
- byte[] received = ByteStreams.toByteArray(response.get().createInputStream());
+ byte[] received = response.get().createInputStream().readAllBytes();
assertArrayEquals(data, received);
} finally {
file.delete();
@@ -299,7 +298,7 @@ public void testFileRegionEncryption() throws Exception {
public void testServerAlwaysEncrypt() {
Exception re = assertThrows(Exception.class,
() -> new SaslTestCtx(mock(RpcHandler.class), false, false,
- ImmutableMap.of("spark.network.sasl.serverAlwaysEncrypt", "true")));
+ Map.of("spark.network.sasl.serverAlwaysEncrypt", "true")));
assertTrue(re.getCause() instanceof SaslException);
}
diff --git a/common/network-common/src/test/java/org/apache/spark/network/util/CryptoUtilsSuite.java b/common/network-common/src/test/java/org/apache/spark/network/util/CryptoUtilsSuite.java
index 47c9ef2490d2e..9673bbfbc2de7 100644
--- a/common/network-common/src/test/java/org/apache/spark/network/util/CryptoUtilsSuite.java
+++ b/common/network-common/src/test/java/org/apache/spark/network/util/CryptoUtilsSuite.java
@@ -20,7 +20,6 @@
import java.util.Map;
import java.util.Properties;
-import com.google.common.collect.ImmutableMap;
import org.junit.jupiter.api.Test;
import static org.junit.jupiter.api.Assertions.*;
@@ -38,7 +37,7 @@ public void testConfConversion() {
String confVal2 = "val2";
String cryptoKey2 = CryptoUtils.COMMONS_CRYPTO_CONFIG_PREFIX + "A.b.c";
- Map conf = ImmutableMap.of(
+ Map conf = Map.of(
confKey1, confVal1,
confKey2, confVal2);
diff --git a/common/network-common/src/test/java/org/apache/spark/network/util/DBProviderSuite.java b/common/network-common/src/test/java/org/apache/spark/network/util/DBProviderSuite.java
index 81bfc55264c4c..f7299e157674e 100644
--- a/common/network-common/src/test/java/org/apache/spark/network/util/DBProviderSuite.java
+++ b/common/network-common/src/test/java/org/apache/spark/network/util/DBProviderSuite.java
@@ -18,7 +18,6 @@
package org.apache.spark.network.util;
import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.commons.lang3.SystemUtils;
import org.apache.spark.network.shuffledb.DBBackend;
import org.apache.spark.network.shuffledb.StoreVersion;
import org.junit.jupiter.api.Assertions;
@@ -38,7 +37,7 @@ public void testRockDBCheckVersionFailed() throws IOException, InterruptedExcept
@Test
public void testLevelDBCheckVersionFailed() throws IOException, InterruptedException {
- assumeFalse(SystemUtils.IS_OS_MAC_OSX && SystemUtils.OS_ARCH.equals("aarch64"));
+ assumeFalse(JavaUtils.isMacOnAppleSilicon);
testCheckVersionFailed(DBBackend.LEVELDB, "leveldb");
}
diff --git a/common/network-common/src/test/java/org/apache/spark/network/util/JavaUtilsSuite.java b/common/network-common/src/test/java/org/apache/spark/network/util/JavaUtilsSuite.java
index 1336a587fd2eb..2edeb3f05c9b0 100644
--- a/common/network-common/src/test/java/org/apache/spark/network/util/JavaUtilsSuite.java
+++ b/common/network-common/src/test/java/org/apache/spark/network/util/JavaUtilsSuite.java
@@ -18,6 +18,7 @@
import java.io.File;
import java.io.IOException;
+import java.nio.file.Files;
import org.junit.jupiter.api.Test;
@@ -55,4 +56,33 @@ public void testCreateDirectory() throws IOException {
() -> JavaUtils.createDirectory(testDirPath, "scenario4"));
assertTrue(testDir.setWritable(true));
}
+
+ @Test
+ public void testListFiles() throws IOException {
+ File tmp = Files.createTempDirectory("testListFiles").toFile();
+ File file = new File(tmp, "file");
+
+ // Return emtpy set on non-existent input
+ assertFalse(file.exists());
+ assertEquals(0, JavaUtils.listFiles(file).size());
+ assertEquals(0, JavaUtils.listPaths(file).size());
+
+ // Return emtpy set on non-directory input
+ file.createNewFile();
+ assertTrue(file.exists());
+ assertEquals(0, JavaUtils.listFiles(file).size());
+ assertEquals(0, JavaUtils.listPaths(file).size());
+
+ // Return empty set on an empty directory location
+ File dir = new File(tmp, "dir");
+ dir.mkdir();
+ new File(dir, "1").createNewFile();
+ assertEquals(1, JavaUtils.listFiles(dir).size());
+ assertEquals(1, JavaUtils.listPaths(dir).size());
+
+ File symlink = new File(tmp, "symlink");
+ Files.createSymbolicLink(symlink.toPath(), dir.toPath());
+ assertEquals(1, JavaUtils.listFiles(symlink).size());
+ assertEquals(1, JavaUtils.listPaths(symlink).size());
+ }
}
diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml
index adfc55d28c357..60ad971573997 100644
--- a/common/network-shuffle/pom.xml
+++ b/common/network-shuffle/pom.xml
@@ -42,6 +42,11 @@
${project.version}
+
+ org.apache.commons
+ commons-lang3
+
+
io.dropwizard.metricsmetrics-core
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/sasl/ShuffleSecretManager.java b/common/network-shuffle/src/main/java/org/apache/spark/network/sasl/ShuffleSecretManager.java
index d67f2a3099d35..625cb2e1257da 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/sasl/ShuffleSecretManager.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/sasl/ShuffleSecretManager.java
@@ -55,7 +55,7 @@ public void registerApp(String appId, String shuffleSecret) {
// to the applicationId since the secrets change between application attempts on yarn.
shuffleSecretMap.put(appId, shuffleSecret);
logger.info("Registered shuffle secret for application {}",
- MDC.of(LogKeys.APP_ID$.MODULE$, appId));
+ MDC.of(LogKeys.APP_ID, appId));
}
/**
@@ -72,7 +72,7 @@ public void registerApp(String appId, ByteBuffer shuffleSecret) {
public void unregisterApp(String appId) {
shuffleSecretMap.remove(appId);
logger.info("Unregistered shuffle secret for application {}",
- MDC.of(LogKeys.APP_ID$.MODULE$, appId));
+ MDC.of(LogKeys.APP_ID, appId));
}
/**
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/AppsWithRecoveryDisabled.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/AppsWithRecoveryDisabled.java
index 6a029a1083a47..7a0b316a3a8ea 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/AppsWithRecoveryDisabled.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/AppsWithRecoveryDisabled.java
@@ -18,11 +18,10 @@
package org.apache.spark.network.shuffle;
import java.util.Collections;
+import java.util.Objects;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
-import com.google.common.base.Preconditions;
-
/**
* Stores the applications which have recovery disabled.
*/
@@ -41,8 +40,7 @@ private AppsWithRecoveryDisabled() {
* @param appId application id
*/
public static void disableRecoveryOfApp(String appId) {
- Preconditions.checkNotNull(appId);
- INSTANCE.appsWithRecoveryDisabled.add(appId);
+ INSTANCE.appsWithRecoveryDisabled.add(Objects.requireNonNull(appId));
}
/**
@@ -51,8 +49,7 @@ public static void disableRecoveryOfApp(String appId) {
* @return true if the application is enabled for recovery; false otherwise.
*/
public static boolean isRecoveryEnabledForApp(String appId) {
- Preconditions.checkNotNull(appId);
- return !INSTANCE.appsWithRecoveryDisabled.contains(appId);
+ return !INSTANCE.appsWithRecoveryDisabled.contains(Objects.requireNonNull(appId));
}
/**
@@ -60,7 +57,6 @@ public static boolean isRecoveryEnabledForApp(String appId) {
* @param appId application id
*/
public static void removeApp(String appId) {
- Preconditions.checkNotNull(appId);
- INSTANCE.appsWithRecoveryDisabled.remove(appId);
+ INSTANCE.appsWithRecoveryDisabled.remove(Objects.requireNonNull(appId));
}
}
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
index dcb0a52b0d66c..ceb5d64699744 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockStoreClient.java
@@ -173,7 +173,7 @@ public void onSuccess(ByteBuffer response) {
((LocalDirsForExecutors) msgObj).getLocalDirsByExec());
} catch (Throwable t) {
logger.warn("Error while trying to get the host local dirs for {}", t.getCause(),
- MDC.of(LogKeys.EXECUTOR_IDS$.MODULE$, Arrays.toString(getLocalDirsMessage.execIds)));
+ MDC.of(LogKeys.EXECUTOR_IDS, Arrays.toString(getLocalDirsMessage.execIds)));
hostLocalDirsCompletable.completeExceptionally(t);
}
}
@@ -181,7 +181,7 @@ public void onSuccess(ByteBuffer response) {
@Override
public void onFailure(Throwable t) {
logger.warn("Error while trying to get the host local dirs for {}", t.getCause(),
- MDC.of(LogKeys.EXECUTOR_IDS$.MODULE$, Arrays.toString(getLocalDirsMessage.execIds)));
+ MDC.of(LogKeys.EXECUTOR_IDS, Arrays.toString(getLocalDirsMessage.execIds)));
hostLocalDirsCompletable.completeExceptionally(t);
}
});
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java
index 31ed10ad76f8f..298611cc8567f 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java
@@ -20,10 +20,9 @@
import java.io.FileNotFoundException;
import java.net.ConnectException;
-import com.google.common.base.Throwables;
-
import org.apache.spark.annotation.Evolving;
import org.apache.spark.network.server.BlockPushNonFatalFailure;
+import org.apache.spark.network.util.JavaUtils;
/**
* Plugs into {@link RetryingBlockTransferor} to further control when an exception should be retried
@@ -105,12 +104,12 @@ class BlockFetchErrorHandler implements ErrorHandler {
@Override
public boolean shouldRetryError(Throwable t) {
- return !Throwables.getStackTraceAsString(t).contains(STALE_SHUFFLE_BLOCK_FETCH);
+ return !JavaUtils.stackTraceToString(t).contains(STALE_SHUFFLE_BLOCK_FETCH);
}
@Override
public boolean shouldLogError(Throwable t) {
- return !Throwables.getStackTraceAsString(t).contains(STALE_SHUFFLE_BLOCK_FETCH);
+ return !JavaUtils.stackTraceToString(t).contains(STALE_SHUFFLE_BLOCK_FETCH);
}
}
}
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java
index 5d33bfb345a9e..45d0ff69de900 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java
@@ -21,8 +21,10 @@
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.Iterator;
import java.util.Map;
+import java.util.Objects;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.function.Function;
@@ -35,8 +37,6 @@
import com.codahale.metrics.Timer;
import com.codahale.metrics.Counter;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Sets;
import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;
@@ -199,7 +199,7 @@ protected void handleMessage(
} else if (msgObj instanceof GetLocalDirsForExecutors msg) {
checkAuth(client, msg.appId);
- Set execIdsForBlockResolver = Sets.newHashSet(msg.execIds);
+ Set execIdsForBlockResolver = new HashSet<>(Set.of(msg.execIds));
boolean fetchMergedBlockDirs = execIdsForBlockResolver.remove(SHUFFLE_MERGER_IDENTIFIER);
Map localDirs = blockManager.getLocalDirs(msg.appId,
execIdsForBlockResolver);
@@ -224,9 +224,9 @@ protected void handleMessage(
} else if (msgObj instanceof RemoveShuffleMerge msg) {
checkAuth(client, msg.appId);
logger.info("Removing shuffle merge data for application {} shuffle {} shuffleMerge {}",
- MDC.of(LogKeys.APP_ID$.MODULE$, msg.appId),
- MDC.of(LogKeys.SHUFFLE_ID$.MODULE$, msg.shuffleId),
- MDC.of(LogKeys.SHUFFLE_MERGE_ID$.MODULE$, msg.shuffleMergeId));
+ MDC.of(LogKeys.APP_ID, msg.appId),
+ MDC.of(LogKeys.SHUFFLE_ID, msg.shuffleId),
+ MDC.of(LogKeys.SHUFFLE_MERGE_ID, msg.shuffleMergeId));
mergeManager.removeShuffleMerge(msg);
} else if (msgObj instanceof DiagnoseCorruption msg) {
checkAuth(client, msg.appId);
@@ -585,7 +585,7 @@ public boolean hasNext() {
@Override
public ManagedBuffer next() {
- ManagedBuffer block = Preconditions.checkNotNull(mergeManager.getMergedBlockData(
+ ManagedBuffer block = Objects.requireNonNull(mergeManager.getMergedBlockData(
appId, shuffleId, shuffleMergeId, reduceIds[reduceIdx], chunkIds[reduceIdx][chunkIdx]));
if (chunkIdx < chunkIds[reduceIdx].length - 1) {
chunkIdx += 1;
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java
index 97723f77723d4..4fdd39c3471fc 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java
@@ -19,6 +19,7 @@
import java.io.IOException;
import java.nio.ByteBuffer;
+import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
@@ -27,7 +28,6 @@
import java.util.concurrent.Future;
import com.codahale.metrics.MetricSet;
-import com.google.common.collect.Lists;
import org.apache.spark.internal.LogKeys;
import org.apache.spark.internal.MDC;
@@ -82,7 +82,7 @@ public void init(String appId) {
this.appId = appId;
TransportContext context = new TransportContext(
transportConf, new NoOpRpcHandler(), true, true);
- List bootstraps = Lists.newArrayList();
+ List bootstraps = new ArrayList<>();
if (authEnabled) {
bootstraps.add(new AuthClientBootstrap(transportConf, appId, secretKeyHolder));
}
@@ -106,7 +106,7 @@ private void setComparableAppAttemptId(String appAttemptId) {
} catch (NumberFormatException e) {
logger.warn("Push based shuffle requires comparable application attemptId, " +
"but the appAttemptId {} cannot be parsed to Integer", e,
- MDC.of(LogKeys.APP_ATTEMPT_ID$.MODULE$, appAttemptId));
+ MDC.of(LogKeys.APP_ATTEMPT_ID, appAttemptId));
}
}
@@ -221,8 +221,8 @@ public void onFailure(Throwable e) {
});
} catch (Exception e) {
logger.error("Exception while sending finalizeShuffleMerge request to {}:{}", e,
- MDC.of(LogKeys.HOST$.MODULE$, host),
- MDC.of(LogKeys.PORT$.MODULE$, port));
+ MDC.of(LogKeys.HOST, host),
+ MDC.of(LogKeys.PORT, port));
listener.onShuffleMergeFailure(e);
}
}
@@ -322,8 +322,8 @@ public void onSuccess(ByteBuffer response) {
} catch (Throwable t) {
logger.warn("Error trying to remove blocks {} via external shuffle service from " +
"executor: {}", t,
- MDC.of(LogKeys.BLOCK_IDS$.MODULE$, Arrays.toString(blockIds)),
- MDC.of(LogKeys.EXECUTOR_ID$.MODULE$, execId));
+ MDC.of(LogKeys.BLOCK_IDS, Arrays.toString(blockIds)),
+ MDC.of(LogKeys.EXECUTOR_ID, execId));
numRemovedBlocksFuture.complete(0);
}
}
@@ -331,8 +331,8 @@ public void onSuccess(ByteBuffer response) {
@Override
public void onFailure(Throwable e) {
logger.warn("Error trying to remove blocks {} via external shuffle service from " +
- "executor: {}", e, MDC.of(LogKeys.BLOCK_IDS$.MODULE$, Arrays.toString(blockIds)),
- MDC.of(LogKeys.EXECUTOR_ID$.MODULE$, execId));
+ "executor: {}", e, MDC.of(LogKeys.BLOCK_IDS, Arrays.toString(blockIds)),
+ MDC.of(LogKeys.EXECUTOR_ID, execId));
numRemovedBlocksFuture.complete(0);
}
});
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java
index e43eedd8b25eb..b3002833fce1a 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java
@@ -20,15 +20,13 @@
import java.io.*;
import java.nio.charset.StandardCharsets;
import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executor;
import java.util.concurrent.Executors;
import java.util.stream.Collectors;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-import org.apache.commons.lang3.tuple.Pair;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.databind.ObjectMapper;
@@ -37,7 +35,6 @@
import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache;
import com.google.common.cache.Weigher;
-import com.google.common.collect.Maps;
import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;
@@ -56,6 +53,7 @@
import org.apache.spark.network.util.JavaUtils;
import org.apache.spark.network.util.NettyUtils;
import org.apache.spark.network.util.TransportConf;
+import org.apache.spark.util.Pair;
/**
* Manages converting shuffle BlockIds into physical segments of local files, from a process outside
@@ -134,11 +132,11 @@ public ShuffleIndexInformation load(String filePath) throws IOException {
db = DBProvider.initDB(dbBackend, this.registeredExecutorFile, CURRENT_VERSION, mapper);
if (db != null) {
logger.info("Use {} as the implementation of {}",
- MDC.of(LogKeys.SHUFFLE_DB_BACKEND_NAME$.MODULE$, dbBackend),
- MDC.of(LogKeys.SHUFFLE_DB_BACKEND_KEY$.MODULE$, Constants.SHUFFLE_SERVICE_DB_BACKEND));
+ MDC.of(LogKeys.SHUFFLE_DB_BACKEND_NAME, dbBackend),
+ MDC.of(LogKeys.SHUFFLE_DB_BACKEND_KEY, Constants.SHUFFLE_SERVICE_DB_BACKEND));
executors = reloadRegisteredExecutors(db);
} else {
- executors = Maps.newConcurrentMap();
+ executors = new ConcurrentHashMap<>();
}
this.directoryCleaner = directoryCleaner;
}
@@ -154,8 +152,8 @@ public void registerExecutor(
ExecutorShuffleInfo executorInfo) {
AppExecId fullId = new AppExecId(appId, execId);
logger.info("Registered executor {} with {}",
- MDC.of(LogKeys.APP_EXECUTOR_ID$.MODULE$, fullId),
- MDC.of(LogKeys.EXECUTOR_SHUFFLE_INFO$.MODULE$, executorInfo));
+ MDC.of(LogKeys.APP_EXECUTOR_ID, fullId),
+ MDC.of(LogKeys.EXECUTOR_SHUFFLE_INFO, executorInfo));
try {
if (db != null && AppsWithRecoveryDisabled.isRecoveryEnabledForApp(appId)) {
byte[] key = dbAppExecKey(fullId);
@@ -221,8 +219,8 @@ public ManagedBuffer getRddBlockData(
*/
public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
logger.info("Application {} removed, cleanupLocalDirs = {}",
- MDC.of(LogKeys.APP_ID$.MODULE$, appId),
- MDC.of(LogKeys.CLEANUP_LOCAL_DIRS$.MODULE$, cleanupLocalDirs));
+ MDC.of(LogKeys.APP_ID, appId),
+ MDC.of(LogKeys.CLEANUP_LOCAL_DIRS, cleanupLocalDirs));
Iterator> it = executors.entrySet().iterator();
while (it.hasNext()) {
Map.Entry entry = it.next();
@@ -237,14 +235,14 @@ public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
db.delete(dbAppExecKey(fullId));
} catch (IOException e) {
logger.error("Error deleting {} from executor state db", e,
- MDC.of(LogKeys.APP_ID$.MODULE$, appId));
+ MDC.of(LogKeys.APP_ID, appId));
}
}
if (cleanupLocalDirs) {
logger.info("Cleaning up executor {}'s {} local dirs",
- MDC.of(LogKeys.APP_EXECUTOR_ID$.MODULE$, fullId),
- MDC.of(LogKeys.NUM_LOCAL_DIRS$.MODULE$, executor.localDirs.length));
+ MDC.of(LogKeys.APP_EXECUTOR_ID, fullId),
+ MDC.of(LogKeys.NUM_LOCAL_DIRS, executor.localDirs.length));
// Execute the actual deletion in a different thread, as it may take some time.
directoryCleaner.execute(() -> deleteExecutorDirs(executor.localDirs));
@@ -259,18 +257,18 @@ public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
*/
public void executorRemoved(String executorId, String appId) {
logger.info("Clean up non-shuffle and non-RDD files associated with the finished executor {}",
- MDC.of(LogKeys.EXECUTOR_ID$.MODULE$, executorId));
+ MDC.of(LogKeys.EXECUTOR_ID, executorId));
AppExecId fullId = new AppExecId(appId, executorId);
final ExecutorShuffleInfo executor = executors.get(fullId);
if (executor == null) {
// Executor not registered, skip clean up of the local directories.
logger.info("Executor is not registered (appId={}, execId={})",
- MDC.of(LogKeys.APP_ID$.MODULE$, appId),
- MDC.of(LogKeys.EXECUTOR_ID$.MODULE$, executorId));
+ MDC.of(LogKeys.APP_ID, appId),
+ MDC.of(LogKeys.EXECUTOR_ID, executorId));
} else {
logger.info("Cleaning up non-shuffle and non-RDD files in executor {}'s {} local dirs",
- MDC.of(LogKeys.APP_EXECUTOR_ID$.MODULE$, fullId),
- MDC.of(LogKeys.NUM_LOCAL_DIRS$.MODULE$, executor.localDirs.length));
+ MDC.of(LogKeys.APP_EXECUTOR_ID, fullId),
+ MDC.of(LogKeys.NUM_LOCAL_DIRS, executor.localDirs.length));
// Execute the actual deletion in a different thread, as it may take some time.
directoryCleaner.execute(() -> deleteNonShuffleServiceServedFiles(executor.localDirs));
@@ -288,7 +286,7 @@ private void deleteExecutorDirs(String[] dirs) {
logger.debug("Successfully cleaned up directory: {}", localDir);
} catch (Exception e) {
logger.error("Failed to delete directory: {}", e,
- MDC.of(LogKeys.PATH$.MODULE$, localDir));
+ MDC.of(LogKeys.PATH, localDir));
}
}
}
@@ -311,7 +309,7 @@ private void deleteNonShuffleServiceServedFiles(String[] dirs) {
localDir);
} catch (Exception e) {
logger.error("Failed to delete files not served by shuffle service in directory: {}", e,
- MDC.of(LogKeys.PATH$.MODULE$, localDir));
+ MDC.of(LogKeys.PATH, localDir));
}
}
}
@@ -384,7 +382,7 @@ public int removeBlocks(String appId, String execId, String[] blockIds) {
numRemovedBlocks++;
} else {
logger.warn("Failed to delete block: {}",
- MDC.of(LogKeys.PATH$.MODULE$, file.getAbsolutePath()));
+ MDC.of(LogKeys.PATH, file.getAbsolutePath()));
}
}
return numRemovedBlocks;
@@ -400,7 +398,7 @@ public Map getLocalDirs(String appId, Set execIds) {
}
return Pair.of(exec, info.localDirs);
})
- .collect(Collectors.toMap(Pair::getKey, Pair::getValue));
+ .collect(Collectors.toMap(Pair::getLeft, Pair::getRight));
}
/**
@@ -451,10 +449,7 @@ public int hashCode() {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("appId", appId)
- .append("execId", execId)
- .toString();
+ return "ExternalShuffleBlockResolver[appId=" + appId + ",execId=" + execId + "]";
}
}
@@ -477,7 +472,7 @@ private static AppExecId parseDbAppExecKey(String s) throws IOException {
@VisibleForTesting
static ConcurrentMap reloadRegisteredExecutors(DB db)
throws IOException {
- ConcurrentMap registeredExecutors = Maps.newConcurrentMap();
+ ConcurrentMap registeredExecutors = new ConcurrentHashMap<>();
if (db != null) {
try (DBIterator itr = db.iterator()) {
itr.seek(APP_KEY_PREFIX.getBytes(StandardCharsets.UTF_8));
@@ -489,7 +484,7 @@ static ConcurrentMap reloadRegisteredExecutors(D
}
AppExecId id = parseDbAppExecKey(key);
logger.info("Reloading registered executors: {}",
- MDC.of(LogKeys.APP_EXECUTOR_ID$.MODULE$, id));
+ MDC.of(LogKeys.APP_EXECUTOR_ID, id));
ExecutorShuffleInfo shuffleInfo =
mapper.readValue(e.getValue(), ExecutorShuffleInfo.class);
registeredExecutors.put(id, shuffleInfo);
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedBlockMeta.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedBlockMeta.java
index 5541b7460ac96..ca8d9bbe65500 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedBlockMeta.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedBlockMeta.java
@@ -20,8 +20,8 @@
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
+import java.util.Objects;
-import com.google.common.base.Preconditions;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import org.roaringbitmap.RoaringBitmap;
@@ -43,7 +43,7 @@ public class MergedBlockMeta {
public MergedBlockMeta(int numChunks, ManagedBuffer chunksBitmapBuffer) {
this.numChunks = numChunks;
- this.chunksBitmapBuffer = Preconditions.checkNotNull(chunksBitmapBuffer);
+ this.chunksBitmapBuffer = Objects.requireNonNull(chunksBitmapBuffer);
}
public int getNumChunks() {
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java
index d90ca1a88a267..05158a6600d0d 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java
@@ -21,8 +21,6 @@
import java.util.Arrays;
import java.util.Map;
-import com.google.common.base.Preconditions;
-
import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;
import org.apache.spark.network.buffer.ManagedBuffer;
@@ -34,6 +32,7 @@
import org.apache.spark.network.shuffle.protocol.BlockPushReturnCode;
import org.apache.spark.network.shuffle.protocol.BlockTransferMessage;
import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
/**
* Similar to {@link OneForOneBlockFetcher}, but for pushing blocks to remote shuffle service to
@@ -90,7 +89,7 @@ public void onSuccess(ByteBuffer response) {
ReturnCode returnCode = BlockPushNonFatalFailure.getReturnCode(pushResponse.returnCode);
if (returnCode != ReturnCode.SUCCESS) {
String blockId = pushResponse.failureBlockId;
- Preconditions.checkArgument(!blockId.isEmpty());
+ JavaUtils.checkArgument(!blockId.isEmpty(), "BlockID should not be empty");
checkAndFailRemainingBlocks(index, new BlockPushNonFatalFailure(returnCode,
BlockPushNonFatalFailure.getErrorMsg(blockId, returnCode)));
} else {
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
index 6e9bd548f5327..a48208bad5b8c 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
@@ -55,7 +55,6 @@
import com.codahale.metrics.Metric;
import com.codahale.metrics.MetricSet;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache;
@@ -187,8 +186,8 @@ public ShuffleIndexInformation load(String filePath) throws IOException {
db = DBProvider.initDB(dbBackend, this.recoveryFile, CURRENT_VERSION, mapper);
if (db != null) {
logger.info("Use {} as the implementation of {}",
- MDC.of(LogKeys.SHUFFLE_DB_BACKEND_NAME$.MODULE$, dbBackend),
- MDC.of(LogKeys.SHUFFLE_DB_BACKEND_KEY$.MODULE$, Constants.SHUFFLE_SERVICE_DB_BACKEND));
+ MDC.of(LogKeys.SHUFFLE_DB_BACKEND_NAME, dbBackend),
+ MDC.of(LogKeys.SHUFFLE_DB_BACKEND_KEY, Constants.SHUFFLE_SERVICE_DB_BACKEND));
reloadAndCleanUpAppShuffleInfo(db);
}
this.pushMergeMetrics = new PushMergeMetrics();
@@ -211,7 +210,7 @@ public boolean shouldLogError(Throwable t) {
protected AppShuffleInfo validateAndGetAppShuffleInfo(String appId) {
// TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
AppShuffleInfo appShuffleInfo = appsShuffleInfo.get(appId);
- Preconditions.checkArgument(appShuffleInfo != null,
+ JavaUtils.checkArgument(appShuffleInfo != null,
"application " + appId + " is not registered or NM was restarted.");
return appShuffleInfo;
}
@@ -234,10 +233,10 @@ AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo(
if (mergePartitionsInfo == null) {
logger.info("{} attempt {} shuffle {} shuffleMerge {}: creating a new shuffle " +
"merge metadata",
- MDC.of(LogKeys.APP_ID$.MODULE$, appShuffleInfo.appId),
- MDC.of(LogKeys.APP_ATTEMPT_ID$.MODULE$, appShuffleInfo.attemptId),
- MDC.of(LogKeys.SHUFFLE_ID$.MODULE$, shuffleId),
- MDC.of(LogKeys.SHUFFLE_MERGE_ID$.MODULE$, shuffleMergeId));
+ MDC.of(LogKeys.APP_ID, appShuffleInfo.appId),
+ MDC.of(LogKeys.APP_ATTEMPT_ID, appShuffleInfo.attemptId),
+ MDC.of(LogKeys.SHUFFLE_ID, shuffleId),
+ MDC.of(LogKeys.SHUFFLE_MERGE_ID, shuffleMergeId));
return new AppShuffleMergePartitionsInfo(shuffleMergeId, false);
} else {
int latestShuffleMergeId = mergePartitionsInfo.shuffleMergeId;
@@ -256,10 +255,10 @@ AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo(
shuffleId, latestShuffleMergeId);
logger.info("{}: creating a new shuffle merge metadata since received " +
"shuffleMergeId {} is higher than latest shuffleMergeId {}",
- MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID$.MODULE$,
+ MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID,
currentAppAttemptShuffleMergeId),
- MDC.of(LogKeys.SHUFFLE_MERGE_ID$.MODULE$, shuffleMergeId),
- MDC.of(LogKeys.LATEST_SHUFFLE_MERGE_ID$.MODULE$, latestShuffleMergeId));
+ MDC.of(LogKeys.SHUFFLE_MERGE_ID, shuffleMergeId),
+ MDC.of(LogKeys.LATEST_SHUFFLE_MERGE_ID, latestShuffleMergeId));
submitCleanupTask(() ->
closeAndDeleteOutdatedPartitions(currentAppAttemptShuffleMergeId,
mergePartitionsInfo.shuffleMergePartitions));
@@ -293,13 +292,13 @@ AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo(
} catch (IOException e) {
logger.error("{} attempt {} shuffle {} shuffleMerge {}: cannot create merged shuffle " +
"partition with data file {}, index file {}, and meta file {}",
- MDC.of(LogKeys.APP_ID$.MODULE$, appShuffleInfo.appId),
- MDC.of(LogKeys.APP_ATTEMPT_ID$.MODULE$, appShuffleInfo.attemptId),
- MDC.of(LogKeys.SHUFFLE_ID$.MODULE$, shuffleId),
- MDC.of(LogKeys.SHUFFLE_MERGE_ID$.MODULE$, shuffleMergeId),
- MDC.of(LogKeys.DATA_FILE$.MODULE$, dataFile.getAbsolutePath()),
- MDC.of(LogKeys.INDEX_FILE$.MODULE$, indexFile.getAbsolutePath()),
- MDC.of(LogKeys.META_FILE$.MODULE$, metaFile.getAbsolutePath()));
+ MDC.of(LogKeys.APP_ID, appShuffleInfo.appId),
+ MDC.of(LogKeys.APP_ATTEMPT_ID, appShuffleInfo.attemptId),
+ MDC.of(LogKeys.SHUFFLE_ID, shuffleId),
+ MDC.of(LogKeys.SHUFFLE_MERGE_ID, shuffleMergeId),
+ MDC.of(LogKeys.DATA_FILE, dataFile.getAbsolutePath()),
+ MDC.of(LogKeys.INDEX_FILE, indexFile.getAbsolutePath()),
+ MDC.of(LogKeys.META_FILE, metaFile.getAbsolutePath()));
throw new RuntimeException(
String.format("Cannot initialize merged shuffle partition for appId %s shuffleId %s "
+ "shuffleMergeId %s reduceId %s", appShuffleInfo.appId, shuffleId, shuffleMergeId,
@@ -411,8 +410,8 @@ private void removeOldApplicationAttemptsFromDb(AppShuffleInfo info) {
@Override
public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
logger.info("Application {} removed, cleanupLocalDirs = {}",
- MDC.of(LogKeys.APP_ID$.MODULE$, appId),
- MDC.of(LogKeys.CLEANUP_LOCAL_DIRS$.MODULE$, cleanupLocalDirs));
+ MDC.of(LogKeys.APP_ID, appId),
+ MDC.of(LogKeys.CLEANUP_LOCAL_DIRS, cleanupLocalDirs));
// Cleanup the DB within critical section to gain the consistency between
// DB and in-memory hashmap.
AtomicReference ref = new AtomicReference<>(null);
@@ -523,7 +522,7 @@ void removeAppAttemptPathInfoFromDB(String appId, int attemptId) {
db.delete(key);
} catch (Exception e) {
logger.error("Failed to remove the application attempt {} local path in DB", e,
- MDC.of(LogKeys.APP_ATTEMPT_ID$.MODULE$, appAttemptId));
+ MDC.of(LogKeys.APP_ATTEMPT_ID, appAttemptId));
}
}
}
@@ -593,10 +592,10 @@ void deleteMergedFiles(
}
}
logger.info("Delete {} data files, {} index files, {} meta files for {}",
- MDC.of(LogKeys.NUM_DATA_FILES$.MODULE$, dataFilesDeleteCnt),
- MDC.of(LogKeys.NUM_INDEX_FILES$.MODULE$, indexFilesDeleteCnt),
- MDC.of(LogKeys.NUM_META_FILES$.MODULE$, metaFilesDeleteCnt),
- MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID$.MODULE$, appAttemptShuffleMergeId));
+ MDC.of(LogKeys.NUM_DATA_FILES, dataFilesDeleteCnt),
+ MDC.of(LogKeys.NUM_INDEX_FILES, indexFilesDeleteCnt),
+ MDC.of(LogKeys.NUM_META_FILES, metaFilesDeleteCnt),
+ MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID, appAttemptShuffleMergeId));
}
/**
@@ -609,7 +608,7 @@ void removeAppShufflePartitionInfoFromDB(AppAttemptShuffleMergeId appAttemptShuf
db.delete(getDbAppAttemptShufflePartitionKey(appAttemptShuffleMergeId));
} catch (Exception e) {
logger.error("Error deleting {} from application shuffle merged partition info in DB", e,
- MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID$.MODULE$, appAttemptShuffleMergeId));
+ MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID, appAttemptShuffleMergeId));
}
}
}
@@ -629,7 +628,7 @@ void deleteExecutorDirs(AppShuffleInfo appShuffleInfo) {
}
} catch (Exception e) {
logger.error("Failed to delete directory: {}", e,
- MDC.of(LogKeys.PATH$.MODULE$, localDir));
+ MDC.of(LogKeys.PATH, localDir));
}
}
}
@@ -759,10 +758,10 @@ public ByteBuffer getCompletionResponse() {
@Override
public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) {
logger.info("{} attempt {} shuffle {} shuffleMerge {}: finalize shuffle merge",
- MDC.of(LogKeys.APP_ID$.MODULE$, msg.appId),
- MDC.of(LogKeys.APP_ATTEMPT_ID$.MODULE$, msg.appAttemptId),
- MDC.of(LogKeys.SHUFFLE_ID$.MODULE$, msg.shuffleId),
- MDC.of(LogKeys.SHUFFLE_MERGE_ID$.MODULE$, msg.shuffleMergeId));
+ MDC.of(LogKeys.APP_ID, msg.appId),
+ MDC.of(LogKeys.APP_ATTEMPT_ID, msg.appAttemptId),
+ MDC.of(LogKeys.SHUFFLE_ID, msg.shuffleId),
+ MDC.of(LogKeys.SHUFFLE_MERGE_ID, msg.shuffleMergeId));
AppShuffleInfo appShuffleInfo = validateAndGetAppShuffleInfo(msg.appId);
if (appShuffleInfo.attemptId != msg.appAttemptId) {
// If finalizeShuffleMerge from a former application attempt, it is considered late,
@@ -846,12 +845,12 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) {
} catch (IOException ioe) {
logger.warn("{} attempt {} shuffle {} shuffleMerge {}: exception while " +
"finalizing shuffle partition {}. Exception message: {}",
- MDC.of(LogKeys.APP_ID$.MODULE$, msg.appId),
- MDC.of(LogKeys.APP_ATTEMPT_ID$.MODULE$, msg.appAttemptId),
- MDC.of(LogKeys.SHUFFLE_ID$.MODULE$, msg.shuffleId),
- MDC.of(LogKeys.SHUFFLE_MERGE_ID$.MODULE$, msg.shuffleMergeId),
- MDC.of(LogKeys.REDUCE_ID$.MODULE$, partition.reduceId),
- MDC.of(LogKeys.EXCEPTION$.MODULE$, ioe.getMessage()));
+ MDC.of(LogKeys.APP_ID, msg.appId),
+ MDC.of(LogKeys.APP_ATTEMPT_ID, msg.appAttemptId),
+ MDC.of(LogKeys.SHUFFLE_ID, msg.shuffleId),
+ MDC.of(LogKeys.SHUFFLE_MERGE_ID, msg.shuffleMergeId),
+ MDC.of(LogKeys.REDUCE_ID, partition.reduceId),
+ MDC.of(LogKeys.EXCEPTION, ioe.getMessage()));
} finally {
partition.cleanable.clean();
}
@@ -863,10 +862,10 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) {
appShuffleInfo.shuffles.get(msg.shuffleId).setReduceIds(Ints.toArray(reduceIds));
}
logger.info("{} attempt {} shuffle {} shuffleMerge {}: finalization of shuffle merge completed",
- MDC.of(LogKeys.APP_ID$.MODULE$, msg.appId),
- MDC.of(LogKeys.APP_ATTEMPT_ID$.MODULE$, msg.appAttemptId),
- MDC.of(LogKeys.SHUFFLE_ID$.MODULE$, msg.shuffleId),
- MDC.of(LogKeys.SHUFFLE_MERGE_ID$.MODULE$, msg.shuffleMergeId));
+ MDC.of(LogKeys.APP_ID, msg.appId),
+ MDC.of(LogKeys.APP_ATTEMPT_ID, msg.appAttemptId),
+ MDC.of(LogKeys.SHUFFLE_ID, msg.shuffleId),
+ MDC.of(LogKeys.SHUFFLE_MERGE_ID, msg.shuffleMergeId));
return mergeStatuses;
}
@@ -934,8 +933,8 @@ public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
if (originalAppShuffleInfo.get() != null) {
AppShuffleInfo appShuffleInfo = originalAppShuffleInfo.get();
logger.warn("Cleanup shuffle info and merged shuffle files for {}_{} as new " +
- "application attempt registered", MDC.of(LogKeys.APP_ID$.MODULE$, appId),
- MDC.of(LogKeys.APP_ATTEMPT_ID$.MODULE$, appShuffleInfo.attemptId));
+ "application attempt registered", MDC.of(LogKeys.APP_ID, appId),
+ MDC.of(LogKeys.APP_ATTEMPT_ID, appShuffleInfo.attemptId));
// Clean up all the merge shuffle related information in the DB for the former attempt
submitCleanupTask(
() -> closeAndDeletePartitionsIfNeeded(appShuffleInfo, true)
@@ -992,12 +991,12 @@ private void shutdownMergedShuffleCleanerNow() {
List unfinishedTasks = mergedShuffleCleaner.shutdownNow();
logger.warn("There are still {} tasks not completed in mergedShuffleCleaner " +
"after {} ms.",
- MDC.of(LogKeys.COUNT$.MODULE$, unfinishedTasks.size()),
- MDC.of(LogKeys.TIMEOUT$.MODULE$, cleanerShutdownTimeout * 1000L));
+ MDC.of(LogKeys.COUNT, unfinishedTasks.size()),
+ MDC.of(LogKeys.TIMEOUT, cleanerShutdownTimeout * 1000L));
// Wait a while for tasks to respond to being cancelled
if (!mergedShuffleCleaner.awaitTermination(cleanerShutdownTimeout, TimeUnit.SECONDS)) {
logger.warn("mergedShuffleCleaner did not terminate in {} ms.",
- MDC.of(LogKeys.TIMEOUT$.MODULE$, cleanerShutdownTimeout * 1000L));
+ MDC.of(LogKeys.TIMEOUT, cleanerShutdownTimeout * 1000L));
}
} catch (InterruptedException ignored) {
Thread.currentThread().interrupt();
@@ -1017,7 +1016,7 @@ private void writeAppPathsInfoToDb(String appId, int attemptId, AppPathsInfo app
db.put(key, value);
} catch (Exception e) {
logger.error("Error saving registered app paths info for {}", e,
- MDC.of(LogKeys.APP_ATTEMPT_ID$.MODULE$, appAttemptId));
+ MDC.of(LogKeys.APP_ATTEMPT_ID, appAttemptId));
}
}
}
@@ -1035,7 +1034,7 @@ private void writeAppAttemptShuffleMergeInfoToDB(
db.put(dbKey, new byte[0]);
} catch (Exception e) {
logger.error("Error saving active app shuffle partition {}", e,
- MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID$.MODULE$, appAttemptShuffleMergeId));
+ MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID, appAttemptShuffleMergeId));
}
}
}
@@ -1137,7 +1136,7 @@ List reloadActiveAppAttemptsPathInfo(DB db) throws IOException {
dbKeysToBeRemoved.add(getDbAppAttemptPathsKey(existingAppAttemptId));
} catch (IOException e) {
logger.error("Failed to get the DB key for {}", e,
- MDC.of(LogKeys.APP_ATTEMPT_ID$.MODULE$, existingAppAttemptId));
+ MDC.of(LogKeys.APP_ATTEMPT_ID, existingAppAttemptId));
}
}
return new AppShuffleInfo(
@@ -1187,7 +1186,7 @@ List reloadFinalizedAppAttemptsShuffleMergeInfo(DB db) throws IOExceptio
getDbAppAttemptShufflePartitionKey(appAttemptShuffleMergeId));
} catch (Exception e) {
logger.error("Error getting the DB key for {}", e, MDC.of(
- LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID$.MODULE$, appAttemptShuffleMergeId));
+ LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID, appAttemptShuffleMergeId));
}
}
return new AppShuffleMergePartitionsInfo(partitionId.shuffleMergeId, true);
@@ -1216,7 +1215,7 @@ void removeOutdatedKeyValuesInDB(List dbKeysToBeRemoved) {
db.delete(key);
} catch (Exception e) {
logger.error("Error deleting dangling key {} in DB", e,
- MDC.of(LogKeys.KEY$.MODULE$, key));
+ MDC.of(LogKeys.KEY, key));
}
}
);
@@ -1267,12 +1266,12 @@ private PushBlockStreamCallback(
String streamId,
AppShufflePartitionInfo partitionInfo,
int mapIndex) {
- Preconditions.checkArgument(mergeManager != null);
+ JavaUtils.checkArgument(mergeManager != null, "mergeManager is null");
this.mergeManager = mergeManager;
- Preconditions.checkArgument(appShuffleInfo != null);
+ JavaUtils.checkArgument(appShuffleInfo != null, "appShuffleInfo is null");
this.appShuffleInfo = appShuffleInfo;
this.streamId = streamId;
- Preconditions.checkArgument(partitionInfo != null);
+ JavaUtils.checkArgument(partitionInfo != null, "partitionInfo is null");
this.partitionInfo = partitionInfo;
this.mapIndex = mapIndex;
abortIfNecessary();
@@ -1599,7 +1598,7 @@ public void onComplete(String streamId) throws IOException {
public void onFailure(String streamId, Throwable throwable) throws IOException {
if (ERROR_HANDLER.shouldLogError(throwable)) {
logger.error("Encountered issue when merging {}", throwable,
- MDC.of(LogKeys.STREAM_ID$.MODULE$, streamId));
+ MDC.of(LogKeys.STREAM_ID, streamId));
} else {
logger.debug("Encountered issue when merging {}", streamId, throwable);
}
@@ -1719,7 +1718,7 @@ public AppAttemptShuffleMergeId(
@JsonProperty("attemptId") int attemptId,
@JsonProperty("shuffleId") int shuffleId,
@JsonProperty("shuffleMergeId") int shuffleMergeId) {
- Preconditions.checkArgument(appId != null, "app id is null");
+ JavaUtils.checkArgument(appId != null, "app id is null");
this.appId = appId;
this.attemptId = attemptId;
this.shuffleId = shuffleId;
@@ -1860,8 +1859,8 @@ void updateChunkInfo(long chunkOffset, int mapIndex) throws IOException {
indexMetaUpdateFailed = false;
} catch (IOException ioe) {
logger.warn("{} reduceId {} update to index/meta failed",
- MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID$.MODULE$, appAttemptShuffleMergeId),
- MDC.of(LogKeys.REDUCE_ID$.MODULE$, reduceId));
+ MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID, appAttemptShuffleMergeId),
+ MDC.of(LogKeys.REDUCE_ID, reduceId));
indexMetaUpdateFailed = true;
// Any exception here is propagated to the caller and the caller can decide whether to
// abort or not.
@@ -1913,8 +1912,8 @@ private void finalizePartition() throws IOException {
private void deleteAllFiles() {
if (!dataFile.delete()) {
logger.info("Error deleting data file for {} reduceId {}",
- MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID$.MODULE$, appAttemptShuffleMergeId),
- MDC.of(LogKeys.REDUCE_ID$.MODULE$, reduceId));
+ MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID, appAttemptShuffleMergeId),
+ MDC.of(LogKeys.REDUCE_ID, reduceId));
}
metaFile.delete();
indexFile.delete();
@@ -1983,22 +1982,22 @@ private void closeAllFiles(
}
} catch (IOException ioe) {
logger.warn("Error closing data channel for {} reduceId {}",
- MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID$.MODULE$, appAttemptShuffleMergeId),
- MDC.of(LogKeys.REDUCE_ID$.MODULE$, reduceId));
+ MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID, appAttemptShuffleMergeId),
+ MDC.of(LogKeys.REDUCE_ID, reduceId));
}
try {
metaFile.close();
} catch (IOException ioe) {
logger.warn("Error closing meta file for {} reduceId {}",
- MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID$.MODULE$, appAttemptShuffleMergeId),
- MDC.of(LogKeys.REDUCE_ID$.MODULE$, reduceId));
+ MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID, appAttemptShuffleMergeId),
+ MDC.of(LogKeys.REDUCE_ID, reduceId));
}
try {
indexFile.close();
} catch (IOException ioe) {
logger.warn("Error closing index file for {} reduceId {}",
- MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID$.MODULE$, appAttemptShuffleMergeId),
- MDC.of(LogKeys.REDUCE_ID$.MODULE$, reduceId));
+ MDC.of(LogKeys.APP_ATTEMPT_SHUFFLE_MERGE_ID, appAttemptShuffleMergeId),
+ MDC.of(LogKeys.REDUCE_ID, reduceId));
}
}
}
@@ -2043,9 +2042,9 @@ private AppPathsInfo(
this.subDirsPerLocalDir = subDirsPerLocalDir;
if (logger.isInfoEnabled()) {
logger.info("Updated active local dirs {} and sub dirs {} for application {}",
- MDC.of(LogKeys.PATHS$.MODULE$, Arrays.toString(activeLocalDirs)),
- MDC.of(LogKeys.NUM_SUB_DIRS$.MODULE$, subDirsPerLocalDir),
- MDC.of(LogKeys.APP_ID$.MODULE$, appId));
+ MDC.of(LogKeys.PATHS, Arrays.toString(activeLocalDirs)),
+ MDC.of(LogKeys.NUM_SUB_DIRS, subDirsPerLocalDir),
+ MDC.of(LogKeys.APP_ID, appId));
}
}
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockTransferor.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockTransferor.java
index 31c454f63a92e..1dae2d54120cb 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockTransferor.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockTransferor.java
@@ -25,8 +25,6 @@
import java.util.concurrent.TimeUnit;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Sets;
import com.google.common.util.concurrent.Uninterruptibles;
import org.apache.spark.internal.SparkLogger;
@@ -35,6 +33,7 @@
import org.apache.spark.internal.MDC;
import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.sasl.SaslTimeoutException;
+import org.apache.spark.network.util.JavaUtils;
import org.apache.spark.network.util.NettyUtils;
import org.apache.spark.network.util.TransportConf;
@@ -131,7 +130,7 @@ public RetryingBlockTransferor(
this.listener = listener;
this.maxRetries = conf.maxIORetries();
this.retryWaitTime = conf.ioRetryWaitTimeMs();
- this.outstandingBlocksIds = Sets.newLinkedHashSet();
+ this.outstandingBlocksIds = new LinkedHashSet<>();
Collections.addAll(outstandingBlocksIds, blockIds);
this.currentListener = new RetryingBlockTransferListener();
this.errorHandler = errorHandler;
@@ -182,13 +181,13 @@ private void transferAllOutstanding() {
} catch (Exception e) {
if (numRetries > 0) {
logger.error("Exception while beginning {} of {} outstanding blocks (after {} retries)", e,
- MDC.of(LogKeys.TRANSFER_TYPE$.MODULE$, listener.getTransferType()),
- MDC.of(LogKeys.NUM_BLOCKS$.MODULE$, blockIdsToTransfer.length),
- MDC.of(LogKeys.NUM_RETRY$.MODULE$, numRetries));
+ MDC.of(LogKeys.TRANSFER_TYPE, listener.getTransferType()),
+ MDC.of(LogKeys.NUM_BLOCKS, blockIdsToTransfer.length),
+ MDC.of(LogKeys.NUM_RETRY, numRetries));
} else {
logger.error("Exception while beginning {} of {} outstanding blocks", e,
- MDC.of(LogKeys.TRANSFER_TYPE$.MODULE$, listener.getTransferType()),
- MDC.of(LogKeys.NUM_BLOCKS$.MODULE$, blockIdsToTransfer.length));
+ MDC.of(LogKeys.TRANSFER_TYPE, listener.getTransferType()),
+ MDC.of(LogKeys.NUM_BLOCKS, blockIdsToTransfer.length));
}
if (shouldRetry(e) && initiateRetry(e)) {
// successfully initiated a retry
@@ -216,11 +215,11 @@ synchronized boolean initiateRetry(Throwable e) {
currentListener = new RetryingBlockTransferListener();
logger.info("Retrying {} ({}/{}) for {} outstanding blocks after {} ms",
- MDC.of(LogKeys.TRANSFER_TYPE$.MODULE$, listener.getTransferType()),
- MDC.of(LogKeys.NUM_RETRY$.MODULE$, retryCount),
- MDC.of(LogKeys.MAX_ATTEMPTS$.MODULE$, maxRetries),
- MDC.of(LogKeys.NUM_BLOCKS$.MODULE$, outstandingBlocksIds.size()),
- MDC.of(LogKeys.RETRY_WAIT_TIME$.MODULE$, retryWaitTime));
+ MDC.of(LogKeys.TRANSFER_TYPE, listener.getTransferType()),
+ MDC.of(LogKeys.NUM_RETRY, retryCount),
+ MDC.of(LogKeys.MAX_ATTEMPTS, maxRetries),
+ MDC.of(LogKeys.NUM_BLOCKS, outstandingBlocksIds.size()),
+ MDC.of(LogKeys.RETRY_WAIT_TIME, retryWaitTime));
try {
executorService.execute(() -> {
@@ -247,7 +246,7 @@ private synchronized boolean shouldRetry(Throwable e) {
// If this is a non SASL request failure, reduce earlier SASL failures from retryCount
// since some subsequent SASL attempt was successful
if (!isSaslTimeout && saslRetryCount > 0) {
- Preconditions.checkState(retryCount >= saslRetryCount,
+ JavaUtils.checkState(retryCount >= saslRetryCount,
"retryCount must be greater than or equal to saslRetryCount");
retryCount -= saslRetryCount;
saslRetryCount = 0;
@@ -282,7 +281,7 @@ private void handleBlockTransferSuccess(String blockId, ManagedBuffer data) {
// If there were SASL failures earlier, remove them from retryCount, as there was
// a SASL success (and some other request post bootstrap was also successful).
if (saslRetryCount > 0) {
- Preconditions.checkState(retryCount >= saslRetryCount,
+ JavaUtils.checkState(retryCount >= saslRetryCount,
"retryCount must be greater than or equal to saslRetryCount");
retryCount -= saslRetryCount;
saslRetryCount = 0;
@@ -311,9 +310,9 @@ private void handleBlockTransferFailure(String blockId, Throwable exception) {
} else {
if (errorHandler.shouldLogError(exception)) {
logger.error("Failed to {} block {}, and will not retry ({} retries)", exception,
- MDC.of(LogKeys.TRANSFER_TYPE$.MODULE$, listener.getTransferType()),
- MDC.of(LogKeys.BLOCK_ID$.MODULE$, blockId),
- MDC.of(LogKeys.NUM_RETRY$.MODULE$,retryCount));
+ MDC.of(LogKeys.TRANSFER_TYPE, listener.getTransferType()),
+ MDC.of(LogKeys.BLOCK_ID, blockId),
+ MDC.of(LogKeys.NUM_RETRY,retryCount));
} else {
logger.debug(
String.format("Failed to %s block %s, and will not retry (%s retries)",
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/checksum/ShuffleChecksumHelper.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/checksum/ShuffleChecksumHelper.java
index 62fcda701d948..2dbf38be954db 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/checksum/ShuffleChecksumHelper.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/checksum/ShuffleChecksumHelper.java
@@ -21,8 +21,6 @@
import java.util.concurrent.TimeUnit;
import java.util.zip.*;
-import com.google.common.io.ByteStreams;
-
import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;
import org.apache.spark.internal.LogKeys;
@@ -88,7 +86,7 @@ public static String getChecksumFileName(String blockName, String algorithm) {
private static long readChecksumByReduceId(File checksumFile, int reduceId) throws IOException {
try (DataInputStream in = new DataInputStream(new FileInputStream(checksumFile))) {
- ByteStreams.skipFully(in, reduceId * 8L);
+ in.skipNBytes(reduceId * 8L);
return in.readLong();
}
}
@@ -156,7 +154,7 @@ public static Cause diagnoseCorruption(
} catch (FileNotFoundException e) {
// Even if checksum is enabled, a checksum file may not exist if error throws during writing.
logger.warn("Checksum file {} doesn't exit",
- MDC.of(LogKeys.PATH$.MODULE$, checksumFile.getName()));
+ MDC.of(LogKeys.PATH, checksumFile.getName()));
cause = Cause.UNKNOWN_ISSUE;
} catch (Exception e) {
logger.warn("Unable to diagnose shuffle block corruption", e);
@@ -169,9 +167,9 @@ public static Cause diagnoseCorruption(
checksumByReader, checksumByWriter, checksumByReCalculation);
} else {
logger.info("Shuffle corruption diagnosis took {} ms, checksum file {}, cause {}",
- MDC.of(LogKeys.TIME$.MODULE$, duration),
- MDC.of(LogKeys.PATH$.MODULE$, checksumFile.getAbsolutePath()),
- MDC.of(LogKeys.REASON$.MODULE$, cause));
+ MDC.of(LogKeys.TIME, duration),
+ MDC.of(LogKeys.PATH, checksumFile.getAbsolutePath()),
+ MDC.of(LogKeys.REASON, cause));
}
return cause;
}
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/AbstractFetchShuffleBlocks.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/AbstractFetchShuffleBlocks.java
index 0fca27cf26dfa..2bc57cc52f2cd 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/AbstractFetchShuffleBlocks.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/AbstractFetchShuffleBlocks.java
@@ -17,7 +17,8 @@
package org.apache.spark.network.shuffle.protocol;
-import com.google.common.base.Objects;
+import java.util.Objects;
+
import io.netty.buffer.ByteBuf;
import org.apache.commons.lang3.builder.ToStringBuilder;
@@ -43,12 +44,14 @@ protected AbstractFetchShuffleBlocks(
this.shuffleId = shuffleId;
}
+ // checkstyle.off: RegexpSinglelineJava
public ToStringBuilder toStringHelper() {
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
.append("appId", appId)
.append("execId", execId)
.append("shuffleId", shuffleId);
}
+ // checkstyle.on: RegexpSinglelineJava
/**
* Returns number of blocks in the request.
@@ -61,7 +64,7 @@ public boolean equals(Object o) {
if (o == null || getClass() != o.getClass()) return false;
AbstractFetchShuffleBlocks that = (AbstractFetchShuffleBlocks) o;
return shuffleId == that.shuffleId
- && Objects.equal(appId, that.appId) && Objects.equal(execId, that.execId);
+ && Objects.equals(appId, that.appId) && Objects.equals(execId, that.execId);
}
@Override
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockPushReturnCode.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockPushReturnCode.java
index 05347c671e002..f4149b6875b26 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockPushReturnCode.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockPushReturnCode.java
@@ -19,10 +19,7 @@
import java.util.Objects;
-import com.google.common.base.Preconditions;
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
import org.apache.spark.network.protocol.Encoders;
import org.apache.spark.network.server.BlockPushNonFatalFailure;
@@ -43,7 +40,7 @@ public class BlockPushReturnCode extends BlockTransferMessage {
public final String failureBlockId;
public BlockPushReturnCode(byte returnCode, String failureBlockId) {
- Preconditions.checkNotNull(BlockPushNonFatalFailure.getReturnCode(returnCode));
+ Objects.requireNonNull(BlockPushNonFatalFailure.getReturnCode(returnCode));
this.returnCode = returnCode;
this.failureBlockId = failureBlockId;
}
@@ -60,10 +57,8 @@ public int hashCode() {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("returnCode", returnCode)
- .append("failureBlockId", failureBlockId)
- .toString();
+ return "BlockPushReturnCode[returnCode=" + returnCode +
+ ",failureBlockId=" + failureBlockId + "]";
}
@Override
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlocksRemoved.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlocksRemoved.java
index 2a050ce40b84b..9942d68297595 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlocksRemoved.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlocksRemoved.java
@@ -20,8 +20,6 @@
import java.util.Objects;
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
// Needed by ScalaDoc. See SPARK-7726
import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type;
@@ -44,9 +42,7 @@ public int hashCode() {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("numRemovedBlocks", numRemovedBlocks)
- .toString();
+ return "BlocksRemoved[numRemovedBlocks=" + numRemovedBlocks + "]";
}
@Override
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/CorruptionCause.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/CorruptionCause.java
index 5690eee53bd13..d9b9d4d8f36c2 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/CorruptionCause.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/CorruptionCause.java
@@ -18,8 +18,6 @@
package org.apache.spark.network.shuffle.protocol;
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
import org.apache.spark.network.shuffle.checksum.Cause;
@@ -38,9 +36,7 @@ protected Type type() {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("cause", cause)
- .toString();
+ return "CorruptionCause[cause=" + cause + "]";
}
@Override
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/DiagnoseCorruption.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/DiagnoseCorruption.java
index 620b5ad71cd75..e509f45a9f0e3 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/DiagnoseCorruption.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/DiagnoseCorruption.java
@@ -18,8 +18,6 @@
package org.apache.spark.network.shuffle.protocol;
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
import org.apache.spark.network.protocol.Encoders;
/** Request to get the cause of a corrupted block. Returns {@link CorruptionCause} */
@@ -56,15 +54,9 @@ protected Type type() {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("appId", appId)
- .append("execId", execId)
- .append("shuffleId", shuffleId)
- .append("mapId", mapId)
- .append("reduceId", reduceId)
- .append("checksum", checksum)
- .append("algorithm", algorithm)
- .toString();
+ return "DiagnoseCorruption[appId=" + appId + ",execId=" + execId + ",shuffleId=" + shuffleId +
+ ",mapId=" + mapId + ",reduceId=" + reduceId + ",checksum=" + checksum +
+ ",algorithm=" + algorithm + "]";
}
@Override
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java
index 8a3ccdef2920b..c53ab911c30da 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java
@@ -23,8 +23,6 @@
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
import org.apache.spark.network.protocol.Encodable;
import org.apache.spark.network.protocol.Encoders;
@@ -60,11 +58,8 @@ public int hashCode() {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("localDirs", Arrays.toString(localDirs))
- .append("subDirsPerLocalDir", subDirsPerLocalDir)
- .append("shuffleManager", shuffleManager)
- .toString();
+ return "ExecutorShuffleInfo[localDirs=" + Arrays.toString(localDirs) +
+ ",subDirsPerLocalDir=" + subDirsPerLocalDir + ",shuffleManager=" + shuffleManager + "]";
}
@Override
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlockChunks.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlockChunks.java
index cf4cbcf1ed08e..a6e1ce374b07f 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlockChunks.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlockChunks.java
@@ -60,11 +60,10 @@ public FetchShuffleBlockChunks(
@Override
public String toString() {
- return toStringHelper()
- .append("shuffleMergeId", shuffleMergeId)
- .append("reduceIds", Arrays.toString(reduceIds))
- .append("chunkIds", Arrays.deepToString(chunkIds))
- .toString();
+ return "FetchShuffleBlockChunks[appId=" + appId + ",execId=" + execId +
+ ",shuffleId=" + shuffleId + ",shuffleMergeId=" + shuffleMergeId +
+ ",reduceIds=" + Arrays.toString(reduceIds) +
+ ",chunkIds=" + Arrays.deepToString(chunkIds) + "]";
}
@Override
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java
index 68550a2fba86e..686207767ca1e 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java
@@ -62,11 +62,9 @@ public FetchShuffleBlocks(
@Override
public String toString() {
- return toStringHelper()
- .append("mapIds", Arrays.toString(mapIds))
- .append("reduceIds", Arrays.deepToString(reduceIds))
- .append("batchFetchEnabled", batchFetchEnabled)
- .toString();
+ return "FetchShuffleBlocks[appId=" + appId + ",execId=" + execId + ",shuffleId=" + shuffleId +
+ ",mapIds=" + Arrays.toString(mapIds) + ",reduceIds=" + Arrays.deepToString(reduceIds) +
+ ",batchFetchEnabled=" + batchFetchEnabled + "]";
}
@Override
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FinalizeShuffleMerge.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FinalizeShuffleMerge.java
index cd5e005348f42..61152f48a85ba 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FinalizeShuffleMerge.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FinalizeShuffleMerge.java
@@ -17,10 +17,9 @@
package org.apache.spark.network.shuffle.protocol;
-import com.google.common.base.Objects;
+import java.util.Objects;
+
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
import org.apache.spark.network.protocol.Encoders;
@@ -54,23 +53,19 @@ protected BlockTransferMessage.Type type() {
@Override
public int hashCode() {
- return Objects.hashCode(appId, appAttemptId, shuffleId, shuffleMergeId);
+ return Objects.hash(appId, appAttemptId, shuffleId, shuffleMergeId);
}
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("appId", appId)
- .append("attemptId", appAttemptId)
- .append("shuffleId", shuffleId)
- .append("shuffleMergeId", shuffleMergeId)
- .toString();
+ return "FinalizeShuffleMerge[appId=" + appId + ",attemptId=" + appAttemptId +
+ ",shuffleId=" + shuffleId + ",shuffleMergeId=" + shuffleMergeId + "]";
}
@Override
public boolean equals(Object other) {
if (other instanceof FinalizeShuffleMerge o) {
- return Objects.equal(appId, o.appId)
+ return Objects.equals(appId, o.appId)
&& appAttemptId == o.appAttemptId
&& shuffleId == o.shuffleId
&& shuffleMergeId == o.shuffleMergeId;
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/GetLocalDirsForExecutors.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/GetLocalDirsForExecutors.java
index f118f0604d9e9..8bd106c94c283 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/GetLocalDirsForExecutors.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/GetLocalDirsForExecutors.java
@@ -21,8 +21,6 @@
import java.util.Objects;
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
import org.apache.spark.network.protocol.Encoders;
@@ -49,10 +47,7 @@ public int hashCode() {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("appId", appId)
- .append("execIds", Arrays.toString(execIds))
- .toString();
+ return "GetLocalDirsForExecutors[appId=" + appId + ",execIds=" + Arrays.toString(execIds) + "]";
}
@Override
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/LocalDirsForExecutors.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/LocalDirsForExecutors.java
index b65f351d3cf3e..060b565d420fd 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/LocalDirsForExecutors.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/LocalDirsForExecutors.java
@@ -20,8 +20,6 @@
import java.util.*;
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
import org.apache.spark.network.protocol.Encoders;
@@ -64,11 +62,9 @@ public int hashCode() {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("execIds", Arrays.toString(execIds))
- .append("numLocalDirsByExec", Arrays.toString(numLocalDirsByExec))
- .append("allLocalDirs", Arrays.toString(allLocalDirs))
- .toString();
+ return "LocalDirsForExecutors[execIds=" + Arrays.toString(execIds) +
+ ",numLocalDirsByExec=" + Arrays.toString(numLocalDirsByExec) +
+ ",allLocalDirs=" + Arrays.toString(allLocalDirs) + "]";
}
@Override
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergeStatuses.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergeStatuses.java
index 892c3a5e77958..d21449016972f 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergeStatuses.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergeStatuses.java
@@ -19,10 +19,9 @@
import java.util.Arrays;
-import com.google.common.base.Objects;
+import java.util.Objects;
+
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
import org.roaringbitmap.RoaringBitmap;
import org.apache.spark.network.protocol.Encoders;
@@ -86,18 +85,15 @@ public int hashCode() {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("shuffleId", shuffleId)
- .append("shuffleMergeId", shuffleMergeId)
- .append("reduceId size", reduceIds.length)
- .toString();
+ return "MergeStatuses[shuffleId=" + shuffleId + ",shuffleMergeId=" + shuffleMergeId +
+ ",reduceId size=" + reduceIds.length + "]";
}
@Override
public boolean equals(Object other) {
if (other instanceof MergeStatuses o) {
- return Objects.equal(shuffleId, o.shuffleId)
- && Objects.equal(shuffleMergeId, o.shuffleMergeId)
+ return Objects.equals(shuffleId, o.shuffleId)
+ && Objects.equals(shuffleMergeId, o.shuffleMergeId)
&& Arrays.equals(bitmaps, o.bitmaps)
&& Arrays.equals(reduceIds, o.reduceIds)
&& Arrays.equals(sizes, o.sizes);
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/OpenBlocks.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/OpenBlocks.java
index 49288eef5c5de..87b40eb1fc6ac 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/OpenBlocks.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/OpenBlocks.java
@@ -21,8 +21,6 @@
import java.util.Objects;
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
import org.apache.spark.network.protocol.Encoders;
@@ -51,11 +49,8 @@ public int hashCode() {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("appId", appId)
- .append("execId", execId)
- .append("blockIds", Arrays.toString(blockIds))
- .toString();
+ return "OpenBlocks[appId=" + appId + ",execId=" + execId + ",blockIds=" +
+ Arrays.toString(blockIds) + "]";
}
@Override
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java
index ceab54a1c0615..20e6e79c31980 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java
@@ -17,11 +17,9 @@
package org.apache.spark.network.shuffle.protocol;
-import com.google.common.base.Objects;
-import io.netty.buffer.ByteBuf;
+import java.util.Objects;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
+import io.netty.buffer.ByteBuf;
import org.apache.spark.network.protocol.Encoders;
@@ -68,27 +66,21 @@ protected Type type() {
@Override
public int hashCode() {
- return Objects.hashCode(appId, appAttemptId, shuffleId, shuffleMergeId, mapIndex , reduceId,
+ return Objects.hash(appId, appAttemptId, shuffleId, shuffleMergeId, mapIndex , reduceId,
index);
}
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("appId", appId)
- .append("attemptId", appAttemptId)
- .append("shuffleId", shuffleId)
- .append("shuffleMergeId", shuffleMergeId)
- .append("mapIndex", mapIndex)
- .append("reduceId", reduceId)
- .append("index", index)
- .toString();
+ return "PushBlockStream[appId=" + appId + ",attemptId=" + appAttemptId +
+ ",shuffleId=" + shuffleId + ",shuffleMergeId=" + shuffleMergeId + ",mapIndex=" + mapIndex +
+ ",reduceId=" + reduceId + ",index=" + index + "]";
}
@Override
public boolean equals(Object other) {
if (other instanceof PushBlockStream o) {
- return Objects.equal(appId, o.appId)
+ return Objects.equals(appId, o.appId)
&& appAttemptId == o.appAttemptId
&& shuffleId == o.shuffleId
&& shuffleMergeId == o.shuffleMergeId
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java
index 9805af67b9f26..a5931126e4ff8 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java
@@ -20,8 +20,6 @@
import java.util.Objects;
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
import org.apache.spark.network.protocol.Encoders;
@@ -56,11 +54,8 @@ public int hashCode() {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("appId", appId)
- .append("execId", execId)
- .append("executorInfo", executorInfo)
- .toString();
+ return "RegisterExecutor[appId=" + appId + ", execId=" + execId +
+ ",executorInfo=" + executorInfo + "]";
}
@Override
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RemoveBlocks.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RemoveBlocks.java
index 7032942331c3e..2743824b3d21c 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RemoveBlocks.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RemoveBlocks.java
@@ -21,8 +21,6 @@
import java.util.Objects;
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
import org.apache.spark.network.protocol.Encoders;
@@ -51,11 +49,8 @@ public int hashCode() {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("appId", appId)
- .append("execId", execId)
- .append("blockIds", Arrays.toString(blockIds))
- .toString();
+ return "RemoveBlocks[appId=" + appId + ",execId=" + execId +
+ ",blockIds=" + Arrays.toString(blockIds) + "]";
}
@Override
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RemoveShuffleMerge.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RemoveShuffleMerge.java
index 8ce2e05e6097d..ac6d981b2e081 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RemoveShuffleMerge.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RemoveShuffleMerge.java
@@ -17,10 +17,9 @@
package org.apache.spark.network.shuffle.protocol;
-import com.google.common.base.Objects;
+import java.util.Objects;
+
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
import org.apache.spark.network.protocol.Encoders;
@@ -54,23 +53,19 @@ protected Type type() {
@Override
public int hashCode() {
- return Objects.hashCode(appId, appAttemptId, shuffleId, shuffleMergeId);
+ return Objects.hash(appId, appAttemptId, shuffleId, shuffleMergeId);
}
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("appId", appId)
- .append("attemptId", appAttemptId)
- .append("shuffleId", shuffleId)
- .append("shuffleMergeId", shuffleMergeId)
- .toString();
+ return "RemoveShuffleMerge[appId=" + appId + ",attemptId=" + appAttemptId +
+ ",shuffleId=" + shuffleId + ",shuffleMergeId=" + shuffleMergeId + "]";
}
@Override
public boolean equals(Object other) {
if (other != null && other instanceof RemoveShuffleMerge o) {
- return Objects.equal(appId, o.appId)
+ return Objects.equals(appId, o.appId)
&& appAttemptId == o.appAttemptId
&& shuffleId == o.shuffleId
&& shuffleMergeId == o.shuffleMergeId;
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/StreamHandle.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/StreamHandle.java
index aebd6f0d5a620..629e3c472e0a6 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/StreamHandle.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/StreamHandle.java
@@ -20,8 +20,6 @@
import java.util.Objects;
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
// Needed by ScalaDoc. See SPARK-7726
import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type;
@@ -49,10 +47,7 @@ public int hashCode() {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("streamId", streamId)
- .append("numChunks", numChunks)
- .toString();
+ return "StreamHandle[streamId=" + streamId + ",numChunks=" + numChunks + "]";
}
@Override
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlock.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlock.java
index fad187971e09a..9222134e6bb7f 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlock.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlock.java
@@ -21,8 +21,6 @@
import java.util.Objects;
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
import org.apache.spark.network.protocol.Encoders;
@@ -68,13 +66,8 @@ public int hashCode() {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("appId", appId)
- .append("execId", execId)
- .append("blockId", blockId)
- .append("metadata size", metadata.length)
- .append("block size", blockData.length)
- .toString();
+ return "UploadBlock[appId=" + appId + ",execId=" + execId + ",blockId=" + blockId +
+ ",metadata size=" + metadata.length + ",block size=" + blockData.length + "]";
}
@Override
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlockStream.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlockStream.java
index 95d0b3835562d..45c4c5f98de74 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlockStream.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlockStream.java
@@ -21,8 +21,6 @@
import java.util.Objects;
import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
import org.apache.spark.network.protocol.Encoders;
@@ -55,10 +53,7 @@ public int hashCode() {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("blockId", blockId)
- .append("metadata size", metadata.length)
- .toString();
+ return "UploadBlockStream[blockId=" + blockId + ",metadata size=" + metadata.length + "]";
}
@Override
diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java
index ccb464c2ce5bd..0c091d88f98b5 100644
--- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java
+++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java
@@ -20,21 +20,17 @@
import java.io.File;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
-import java.nio.file.Files;
-import java.nio.file.Path;
import java.util.*;
import java.util.concurrent.Executor;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.stream.Collectors;
-import java.util.stream.Stream;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
import org.junit.jupiter.api.Test;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
+import org.apache.spark.network.util.JavaUtils;
import org.apache.spark.network.util.MapConfigProvider;
import org.apache.spark.network.util.TransportConf;
@@ -46,15 +42,15 @@ public class CleanupNonShuffleServiceServedFilesSuite {
private static final String SORT_MANAGER = "org.apache.spark.shuffle.sort.SortShuffleManager";
private static Set expectedShuffleFilesToKeep =
- ImmutableSet.of("shuffle_782_450_0.index", "shuffle_782_450_0.data");
+ Set.of("shuffle_782_450_0.index", "shuffle_782_450_0.data");
private static Set expectedShuffleAndRddFilesToKeep =
- ImmutableSet.of("shuffle_782_450_0.index", "shuffle_782_450_0.data", "rdd_12_34");
+ Set.of("shuffle_782_450_0.index", "shuffle_782_450_0.data", "rdd_12_34");
private TransportConf getConf(boolean isFetchRddEnabled) {
return new TransportConf(
"shuffle",
- new MapConfigProvider(ImmutableMap.of(
+ new MapConfigProvider(Map.of(
Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED,
Boolean.toString(isFetchRddEnabled))));
}
@@ -200,28 +196,13 @@ private static void assertStillThere(TestShuffleDataContext dataContext) {
}
}
- private static Set collectFilenames(File[] files) throws IOException {
- Set result = new HashSet<>();
- for (File file : files) {
- if (file.exists()) {
- try (Stream walk = Files.walk(file.toPath())) {
- result.addAll(walk
- .filter(Files::isRegularFile)
- .map(x -> x.toFile().getName())
- .collect(Collectors.toSet()));
- }
- }
- }
- return result;
- }
-
private static void assertContainedFilenames(
TestShuffleDataContext dataContext,
Set expectedFilenames) throws IOException {
Set collectedFilenames = new HashSet<>();
for (String localDir : dataContext.localDirs) {
- File[] dirs = new File[] { new File(localDir) };
- collectedFilenames.addAll(collectFilenames(dirs));
+ JavaUtils.listFiles(new File(localDir)).stream().map(File::getName)
+ .collect(Collectors.toCollection(() -> collectedFilenames));
}
assertEquals(expectedFilenames, collectedFilenames);
}
diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java
index f7edc8837fde7..2a3135e3c8aeb 100644
--- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java
+++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java
@@ -27,7 +27,6 @@
import com.codahale.metrics.Meter;
import com.codahale.metrics.Metric;
import com.codahale.metrics.Timer;
-import com.google.common.io.ByteStreams;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.mockito.ArgumentCaptor;
@@ -136,7 +135,7 @@ private void checkDiagnosisResult(
CheckedInputStream checkedIn = new CheckedInputStream(
blockMarkers[0].createInputStream(), checksum);
byte[] buffer = new byte[10];
- ByteStreams.readFully(checkedIn, buffer, 0, (int) blockMarkers[0].size());
+ JavaUtils.readFully(checkedIn, buffer, 0, (int) blockMarkers[0].size());
long checksumByWriter = checkedIn.getChecksum().getValue();
// when checksumByWriter == checksumRecalculated and checksumByReader != checksumByWriter
diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java
index 311827dbed4c5..488d02d63d552 100644
--- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java
+++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java
@@ -19,12 +19,11 @@
import java.io.IOException;
import java.io.InputStream;
-import java.io.InputStreamReader;
import java.nio.charset.StandardCharsets;
import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.io.CharStreams;
import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.util.JavaUtils;
import org.apache.spark.network.util.MapConfigProvider;
import org.apache.spark.network.util.TransportConf;
import org.apache.spark.network.shuffle.ExternalShuffleBlockResolver.AppExecId;
@@ -83,23 +82,17 @@ public void testSortShuffleBlocks() throws IOException {
try (InputStream block0Stream = resolver.getBlockData(
"app0", "exec0", 0, 0, 0).createInputStream()) {
- String block0 =
- CharStreams.toString(new InputStreamReader(block0Stream, StandardCharsets.UTF_8));
- assertEquals(sortBlock0, block0);
+ assertEquals(sortBlock0, JavaUtils.toString(block0Stream));
}
try (InputStream block1Stream = resolver.getBlockData(
"app0", "exec0", 0, 0, 1).createInputStream()) {
- String block1 =
- CharStreams.toString(new InputStreamReader(block1Stream, StandardCharsets.UTF_8));
- assertEquals(sortBlock1, block1);
+ assertEquals(sortBlock1, JavaUtils.toString(block1Stream));
}
try (InputStream blocksStream = resolver.getContinuousBlocksData(
"app0", "exec0", 0, 0, 0, 2).createInputStream()) {
- String blocks =
- CharStreams.toString(new InputStreamReader(blocksStream, StandardCharsets.UTF_8));
- assertEquals(sortBlock0 + sortBlock1, blocks);
+ assertEquals(sortBlock0 + sortBlock1, JavaUtils.toString(blocksStream));
}
}
diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java
index ec71f83ba743c..59381cabe063a 100644
--- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java
+++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java
@@ -32,7 +32,6 @@
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
-import com.google.common.collect.Sets;
import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
import org.apache.spark.network.server.OneForOneStreamManager;
import org.junit.jupiter.api.AfterAll;
@@ -222,7 +221,7 @@ public void testFetchOneSort() throws Exception {
try (ExternalBlockStoreClient client = createExternalBlockStoreClient()) {
registerExecutor(client, "exec-0", dataContext0.createExecutorInfo(SORT_MANAGER));
FetchResult exec0Fetch = fetchBlocks("exec-0", new String[] { "shuffle_0_0_0" });
- assertEquals(Sets.newHashSet("shuffle_0_0_0"), exec0Fetch.successBlocks);
+ assertEquals(Set.of("shuffle_0_0_0"), exec0Fetch.successBlocks);
assertTrue(exec0Fetch.failedBlocks.isEmpty());
assertBufferListsEqual(exec0Fetch.buffers, Arrays.asList(exec0Blocks[0]));
exec0Fetch.releaseBuffers();
@@ -235,7 +234,7 @@ public void testFetchThreeSort() throws Exception {
registerExecutor(client,"exec-0", dataContext0.createExecutorInfo(SORT_MANAGER));
FetchResult exec0Fetch = fetchBlocks("exec-0",
new String[]{"shuffle_0_0_0", "shuffle_0_0_1", "shuffle_0_0_2"});
- assertEquals(Sets.newHashSet("shuffle_0_0_0", "shuffle_0_0_1", "shuffle_0_0_2"),
+ assertEquals(Set.of("shuffle_0_0_0", "shuffle_0_0_1", "shuffle_0_0_2"),
exec0Fetch.successBlocks);
assertTrue(exec0Fetch.failedBlocks.isEmpty());
assertBufferListsEqual(exec0Fetch.buffers, Arrays.asList(exec0Blocks));
@@ -256,7 +255,7 @@ public void testFetchWrongBlockId() throws Exception {
registerExecutor(client, "exec-1", dataContext0.createExecutorInfo(SORT_MANAGER));
FetchResult execFetch = fetchBlocks("exec-1", new String[]{"broadcast_1"});
assertTrue(execFetch.successBlocks.isEmpty());
- assertEquals(Sets.newHashSet("broadcast_1"), execFetch.failedBlocks);
+ assertEquals(Set.of("broadcast_1"), execFetch.failedBlocks);
}
}
@@ -267,7 +266,7 @@ public void testFetchValidRddBlock() throws Exception {
String validBlockId = "rdd_" + RDD_ID + "_" + SPLIT_INDEX_VALID_BLOCK;
FetchResult execFetch = fetchBlocks("exec-1", new String[]{validBlockId});
assertTrue(execFetch.failedBlocks.isEmpty());
- assertEquals(Sets.newHashSet(validBlockId), execFetch.successBlocks);
+ assertEquals(Set.of(validBlockId), execFetch.successBlocks);
assertBuffersEqual(new NioManagedBuffer(ByteBuffer.wrap(exec0RddBlockValid)),
execFetch.buffers.get(0));
}
@@ -280,7 +279,7 @@ public void testFetchDeletedRddBlock() throws Exception {
String missingBlockId = "rdd_" + RDD_ID + "_" + SPLIT_INDEX_MISSING_FILE;
FetchResult execFetch = fetchBlocks("exec-1", new String[]{missingBlockId});
assertTrue(execFetch.successBlocks.isEmpty());
- assertEquals(Sets.newHashSet(missingBlockId), execFetch.failedBlocks);
+ assertEquals(Set.of(missingBlockId), execFetch.failedBlocks);
}
}
@@ -310,7 +309,7 @@ public void testFetchCorruptRddBlock() throws Exception {
String corruptBlockId = "rdd_" + RDD_ID + "_" + SPLIT_INDEX_CORRUPT_LENGTH;
FetchResult execFetch = fetchBlocks("exec-1", new String[]{corruptBlockId});
assertTrue(execFetch.successBlocks.isEmpty());
- assertEquals(Sets.newHashSet(corruptBlockId), execFetch.failedBlocks);
+ assertEquals(Set.of(corruptBlockId), execFetch.failedBlocks);
}
}
@@ -321,7 +320,7 @@ public void testFetchNonexistent() throws Exception {
FetchResult execFetch = fetchBlocks("exec-0",
new String[]{"shuffle_2_0_0"});
assertTrue(execFetch.successBlocks.isEmpty());
- assertEquals(Sets.newHashSet("shuffle_2_0_0"), execFetch.failedBlocks);
+ assertEquals(Set.of("shuffle_2_0_0"), execFetch.failedBlocks);
}
}
@@ -331,8 +330,8 @@ public void testFetchWrongExecutor() throws Exception {
registerExecutor(client,"exec-0", dataContext0.createExecutorInfo(SORT_MANAGER));
FetchResult execFetch0 = fetchBlocks("exec-0", new String[]{"shuffle_0_0_0" /* right */});
FetchResult execFetch1 = fetchBlocks("exec-0", new String[]{"shuffle_1_0_0" /* wrong */});
- assertEquals(Sets.newHashSet("shuffle_0_0_0"), execFetch0.successBlocks);
- assertEquals(Sets.newHashSet("shuffle_1_0_0"), execFetch1.failedBlocks);
+ assertEquals(Set.of("shuffle_0_0_0"), execFetch0.successBlocks);
+ assertEquals(Set.of("shuffle_1_0_0"), execFetch1.failedBlocks);
}
}
@@ -343,7 +342,7 @@ public void testFetchUnregisteredExecutor() throws Exception {
FetchResult execFetch = fetchBlocks("exec-2",
new String[]{"shuffle_0_0_0", "shuffle_1_0_0"});
assertTrue(execFetch.successBlocks.isEmpty());
- assertEquals(Sets.newHashSet("shuffle_0_0_0", "shuffle_1_0_0"), execFetch.failedBlocks);
+ assertEquals(Set.of("shuffle_0_0_0", "shuffle_1_0_0"), execFetch.failedBlocks);
}
}
@@ -355,7 +354,7 @@ public void testFetchNoServer() throws Exception {
FetchResult execFetch = fetchBlocks("exec-0",
new String[]{"shuffle_1_0_0", "shuffle_1_0_1"}, clientConf, 1 /* port */);
assertTrue(execFetch.successBlocks.isEmpty());
- assertEquals(Sets.newHashSet("shuffle_1_0_0", "shuffle_1_0_1"), execFetch.failedBlocks);
+ assertEquals(Set.of("shuffle_1_0_0", "shuffle_1_0_1"), execFetch.failedBlocks);
}
}
diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java
index 76f82800c502a..170b72b409e12 100644
--- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java
+++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java
@@ -19,8 +19,8 @@
import java.io.IOException;
import java.util.Arrays;
+import java.util.Map;
-import com.google.common.collect.ImmutableMap;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
@@ -46,7 +46,7 @@ public class ExternalShuffleSecuritySuite {
protected TransportConf createTransportConf(boolean encrypt) {
if (encrypt) {
return new TransportConf("shuffle", new MapConfigProvider(
- ImmutableMap.of("spark.authenticate.enableSaslEncryption", "true")));
+ Map.of("spark.authenticate.enableSaslEncryption", "true")));
} else {
return new TransportConf("shuffle", MapConfigProvider.EMPTY);
}
diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java
index 7151d044105c7..f127568c8a333 100644
--- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java
+++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java
@@ -23,7 +23,6 @@
import java.util.LinkedHashMap;
import java.util.concurrent.atomic.AtomicInteger;
-import com.google.common.collect.Maps;
import io.netty.buffer.Unpooled;
import org.junit.jupiter.api.Test;
@@ -57,7 +56,7 @@ public class OneForOneBlockFetcherSuite {
@Test
public void testFetchOne() {
- LinkedHashMap blocks = Maps.newLinkedHashMap();
+ LinkedHashMap blocks = new LinkedHashMap<>();
blocks.put("shuffle_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[0])));
String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]);
@@ -72,7 +71,7 @@ public void testFetchOne() {
@Test
public void testUseOldProtocol() {
- LinkedHashMap blocks = Maps.newLinkedHashMap();
+ LinkedHashMap blocks = new LinkedHashMap<>();
blocks.put("shuffle_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[0])));
String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]);
@@ -91,7 +90,7 @@ public void testUseOldProtocol() {
@Test
public void testFetchThreeShuffleBlocks() {
- LinkedHashMap blocks = Maps.newLinkedHashMap();
+ LinkedHashMap blocks = new LinkedHashMap<>();
blocks.put("shuffle_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12])));
blocks.put("shuffle_0_0_1", new NioManagedBuffer(ByteBuffer.wrap(new byte[23])));
blocks.put("shuffle_0_0_2", new NettyManagedBuffer(Unpooled.wrappedBuffer(new byte[23])));
@@ -112,7 +111,7 @@ public void testFetchThreeShuffleBlocks() {
@Test
public void testBatchFetchThreeShuffleBlocks() {
- LinkedHashMap blocks = Maps.newLinkedHashMap();
+ LinkedHashMap blocks = new LinkedHashMap<>();
blocks.put("shuffle_0_0_0_3", new NioManagedBuffer(ByteBuffer.wrap(new byte[58])));
String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]);
@@ -129,7 +128,7 @@ public void testBatchFetchThreeShuffleBlocks() {
@Test
public void testFetchThree() {
- LinkedHashMap blocks = Maps.newLinkedHashMap();
+ LinkedHashMap blocks = new LinkedHashMap<>();
blocks.put("b0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12])));
blocks.put("b1", new NioManagedBuffer(ByteBuffer.wrap(new byte[23])));
blocks.put("b2", new NettyManagedBuffer(Unpooled.wrappedBuffer(new byte[23])));
@@ -148,7 +147,7 @@ public void testFetchThree() {
@Test
public void testFailure() {
- LinkedHashMap blocks = Maps.newLinkedHashMap();
+ LinkedHashMap blocks = new LinkedHashMap<>();
blocks.put("b0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12])));
blocks.put("b1", null);
blocks.put("b2", null);
@@ -168,7 +167,7 @@ public void testFailure() {
@Test
public void testFailureAndSuccess() {
- LinkedHashMap blocks = Maps.newLinkedHashMap();
+ LinkedHashMap blocks = new LinkedHashMap<>();
blocks.put("b0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12])));
blocks.put("b1", null);
blocks.put("b2", new NioManagedBuffer(ByteBuffer.wrap(new byte[21])));
@@ -190,14 +189,14 @@ public void testFailureAndSuccess() {
@Test
public void testEmptyBlockFetch() {
IllegalArgumentException e = assertThrows(IllegalArgumentException.class,
- () -> fetchBlocks(Maps.newLinkedHashMap(), new String[] {},
+ () -> fetchBlocks(new LinkedHashMap<>(), new String[] {},
new OpenBlocks("app-id", "exec-id", new String[] {}), conf));
assertEquals("Zero-sized blockIds array", e.getMessage());
}
@Test
public void testFetchShuffleBlocksOrder() {
- LinkedHashMap blocks = Maps.newLinkedHashMap();
+ LinkedHashMap blocks = new LinkedHashMap<>();
blocks.put("shuffle_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[1])));
blocks.put("shuffle_0_2_1", new NioManagedBuffer(ByteBuffer.wrap(new byte[2])));
blocks.put("shuffle_0_10_2", new NettyManagedBuffer(Unpooled.wrappedBuffer(new byte[3])));
@@ -217,7 +216,7 @@ public void testFetchShuffleBlocksOrder() {
@Test
public void testBatchFetchShuffleBlocksOrder() {
- LinkedHashMap blocks = Maps.newLinkedHashMap();
+ LinkedHashMap blocks = new LinkedHashMap<>();
blocks.put("shuffle_0_0_1_2", new NioManagedBuffer(ByteBuffer.wrap(new byte[1])));
blocks.put("shuffle_0_2_2_3", new NioManagedBuffer(ByteBuffer.wrap(new byte[2])));
blocks.put("shuffle_0_10_3_4", new NettyManagedBuffer(Unpooled.wrappedBuffer(new byte[3])));
@@ -237,7 +236,7 @@ public void testBatchFetchShuffleBlocksOrder() {
@Test
public void testShuffleBlockChunksFetch() {
- LinkedHashMap blocks = Maps.newLinkedHashMap();
+ LinkedHashMap blocks = new LinkedHashMap<>();
blocks.put("shuffleChunk_0_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12])));
blocks.put("shuffleChunk_0_0_0_1", new NioManagedBuffer(ByteBuffer.wrap(new byte[23])));
blocks.put("shuffleChunk_0_0_0_2",
@@ -255,7 +254,7 @@ public void testShuffleBlockChunksFetch() {
@Test
public void testShuffleBlockChunkFetchFailure() {
- LinkedHashMap blocks = Maps.newLinkedHashMap();
+ LinkedHashMap blocks = new LinkedHashMap<>();
blocks.put("shuffleChunk_0_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12])));
blocks.put("shuffleChunk_0_0_0_1", null);
blocks.put("shuffleChunk_0_0_0_2",
diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockPusherSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockPusherSuite.java
index 32c6a8cd37eae..345ac7546af48 100644
--- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockPusherSuite.java
+++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockPusherSuite.java
@@ -23,7 +23,6 @@
import java.util.LinkedHashMap;
import java.util.Map;
-import com.google.common.collect.Maps;
import io.netty.buffer.Unpooled;
import org.junit.jupiter.api.Test;
@@ -47,7 +46,7 @@ public class OneForOneBlockPusherSuite {
@Test
public void testPushOne() {
- LinkedHashMap blocks = Maps.newLinkedHashMap();
+ LinkedHashMap blocks = new LinkedHashMap<>();
blocks.put("shufflePush_0_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[1])));
String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]);
@@ -61,7 +60,7 @@ public void testPushOne() {
@Test
public void testPushThree() {
- LinkedHashMap blocks = Maps.newLinkedHashMap();
+ LinkedHashMap blocks = new LinkedHashMap<>();
blocks.put("shufflePush_0_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12])));
blocks.put("shufflePush_0_0_1_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[23])));
blocks.put("shufflePush_0_0_2_0",
@@ -82,7 +81,7 @@ public void testPushThree() {
@Test
public void testServerFailures() {
- LinkedHashMap blocks = Maps.newLinkedHashMap();
+ LinkedHashMap blocks = new LinkedHashMap<>();
blocks.put("shufflePush_0_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12])));
blocks.put("shufflePush_0_0_1_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[0])));
blocks.put("shufflePush_0_0_2_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[0])));
@@ -102,7 +101,7 @@ public void testServerFailures() {
@Test
public void testHandlingRetriableFailures() {
- LinkedHashMap blocks = Maps.newLinkedHashMap();
+ LinkedHashMap blocks = new LinkedHashMap<>();
blocks.put("shufflePush_0_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12])));
blocks.put("shufflePush_0_0_1_0", null);
blocks.put("shufflePush_0_0_2_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[0])));
diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java
index edd5e1961a501..b7e24fe3da8fe 100644
--- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java
+++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java
@@ -39,9 +39,7 @@
import java.util.concurrent.TimeUnit;
import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.collect.ImmutableMap;
-import org.apache.commons.io.FileUtils;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
@@ -68,6 +66,7 @@
import org.apache.spark.network.shuffle.protocol.RemoveShuffleMerge;
import org.apache.spark.network.util.MapConfigProvider;
import org.apache.spark.network.util.TransportConf;
+import org.apache.spark.network.util.JavaUtils;
/**
* Tests for {@link RemoteBlockPushResolver}.
@@ -97,7 +96,7 @@ public class RemoteBlockPushResolverSuite {
public void before() throws IOException {
localDirs = createLocalDirs(2);
MapConfigProvider provider = new MapConfigProvider(
- ImmutableMap.of("spark.shuffle.push.server.minChunkSizeInMergedShuffleFile", "4"));
+ Map.of("spark.shuffle.push.server.minChunkSizeInMergedShuffleFile", "4"));
conf = new TransportConf("shuffle", provider);
pushResolver = new RemoteBlockPushResolver(conf, null);
registerExecutor(TEST_APP, prepareLocalDirs(localDirs, MERGE_DIRECTORY), MERGE_DIRECTORY_META);
@@ -107,7 +106,7 @@ public void before() throws IOException {
public void after() {
try {
for (Path local : localDirs) {
- FileUtils.deleteDirectory(local.toFile());
+ JavaUtils.deleteRecursively(local.toFile());
}
removeApplication(TEST_APP);
} catch (Exception e) {
diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockTransferorSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockTransferorSuite.java
index 84c8b1b3353f2..cbbade779ab68 100644
--- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockTransferorSuite.java
+++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockTransferorSuite.java
@@ -29,7 +29,6 @@
import java.util.concurrent.TimeoutException;
import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Sets;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.mockito.stubbing.Answer;
@@ -353,15 +352,15 @@ public void testIOExceptionFailsConnectionEvenWithSaslException()
new TimeoutException());
IOException ioException = new IOException();
List extends Map> interactions = Arrays.asList(
- ImmutableMap.of("b0", saslExceptionInitial),
- ImmutableMap.of("b0", ioException),
- ImmutableMap.of("b0", saslExceptionInitial),
- ImmutableMap.of("b0", ioException),
- ImmutableMap.of("b0", saslExceptionFinal),
+ Map.of("b0", saslExceptionInitial),
+ Map.of("b0", ioException),
+ Map.of("b0", saslExceptionInitial),
+ Map.of("b0", ioException),
+ Map.of("b0", saslExceptionFinal),
// will not get invoked because the connection fails
- ImmutableMap.of("b0", ioException),
+ Map.of("b0", ioException),
// will not get invoked
- ImmutableMap.of("b0", block0)
+ Map.of("b0", block0)
);
configMap.put("spark.shuffle.sasl.enableRetries", "true");
performInteractions(interactions, listener);
@@ -425,7 +424,7 @@ private static void configureInteractions(List extends Map> in
Stubber stub = null;
// Contains all blockIds that are referenced across all interactions.
- LinkedHashSet blockIds = Sets.newLinkedHashSet();
+ LinkedHashSet blockIds = new LinkedHashSet<>();
for (Map interaction : interactions) {
blockIds.addAll(interaction.keySet());
diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ShuffleTransportContextSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ShuffleTransportContextSuite.java
index aef3bc51bcd4b..bd9884e81ba92 100644
--- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ShuffleTransportContextSuite.java
+++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ShuffleTransportContextSuite.java
@@ -18,12 +18,11 @@
package org.apache.spark.network.shuffle;
import java.io.IOException;
+import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
-import com.google.common.collect.Lists;
-
import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufAllocator;
import io.netty.buffer.Unpooled;
@@ -74,7 +73,7 @@ ShuffleTransportContext createShuffleTransportContext(boolean separateFinalizeTh
}
private ByteBuf getDecodableMessageBuf(Message req) throws Exception {
- List out = Lists.newArrayList();
+ List out = new ArrayList<>();
ChannelHandlerContext context = mock(ChannelHandlerContext.class);
when(context.alloc()).thenReturn(ByteBufAllocator.DEFAULT);
MessageEncoder.INSTANCE.encode(context, req, out);
@@ -118,7 +117,7 @@ public void testDecodeOfFinalizeShuffleMessage() throws Exception {
try (ShuffleTransportContext shuffleTransportContext = createShuffleTransportContext(true)) {
ShuffleTransportContext.ShuffleMessageDecoder decoder =
(ShuffleTransportContext.ShuffleMessageDecoder) shuffleTransportContext.getDecoder();
- List out = Lists.newArrayList();
+ List out = new ArrayList<>();
decoder.decode(mock(ChannelHandlerContext.class), messageBuf, out);
Assertions.assertEquals(1, out.size());
@@ -137,7 +136,7 @@ public void testDecodeOfAnyOtherRpcMessage() throws Exception {
try (ShuffleTransportContext shuffleTransportContext = createShuffleTransportContext(true)) {
ShuffleTransportContext.ShuffleMessageDecoder decoder =
(ShuffleTransportContext.ShuffleMessageDecoder) shuffleTransportContext.getDecoder();
- List out = Lists.newArrayList();
+ List out = new ArrayList<>();
decoder.decode(mock(ChannelHandlerContext.class), messageBuf, out);
Assertions.assertEquals(1, out.size());
diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/SslExternalShuffleSecuritySuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/SslExternalShuffleSecuritySuite.java
index 061d63dbcd72d..a04ec60ca1c1c 100644
--- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/SslExternalShuffleSecuritySuite.java
+++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/SslExternalShuffleSecuritySuite.java
@@ -17,7 +17,7 @@
package org.apache.spark.network.shuffle;
-import com.google.common.collect.ImmutableMap;
+import java.util.Map;
import org.apache.spark.network.ssl.SslSampleConfigs;
import org.apache.spark.network.util.TransportConf;
@@ -30,9 +30,7 @@ protected TransportConf createTransportConf(boolean encrypt) {
return new TransportConf(
"shuffle",
SslSampleConfigs.createDefaultConfigProviderForRpcNamespaceWithAdditionalEntries(
- ImmutableMap.of(
- "spark.authenticate.enableSaslEncryption",
- "true")
+ Map.of("spark.authenticate.enableSaslEncryption", "true")
)
);
} else {
diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/SslShuffleTransportContextSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/SslShuffleTransportContextSuite.java
index 51463bbad5576..1a85838792d29 100644
--- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/SslShuffleTransportContextSuite.java
+++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/SslShuffleTransportContextSuite.java
@@ -17,7 +17,7 @@
package org.apache.spark.network.shuffle;
-import com.google.common.collect.ImmutableMap;
+import java.util.Map;
import org.apache.spark.network.ssl.SslSampleConfigs;
import org.apache.spark.network.util.TransportConf;
@@ -29,7 +29,7 @@ protected TransportConf createTransportConf(boolean separateFinalizeThread) {
return new TransportConf(
"shuffle",
SslSampleConfigs.createDefaultConfigProviderForRpcNamespaceWithAdditionalEntries(
- ImmutableMap.of(
+ Map.of(
"spark.shuffle.server.finalizeShuffleMergeThreadsPercent",
separateFinalizeThread ? "1" : "0")
)
diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java
index 49b17824c3c72..4b8dc33c6bf52 100644
--- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java
+++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java
@@ -22,6 +22,7 @@
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.OutputStream;
+import java.nio.file.Files;
import com.google.common.io.Closeables;
@@ -54,7 +55,7 @@ public void create() throws IOException {
localDirs[i] = JavaUtils.createDirectory(root, "spark").getAbsolutePath();
for (int p = 0; p < subDirsPerLocalDir; p ++) {
- new File(localDirs[i], String.format("%02x", p)).mkdirs();
+ Files.createDirectories(new File(localDirs[i], String.format("%02x", p)).toPath());
}
}
}
diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml
index 78289684960ed..c4451923b17a5 100644
--- a/common/network-yarn/pom.xml
+++ b/common/network-yarn/pom.xml
@@ -48,7 +48,7 @@
org.apache.sparkspark-tags_${scala.binary.version}
- test
+ provided
diff --git a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationAwareUTF8String.java b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationAwareUTF8String.java
index e455e531de0dd..2b9457c58560f 100644
--- a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationAwareUTF8String.java
+++ b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationAwareUTF8String.java
@@ -1529,9 +1529,10 @@ public static UTF8String trimRight(
}
public static UTF8String[] splitSQL(final UTF8String input, final UTF8String delim,
- final int limit, final int collationId) {
+ final int limit, final int collationId, boolean legacySplitTruncate) {
if (CollationFactory.fetchCollation(collationId).isUtf8BinaryType) {
- return input.split(delim, limit);
+ return legacySplitTruncate ?
+ input.splitLegacyTruncate(delim, limit) : input.split(delim, limit);
} else if (CollationFactory.fetchCollation(collationId).isUtf8LcaseType) {
return lowercaseSplitSQL(input, delim, limit);
} else {
diff --git a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationFactory.java b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationFactory.java
index 4bcd75a731059..59c23064858d0 100644
--- a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationFactory.java
+++ b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationFactory.java
@@ -22,7 +22,6 @@
import java.util.concurrent.ConcurrentHashMap;
import java.util.function.Function;
import java.util.function.BiFunction;
-import java.util.function.ToLongFunction;
import java.util.stream.Stream;
import com.ibm.icu.text.CollationKey;
@@ -125,10 +124,19 @@ public static class Collation {
public final String version;
/**
- * Collation sensitive hash function. Output for two UTF8Strings will be the same if they are
- * equal according to the collation.
+ * Returns the sort key of the input UTF8String. Two UTF8String values are equal iff their
+ * sort keys are equal (compared as byte arrays).
+ * The sort key is defined as follows for collations without the RTRIM modifier:
+ * - UTF8_BINARY: It is the bytes of the string.
+ * - UTF8_LCASE: It is byte array we get by replacing all invalid UTF8 sequences with the
+ * Unicode replacement character and then converting all characters of the replaced string
+ * with their lowercase equivalents (the Greek capital and Greek small sigma both map to
+ * the Greek final sigma).
+ * - ICU collations: It is the byte array returned by the ICU library for the collated string.
+ * For strings with the RTRIM modifier, we right-trim the string and return the collation key
+ * of the resulting right-trimmed string.
*/
- public final ToLongFunction hashFunction;
+ public final Function sortKeyFunction;
/**
* Potentially faster way than using comparator to compare two UTF8Strings for equality.
@@ -182,7 +190,7 @@ public Collation(
Collator collator,
Comparator comparator,
String version,
- ToLongFunction hashFunction,
+ Function sortKeyFunction,
BiFunction equalsFunction,
boolean isUtf8BinaryType,
boolean isUtf8LcaseType,
@@ -192,7 +200,7 @@ public Collation(
this.collator = collator;
this.comparator = comparator;
this.version = version;
- this.hashFunction = hashFunction;
+ this.sortKeyFunction = sortKeyFunction;
this.isUtf8BinaryType = isUtf8BinaryType;
this.isUtf8LcaseType = isUtf8LcaseType;
this.equalsFunction = equalsFunction;
@@ -581,18 +589,18 @@ private static boolean isValidCollationId(int collationId) {
protected Collation buildCollation() {
if (caseSensitivity == CaseSensitivity.UNSPECIFIED) {
Comparator comparator;
- ToLongFunction hashFunction;
+ Function sortKeyFunction;
BiFunction equalsFunction;
boolean supportsSpaceTrimming = spaceTrimming != SpaceTrimming.NONE;
if (spaceTrimming == SpaceTrimming.NONE) {
comparator = UTF8String::binaryCompare;
- hashFunction = s -> (long) s.hashCode();
+ sortKeyFunction = s -> s.getBytes();
equalsFunction = UTF8String::equals;
} else {
comparator = (s1, s2) -> applyTrimmingPolicy(s1, spaceTrimming).binaryCompare(
applyTrimmingPolicy(s2, spaceTrimming));
- hashFunction = s -> (long) applyTrimmingPolicy(s, spaceTrimming).hashCode();
+ sortKeyFunction = s -> applyTrimmingPolicy(s, spaceTrimming).getBytes();
equalsFunction = (s1, s2) -> applyTrimmingPolicy(s1, spaceTrimming).equals(
applyTrimmingPolicy(s2, spaceTrimming));
}
@@ -603,25 +611,25 @@ protected Collation buildCollation() {
null,
comparator,
CollationSpecICU.ICU_VERSION,
- hashFunction,
+ sortKeyFunction,
equalsFunction,
/* isUtf8BinaryType = */ true,
/* isUtf8LcaseType = */ false,
spaceTrimming != SpaceTrimming.NONE);
} else {
Comparator comparator;
- ToLongFunction hashFunction;
+ Function sortKeyFunction;
if (spaceTrimming == SpaceTrimming.NONE) {
comparator = CollationAwareUTF8String::compareLowerCase;
- hashFunction = s ->
- (long) CollationAwareUTF8String.lowerCaseCodePoints(s).hashCode();
+ sortKeyFunction = s ->
+ CollationAwareUTF8String.lowerCaseCodePoints(s).getBytes();
} else {
comparator = (s1, s2) -> CollationAwareUTF8String.compareLowerCase(
applyTrimmingPolicy(s1, spaceTrimming),
applyTrimmingPolicy(s2, spaceTrimming));
- hashFunction = s -> (long) CollationAwareUTF8String.lowerCaseCodePoints(
- applyTrimmingPolicy(s, spaceTrimming)).hashCode();
+ sortKeyFunction = s -> CollationAwareUTF8String.lowerCaseCodePoints(
+ applyTrimmingPolicy(s, spaceTrimming)).getBytes();
}
return new Collation(
@@ -630,7 +638,7 @@ protected Collation buildCollation() {
null,
comparator,
CollationSpecICU.ICU_VERSION,
- hashFunction,
+ sortKeyFunction,
(s1, s2) -> comparator.compare(s1, s2) == 0,
/* isUtf8BinaryType = */ false,
/* isUtf8LcaseType = */ true,
@@ -1013,19 +1021,18 @@ protected Collation buildCollation() {
collator.freeze();
Comparator comparator;
- ToLongFunction hashFunction;
+ Function sortKeyFunction;
if (spaceTrimming == SpaceTrimming.NONE) {
- hashFunction = s -> (long) collator.getCollationKey(
- s.toValidString()).hashCode();
comparator = (s1, s2) ->
collator.compare(s1.toValidString(), s2.toValidString());
+ sortKeyFunction = s -> collator.getCollationKey(s.toValidString()).toByteArray();
} else {
comparator = (s1, s2) -> collator.compare(
applyTrimmingPolicy(s1, spaceTrimming).toValidString(),
applyTrimmingPolicy(s2, spaceTrimming).toValidString());
- hashFunction = s -> (long) collator.getCollationKey(
- applyTrimmingPolicy(s, spaceTrimming).toValidString()).hashCode();
+ sortKeyFunction = s -> collator.getCollationKey(
+ applyTrimmingPolicy(s, spaceTrimming).toValidString()).toByteArray();
}
return new Collation(
@@ -1034,7 +1041,7 @@ protected Collation buildCollation() {
collator,
comparator,
ICU_VERSION,
- hashFunction,
+ sortKeyFunction,
(s1, s2) -> comparator.compare(s1, s2) == 0,
/* isUtf8BinaryType = */ false,
/* isUtf8LcaseType = */ false,
diff --git a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationSupport.java b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationSupport.java
index 135250e482b16..f950fd864c576 100644
--- a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationSupport.java
+++ b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationSupport.java
@@ -706,8 +706,10 @@ public static int collationAwareRegexFlags(final int collationId) {
public static UTF8String lowercaseRegex(final UTF8String regex) {
return UTF8String.concat(lowercaseRegexPrefix, regex);
}
- public static UTF8String collationAwareRegex(final UTF8String regex, final int collationId) {
- return supportsLowercaseRegex(collationId) ? lowercaseRegex(regex) : regex;
+ public static UTF8String collationAwareRegex(
+ final UTF8String regex, final int collationId, boolean notIgnoreEmpty) {
+ return supportsLowercaseRegex(collationId) && (notIgnoreEmpty || regex.numBytes() != 0)
+ ? lowercaseRegex(regex) : regex;
}
/**
diff --git a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/DateTimeConstants.java b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/DateTimeConstants.java
index 0ae238564d591..d52207ad860cd 100644
--- a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/DateTimeConstants.java
+++ b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/DateTimeConstants.java
@@ -45,4 +45,5 @@ public class DateTimeConstants {
public static final long NANOS_PER_MICROS = 1000L;
public static final long NANOS_PER_MILLIS = MICROS_PER_MILLIS * NANOS_PER_MICROS;
public static final long NANOS_PER_SECOND = MILLIS_PER_SECOND * NANOS_PER_MILLIS;
+ public static final long NANOS_PER_DAY = MICROS_PER_DAY * NANOS_PER_MICROS;
}
diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java
index f12408fb49313..310dbce9eaab6 100644
--- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java
+++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java
@@ -20,9 +20,8 @@
import java.nio.ByteOrder;
import java.util.Arrays;
-import com.google.common.primitives.Ints;
-
import org.apache.spark.unsafe.Platform;
+import org.apache.spark.network.util.JavaUtils;
public final class ByteArray {
@@ -169,7 +168,7 @@ public static byte[] concatWS(byte[] delimiter, byte[]... inputs) {
}
if (totalLength > 0) totalLength -= delimiter.length;
// Allocate a new byte array, and copy the inputs one by one into it
- final byte[] result = new byte[Ints.checkedCast(totalLength)];
+ final byte[] result = new byte[JavaUtils.checkedCast(totalLength)];
int offset = 0;
for (int i = 0; i < inputs.length; i++) {
byte[] input = inputs[i];
diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java
index caf8461b0b5d6..9209bd3135099 100644
--- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java
+++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java
@@ -1168,10 +1168,21 @@ public UTF8String reverse() {
}
public UTF8String repeat(int times) {
- if (times <= 0) {
+ if (times <= 0 || numBytes == 0) {
return EMPTY_UTF8;
}
+ if (times == 1) {
+ return this;
+ }
+
+ if (numBytes == 1) {
+ byte[] newBytes = new byte[times];
+ byte b = getByte(0);
+ Arrays.fill(newBytes, b);
+ return fromBytes(newBytes);
+ }
+
byte[] newBytes = new byte[Math.multiplyExact(numBytes, times)];
copyMemory(this.base, this.offset, newBytes, BYTE_ARRAY_OFFSET, numBytes);
@@ -1483,6 +1494,25 @@ public static UTF8String concatWs(UTF8String separator, UTF8String... inputs) {
}
public UTF8String[] split(UTF8String pattern, int limit) {
+ // For the empty `pattern` a `split` function ignores trailing empty strings unless original
+ // string is empty.
+ if (numBytes() != 0 && pattern.numBytes() == 0) {
+ int newLimit = limit > numChars() || limit <= 0 ? numChars() : limit;
+ byte[] input = getBytes();
+ int byteIndex = 0;
+ UTF8String[] result = new UTF8String[newLimit];
+ for (int charIndex = 0; charIndex < newLimit - 1; charIndex++) {
+ int currCharNumBytes = numBytesForFirstByte(input[byteIndex]);
+ result[charIndex] = UTF8String.fromBytes(input, byteIndex, currCharNumBytes);
+ byteIndex += currCharNumBytes;
+ }
+ result[newLimit - 1] = UTF8String.fromBytes(input, byteIndex, numBytes() - byteIndex);
+ return result;
+ }
+ return split(pattern.toString(), limit);
+ }
+
+ public UTF8String[] splitLegacyTruncate(UTF8String pattern, int limit) {
// For the empty `pattern` a `split` function ignores trailing empty strings unless original
// string is empty.
if (numBytes() != 0 && pattern.numBytes() == 0) {
diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java
index c4a66fdffdd4d..26b96155377e8 100644
--- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java
+++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java
@@ -24,7 +24,6 @@
import java.nio.charset.StandardCharsets;
import java.util.*;
-import com.google.common.collect.ImmutableMap;
import org.apache.spark.unsafe.Platform;
import org.apache.spark.unsafe.UTF8StringBuilder;
@@ -432,7 +431,7 @@ public void split() {
new UTF8String[]{fromString("a"), fromString("b")},
fromString("ab").split(fromString(""), 100));
assertArrayEquals(
- new UTF8String[]{fromString("a")},
+ new UTF8String[]{fromString("ab")},
fromString("ab").split(fromString(""), 1));
assertArrayEquals(
new UTF8String[]{fromString("")},
@@ -495,7 +494,7 @@ public void levenshteinDistance() {
public void translate() {
assertEquals(
fromString("1a2s3ae"),
- fromString("translate").translate(ImmutableMap.of(
+ fromString("translate").translate(Map.of(
"r", "1",
"n", "2",
"l", "3",
@@ -506,7 +505,7 @@ public void translate() {
fromString("translate").translate(new HashMap<>()));
assertEquals(
fromString("asae"),
- fromString("translate").translate(ImmutableMap.of(
+ fromString("translate").translate(Map.of(
"r", "\0",
"n", "\0",
"l", "\0",
@@ -514,7 +513,7 @@ public void translate() {
)));
assertEquals(
fromString("aa世b"),
- fromString("花花世界").translate(ImmutableMap.of(
+ fromString("花花世界").translate(Map.of(
"花", "a",
"界", "b"
)));
diff --git a/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/CollationFactorySuite.scala b/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/CollationFactorySuite.scala
index 8e9d33efe7a6d..ddf588b6c64c7 100644
--- a/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/CollationFactorySuite.scala
+++ b/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/CollationFactorySuite.scala
@@ -17,7 +17,8 @@
package org.apache.spark.unsafe.types
-import scala.collection.parallel.immutable.ParSeq
+import java.util.stream.IntStream
+
import scala.jdk.CollectionConverters.MapHasAsScala
import com.ibm.icu.util.ULocale
@@ -139,7 +140,7 @@ class CollationFactorySuite extends AnyFunSuite with Matchers { // scalastyle:ig
case class CollationTestCase[R](collationName: String, s1: String, s2: String, expectedResult: R)
- test("collation aware equality and hash") {
+ test("collation aware equality and sort key") {
val checks = Seq(
CollationTestCase("UTF8_BINARY", "aaa", "aaa", true),
CollationTestCase("UTF8_BINARY", "aaa", "AAA", false),
@@ -194,9 +195,9 @@ class CollationFactorySuite extends AnyFunSuite with Matchers { // scalastyle:ig
assert(collation.equalsFunction(toUTF8(testCase.s1), toUTF8(testCase.s2)) ==
testCase.expectedResult)
- val hash1 = collation.hashFunction.applyAsLong(toUTF8(testCase.s1))
- val hash2 = collation.hashFunction.applyAsLong(toUTF8(testCase.s2))
- assert((hash1 == hash2) == testCase.expectedResult)
+ val sortKey1 = collation.sortKeyFunction.apply(toUTF8(testCase.s1)).asInstanceOf[Array[Byte]]
+ val sortKey2 = collation.sortKeyFunction.apply(toUTF8(testCase.s2)).asInstanceOf[Array[Byte]]
+ assert(sortKey1.sameElements(sortKey2) == testCase.expectedResult)
})
}
@@ -293,7 +294,7 @@ class CollationFactorySuite extends AnyFunSuite with Matchers { // scalastyle:ig
(0 to 10).foreach(_ => {
val collator = fetchCollation("UNICODE").getCollator
- ParSeq(0 to 100).foreach { _ =>
+ IntStream.rangeClosed(0, 100).parallel().forEach { _ =>
collator.getCollationKey("aaa")
}
})
diff --git a/common/utils-java/pom.xml b/common/utils-java/pom.xml
new file mode 100644
index 0000000000000..ba3603f810856
--- /dev/null
+++ b/common/utils-java/pom.xml
@@ -0,0 +1,84 @@
+
+
+
+
+ 4.0.0
+
+ org.apache.spark
+ spark-parent_2.13
+ 4.1.0-SNAPSHOT
+ ../../pom.xml
+
+
+ spark-common-utils-java_2.13
+ jar
+ Spark Project Common Java Utils
+ https://spark.apache.org/
+
+ common-utils-java
+
+
+
+
+ org.apache.spark
+ spark-tags_${scala.binary.version}
+
+
+ com.fasterxml.jackson.core
+ jackson-databind
+
+
+ org.slf4j
+ slf4j-api
+
+
+
+ org.slf4j
+ jul-to-slf4j
+
+
+ org.slf4j
+ jcl-over-slf4j
+
+
+ org.apache.logging.log4j
+ log4j-slf4j2-impl
+
+
+ org.apache.logging.log4j
+ log4j-api
+
+
+ org.apache.logging.log4j
+ log4j-core
+
+
+ org.apache.logging.log4j
+ log4j-1.2-api
+
+
+ org.apache.logging.log4j
+ log4j-layout-template-json
+
+
+
+ target/scala-${scala.binary.version}/classes
+ target/scala-${scala.binary.version}/test-classes
+
+
diff --git a/common/utils/src/main/java/org/apache/spark/QueryContext.java b/common/utils-java/src/main/java/org/apache/spark/QueryContext.java
similarity index 100%
rename from common/utils/src/main/java/org/apache/spark/QueryContext.java
rename to common/utils-java/src/main/java/org/apache/spark/QueryContext.java
diff --git a/common/utils/src/main/java/org/apache/spark/QueryContextType.java b/common/utils-java/src/main/java/org/apache/spark/QueryContextType.java
similarity index 100%
rename from common/utils/src/main/java/org/apache/spark/QueryContextType.java
rename to common/utils-java/src/main/java/org/apache/spark/QueryContextType.java
diff --git a/common/utils/src/main/java/org/apache/spark/api/java/function/CoGroupFunction.java b/common/utils-java/src/main/java/org/apache/spark/api/java/function/CoGroupFunction.java
similarity index 100%
rename from common/utils/src/main/java/org/apache/spark/api/java/function/CoGroupFunction.java
rename to common/utils-java/src/main/java/org/apache/spark/api/java/function/CoGroupFunction.java
diff --git a/common/utils/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction.java b/common/utils-java/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction.java
similarity index 100%
rename from common/utils/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction.java
rename to common/utils-java/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction.java
diff --git a/common/utils/src/main/java/org/apache/spark/api/java/function/DoubleFunction.java b/common/utils-java/src/main/java/org/apache/spark/api/java/function/DoubleFunction.java
similarity index 100%
rename from common/utils/src/main/java/org/apache/spark/api/java/function/DoubleFunction.java
rename to common/utils-java/src/main/java/org/apache/spark/api/java/function/DoubleFunction.java
diff --git a/common/utils/src/main/java/org/apache/spark/api/java/function/FilterFunction.java b/common/utils-java/src/main/java/org/apache/spark/api/java/function/FilterFunction.java
similarity index 100%
rename from common/utils/src/main/java/org/apache/spark/api/java/function/FilterFunction.java
rename to common/utils-java/src/main/java/org/apache/spark/api/java/function/FilterFunction.java
diff --git a/common/utils/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java b/common/utils-java/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java
similarity index 100%
rename from common/utils/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java
rename to common/utils-java/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java
diff --git a/common/utils/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java b/common/utils-java/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java
similarity index 100%
rename from common/utils/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java
rename to common/utils-java/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java
diff --git a/common/utils/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsFunction.java b/common/utils-java/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsFunction.java
similarity index 100%
rename from common/utils/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsFunction.java
rename to common/utils-java/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsFunction.java
diff --git a/common/utils/src/main/java/org/apache/spark/api/java/function/ForeachFunction.java b/common/utils-java/src/main/java/org/apache/spark/api/java/function/ForeachFunction.java
similarity index 100%
rename from common/utils/src/main/java/org/apache/spark/api/java/function/ForeachFunction.java
rename to common/utils-java/src/main/java/org/apache/spark/api/java/function/ForeachFunction.java
diff --git a/common/utils/src/main/java/org/apache/spark/api/java/function/ForeachPartitionFunction.java b/common/utils-java/src/main/java/org/apache/spark/api/java/function/ForeachPartitionFunction.java
similarity index 100%
rename from common/utils/src/main/java/org/apache/spark/api/java/function/ForeachPartitionFunction.java
rename to common/utils-java/src/main/java/org/apache/spark/api/java/function/ForeachPartitionFunction.java
diff --git a/common/utils/src/main/java/org/apache/spark/api/java/function/Function.java b/common/utils-java/src/main/java/org/apache/spark/api/java/function/Function.java
similarity index 100%
rename from common/utils/src/main/java/org/apache/spark/api/java/function/Function.java
rename to common/utils-java/src/main/java/org/apache/spark/api/java/function/Function.java
diff --git a/common/utils/src/main/java/org/apache/spark/api/java/function/Function0.java b/common/utils-java/src/main/java/org/apache/spark/api/java/function/Function0.java
similarity index 100%
rename from common/utils/src/main/java/org/apache/spark/api/java/function/Function0.java
rename to common/utils-java/src/main/java/org/apache/spark/api/java/function/Function0.java
diff --git a/common/utils/src/main/java/org/apache/spark/api/java/function/Function2.java b/common/utils-java/src/main/java/org/apache/spark/api/java/function/Function2.java
similarity index 100%
rename from common/utils/src/main/java/org/apache/spark/api/java/function/Function2.java
rename to common/utils-java/src/main/java/org/apache/spark/api/java/function/Function2.java
diff --git a/common/utils/src/main/java/org/apache/spark/api/java/function/Function3.java b/common/utils-java/src/main/java/org/apache/spark/api/java/function/Function3.java
similarity index 100%
rename from common/utils/src/main/java/org/apache/spark/api/java/function/Function3.java
rename to common/utils-java/src/main/java/org/apache/spark/api/java/function/Function3.java
diff --git a/common/utils/src/main/java/org/apache/spark/api/java/function/Function4.java b/common/utils-java/src/main/java/org/apache/spark/api/java/function/Function4.java
similarity index 100%
rename from common/utils/src/main/java/org/apache/spark/api/java/function/Function4.java
rename to common/utils-java/src/main/java/org/apache/spark/api/java/function/Function4.java
diff --git a/common/utils/src/main/java/org/apache/spark/api/java/function/MapFunction.java b/common/utils-java/src/main/java/org/apache/spark/api/java/function/MapFunction.java
similarity index 100%
rename from common/utils/src/main/java/org/apache/spark/api/java/function/MapFunction.java
rename to common/utils-java/src/main/java/org/apache/spark/api/java/function/MapFunction.java
diff --git a/common/utils/src/main/java/org/apache/spark/api/java/function/MapGroupsFunction.java b/common/utils-java/src/main/java/org/apache/spark/api/java/function/MapGroupsFunction.java
similarity index 100%
rename from common/utils/src/main/java/org/apache/spark/api/java/function/MapGroupsFunction.java
rename to common/utils-java/src/main/java/org/apache/spark/api/java/function/MapGroupsFunction.java
diff --git a/common/utils/src/main/java/org/apache/spark/api/java/function/MapPartitionsFunction.java b/common/utils-java/src/main/java/org/apache/spark/api/java/function/MapPartitionsFunction.java
similarity index 100%
rename from common/utils/src/main/java/org/apache/spark/api/java/function/MapPartitionsFunction.java
rename to common/utils-java/src/main/java/org/apache/spark/api/java/function/MapPartitionsFunction.java
diff --git a/common/utils/src/main/java/org/apache/spark/api/java/function/PairFlatMapFunction.java b/common/utils-java/src/main/java/org/apache/spark/api/java/function/PairFlatMapFunction.java
similarity index 100%
rename from common/utils/src/main/java/org/apache/spark/api/java/function/PairFlatMapFunction.java
rename to common/utils-java/src/main/java/org/apache/spark/api/java/function/PairFlatMapFunction.java
diff --git a/common/utils/src/main/java/org/apache/spark/api/java/function/PairFunction.java b/common/utils-java/src/main/java/org/apache/spark/api/java/function/PairFunction.java
similarity index 100%
rename from common/utils/src/main/java/org/apache/spark/api/java/function/PairFunction.java
rename to common/utils-java/src/main/java/org/apache/spark/api/java/function/PairFunction.java
diff --git a/common/utils/src/main/java/org/apache/spark/api/java/function/ReduceFunction.java b/common/utils-java/src/main/java/org/apache/spark/api/java/function/ReduceFunction.java
similarity index 100%
rename from common/utils/src/main/java/org/apache/spark/api/java/function/ReduceFunction.java
rename to common/utils-java/src/main/java/org/apache/spark/api/java/function/ReduceFunction.java
diff --git a/common/utils/src/main/java/org/apache/spark/api/java/function/VoidFunction.java b/common/utils-java/src/main/java/org/apache/spark/api/java/function/VoidFunction.java
similarity index 100%
rename from common/utils/src/main/java/org/apache/spark/api/java/function/VoidFunction.java
rename to common/utils-java/src/main/java/org/apache/spark/api/java/function/VoidFunction.java
diff --git a/common/utils/src/main/java/org/apache/spark/api/java/function/VoidFunction2.java b/common/utils-java/src/main/java/org/apache/spark/api/java/function/VoidFunction2.java
similarity index 100%
rename from common/utils/src/main/java/org/apache/spark/api/java/function/VoidFunction2.java
rename to common/utils-java/src/main/java/org/apache/spark/api/java/function/VoidFunction2.java
diff --git a/common/utils/src/main/java/org/apache/spark/api/java/function/package-info.java b/common/utils-java/src/main/java/org/apache/spark/api/java/function/package-info.java
similarity index 100%
rename from common/utils/src/main/java/org/apache/spark/api/java/function/package-info.java
rename to common/utils-java/src/main/java/org/apache/spark/api/java/function/package-info.java
diff --git a/common/utils-java/src/main/java/org/apache/spark/internal/LogKey.java b/common/utils-java/src/main/java/org/apache/spark/internal/LogKey.java
new file mode 100644
index 0000000000000..0bd0fecb43976
--- /dev/null
+++ b/common/utils-java/src/main/java/org/apache/spark/internal/LogKey.java
@@ -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 org.apache.spark.internal;
+
+/**
+ * All structured logging `keys` used in `MDC` must be extends `LogKey`
+ *
+ *
+ * `LogKey`s serve as identifiers for mapped diagnostic contexts (MDC) within logs.
+ * Follow these guidelines when adding a new LogKey:
+ *
+ *
+ * Define all structured logging keys in `LogKeys.java`, and sort them alphabetically for
+ * ease of search.
+ *
+ *
+ * Use `UPPER_SNAKE_CASE` for key names.
+ *
+ *
+ * Key names should be both simple and broad, yet include specific identifiers like `STAGE_ID`,
+ * `TASK_ID`, and `JOB_ID` when needed for clarity. For instance, use `MAX_ATTEMPTS` as a
+ * general key instead of creating separate keys for each scenario such as
+ * `EXECUTOR_STATE_SYNC_MAX_ATTEMPTS` and `MAX_TASK_FAILURES`.
+ * This balances simplicity with the detail needed for effective logging.
+ *
+ *
+ * Use abbreviations in names if they are widely understood,
+ * such as `APP_ID` for APPLICATION_ID, and `K8S` for KUBERNETES.
+ *
+ *
+ * For time-related keys, use milliseconds as the unit of time.
+ *
*
* Constant String Messages:
@@ -65,8 +66,10 @@
* you can define `custom LogKey` and use it in `java` code as follows:
*
*
- * // To add a `custom LogKey`, implement `LogKey`
- * public static class CUSTOM_LOG_KEY implements LogKey { }
+ * // Add a `CustomLogKeys`, implement `LogKey`
+ * public enum CustomLogKeys implements LogKey {
+ * CUSTOM_LOG_KEY
+ * }
* import org.apache.spark.internal.MDC;
* logger.error("Unable to delete key {} for cache", MDC.of(CUSTOM_LOG_KEY, "key"));
*/
@@ -222,8 +225,8 @@ private void withLogContext(
for (int index = 0; index < mdcs.length; index++) {
MDC mdc = mdcs[index];
String value = (mdc.value() != null) ? mdc.value().toString() : null;
- if (Logging$.MODULE$.isStructuredLoggingEnabled()) {
- context.put(mdc.key().name(), value);
+ if (SparkLoggerFactory.isStructuredLoggingEnabled()) {
+ context.put(mdc.key().name().toLowerCase(Locale.ROOT), value);
}
args[index] = value;
}
diff --git a/common/utils/src/main/java/org/apache/spark/internal/SparkLoggerFactory.java b/common/utils-java/src/main/java/org/apache/spark/internal/SparkLoggerFactory.java
similarity index 77%
rename from common/utils/src/main/java/org/apache/spark/internal/SparkLoggerFactory.java
rename to common/utils-java/src/main/java/org/apache/spark/internal/SparkLoggerFactory.java
index a59c007362419..f5be570fa5b39 100644
--- a/common/utils/src/main/java/org/apache/spark/internal/SparkLoggerFactory.java
+++ b/common/utils-java/src/main/java/org/apache/spark/internal/SparkLoggerFactory.java
@@ -23,6 +23,20 @@
public class SparkLoggerFactory {
+ private static volatile boolean structuredLoggingEnabled = false;
+
+ public static void enableStructuredLogging() {
+ structuredLoggingEnabled = true;
+ }
+
+ public static void disableStructuredLogging() {
+ structuredLoggingEnabled = false;
+ }
+
+ public static boolean isStructuredLoggingEnabled() {
+ return structuredLoggingEnabled;
+ }
+
public static SparkLogger getLogger(String name) {
return new SparkLogger(LoggerFactory.getLogger(name));
}
diff --git a/common/utils/src/main/java/org/apache/spark/memory/MemoryMode.java b/common/utils-java/src/main/java/org/apache/spark/memory/MemoryMode.java
similarity index 100%
rename from common/utils/src/main/java/org/apache/spark/memory/MemoryMode.java
rename to common/utils-java/src/main/java/org/apache/spark/memory/MemoryMode.java
diff --git a/common/utils/src/main/java/org/apache/spark/network/util/ByteUnit.java b/common/utils-java/src/main/java/org/apache/spark/network/util/ByteUnit.java
similarity index 100%
rename from common/utils/src/main/java/org/apache/spark/network/util/ByteUnit.java
rename to common/utils-java/src/main/java/org/apache/spark/network/util/ByteUnit.java
diff --git a/common/utils/src/main/java/org/apache/spark/network/util/JavaUtils.java b/common/utils-java/src/main/java/org/apache/spark/network/util/JavaUtils.java
similarity index 55%
rename from common/utils/src/main/java/org/apache/spark/network/util/JavaUtils.java
rename to common/utils-java/src/main/java/org/apache/spark/network/util/JavaUtils.java
index 94f9f02ed2c9b..cf500926fa3aa 100644
--- a/common/utils/src/main/java/org/apache/spark/network/util/JavaUtils.java
+++ b/common/utils-java/src/main/java/org/apache/spark/network/util/JavaUtils.java
@@ -18,18 +18,26 @@
package org.apache.spark.network.util;
import java.io.*;
+import java.net.URL;
import java.nio.ByteBuffer;
import java.nio.channels.ReadableByteChannel;
import java.nio.charset.StandardCharsets;
+import java.nio.file.DirectoryStream;
import java.nio.file.Files;
import java.nio.file.LinkOption;
+import java.nio.file.Path;
+import java.nio.file.FileVisitOption;
+import java.nio.file.FileVisitResult;
+import java.nio.file.SimpleFileVisitor;
+import java.nio.file.StandardCopyOption;
import java.nio.file.attribute.BasicFileAttributes;
import java.util.*;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
-
-import org.apache.commons.lang3.SystemUtils;
+import java.util.stream.Stream;
+import java.util.stream.Collectors;
import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;
@@ -60,6 +68,109 @@ public static void closeQuietly(Closeable closeable) {
}
}
+ /** Delete a file or directory and its contents recursively without throwing exceptions. */
+ public static void deleteQuietly(File file) {
+ if (file != null && file.exists()) {
+ Path path = file.toPath();
+ try (Stream walk = Files.walk(path)) {
+ walk.sorted(Comparator.reverseOrder()).map(Path::toFile).forEach(File::delete);
+ } catch (Exception ignored) { /* No-op */ }
+ }
+ }
+
+ /** Registers the file or directory for deletion when the JVM exists. */
+ public static void forceDeleteOnExit(File file) throws IOException {
+ if (file != null && file.exists()) {
+ if (!file.isDirectory()) {
+ file.deleteOnExit();
+ } else {
+ Path path = file.toPath();
+ Files.walkFileTree(path, new SimpleFileVisitor() {
+ @Override
+ public FileVisitResult preVisitDirectory(Path p, BasicFileAttributes a)
+ throws IOException {
+ p.toFile().deleteOnExit();
+ return a.isSymbolicLink() ? FileVisitResult.SKIP_SUBTREE : FileVisitResult.CONTINUE;
+ }
+
+ @Override
+ public FileVisitResult visitFile(Path p, BasicFileAttributes a) throws IOException {
+ p.toFile().deleteOnExit();
+ return FileVisitResult.CONTINUE;
+ }
+ });
+ }
+ }
+ }
+
+ /** Move a file from src to dst. */
+ public static void moveFile(File src, File dst) throws IOException {
+ if (src == null || dst == null || !src.exists() || src.isDirectory() || dst.exists()) {
+ throw new IllegalArgumentException("Invalid input " + src + " or " + dst);
+ }
+ if (!src.renameTo(dst)) { // Try to use File.renameTo first
+ Files.move(src.toPath(), dst.toPath());
+ }
+ }
+
+ /** Move a directory from src to dst. */
+ public static void moveDirectory(File src, File dst) throws IOException {
+ if (src == null || dst == null || !src.exists() || !src.isDirectory() || dst.exists()) {
+ throw new IllegalArgumentException("Invalid input " + src + " or " + dst);
+ }
+ if (!src.renameTo(dst)) {
+ Path from = src.toPath().toAbsolutePath().normalize();
+ Path to = dst.toPath().toAbsolutePath().normalize();
+ if (to.startsWith(from)) {
+ throw new IllegalArgumentException("Cannot move directory to itself or its subdirectory");
+ }
+ moveDirectory(from, to);
+ }
+ }
+
+ private static void moveDirectory(Path src, Path dst) throws IOException {
+ Files.createDirectories(dst);
+ try (DirectoryStream stream = Files.newDirectoryStream(src)) {
+ for (Path from : stream) {
+ Path to = dst.resolve(from.getFileName());
+ if (Files.isDirectory(from)) {
+ moveDirectory(from, to);
+ } else {
+ Files.move(from, to, StandardCopyOption.REPLACE_EXISTING);
+ }
+ }
+ }
+ Files.delete(src);
+ }
+
+ /** Copy src to the target directory simply. File attribute times are not copied. */
+ public static void copyDirectory(File src, File dst) throws IOException {
+ if (src == null || dst == null || !src.exists() || !src.isDirectory() ||
+ (dst.exists() && !dst.isDirectory())) {
+ throw new IllegalArgumentException("Invalid input file " + src + " or directory " + dst);
+ }
+ Path from = src.toPath().toAbsolutePath().normalize();
+ Path to = dst.toPath().toAbsolutePath().normalize();
+ if (to.startsWith(from)) {
+ throw new IllegalArgumentException("Cannot copy directory to itself or its subdirectory");
+ }
+ Files.createDirectories(to);
+ Files.walkFileTree(from, new SimpleFileVisitor() {
+ @Override
+ public FileVisitResult preVisitDirectory(Path dir, BasicFileAttributes attrs)
+ throws IOException {
+ Files.createDirectories(to.resolve(from.relativize(dir)));
+ return FileVisitResult.CONTINUE;
+ }
+
+ @Override
+ public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException {
+ Files.copy(file, to.resolve(from.relativize(file)), StandardCopyOption.REPLACE_EXISTING);
+ return FileVisitResult.CONTINUE;
+ }
+ });
+ }
+
/** Returns a hash consistent with Spark's Utils.nonNegativeHash(). */
public static int nonNegativeHash(Object obj) {
if (obj == null) { return 0; }
@@ -83,6 +194,49 @@ public static String bytesToString(ByteBuffer b) {
return StandardCharsets.UTF_8.decode(b.slice()).toString();
}
+ public static long sizeOf(File file) throws IOException {
+ if (!file.exists()) {
+ throw new IllegalArgumentException(file.getAbsolutePath() + " not found");
+ }
+ return sizeOf(file.toPath());
+ }
+
+ public static long sizeOf(Path dirPath) throws IOException {
+ AtomicLong size = new AtomicLong(0);
+ Files.walkFileTree(dirPath, new SimpleFileVisitor() {
+ @Override
+ public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException {
+ size.addAndGet(attrs.size());
+ return FileVisitResult.CONTINUE;
+ }
+ });
+ return size.get();
+ }
+
+ public static void cleanDirectory(File dir) throws IOException {
+ if (dir == null || !dir.exists() || !dir.isDirectory()) {
+ throw new IllegalArgumentException("Invalid input directory " + dir);
+ }
+ cleanDirectory(dir.toPath());
+ }
+
+ private static void cleanDirectory(Path rootDir) throws IOException {
+ Files.walkFileTree(rootDir, new SimpleFileVisitor() {
+ @Override
+ public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException {
+ Files.delete(file);
+ return FileVisitResult.CONTINUE;
+ }
+
+ @Override
+ public FileVisitResult postVisitDirectory(Path dir, IOException e) throws IOException {
+ if (e != null) throw e;
+ if (!dir.equals(rootDir)) Files.delete(dir);
+ return FileVisitResult.CONTINUE;
+ }
+ });
+ }
+
/**
* Delete a file or directory and its contents recursively.
* Don't follow directories if they are symlinks.
@@ -110,14 +264,13 @@ public static void deleteRecursively(File file, FilenameFilter filter)
// On Unix systems, use operating system command to run faster
// If that does not work out, fallback to the Java IO way
// We exclude Apple Silicon test environment due to the limited resource issues.
- if (SystemUtils.IS_OS_UNIX && filter == null && !(SystemUtils.IS_OS_MAC_OSX &&
- (System.getenv("SPARK_TESTING") != null || System.getProperty("spark.testing") != null))) {
+ if (isUnix && filter == null && !(isMac && isTesting())) {
try {
deleteRecursivelyUsingUnixNative(file);
return;
} catch (IOException e) {
logger.warn("Attempt to delete using native Unix OS command failed for path = {}. " +
- "Falling back to Java IO way", e, MDC.of(LogKeys.PATH$.MODULE$, file.getAbsolutePath()));
+ "Falling back to Java IO way", e, MDC.of(LogKeys.PATH, file.getAbsolutePath()));
}
}
@@ -212,6 +365,25 @@ private static File[] listFilesSafely(File file, FilenameFilter filter) throws I
}
}
+ public static Set listPaths(File dir) throws IOException {
+ if (dir == null) throw new IllegalArgumentException("Input directory is null");
+ if (!dir.exists() || !dir.isDirectory()) return Collections.emptySet();
+ try (var stream = Files.walk(dir.toPath(), FileVisitOption.FOLLOW_LINKS)) {
+ return stream.filter(Files::isRegularFile).collect(Collectors.toCollection(HashSet::new));
+ }
+ }
+
+ public static Set listFiles(File dir) throws IOException {
+ if (dir == null) throw new IllegalArgumentException("Input directory is null");
+ if (!dir.exists() || !dir.isDirectory()) return Collections.emptySet();
+ try (var stream = Files.walk(dir.toPath(), FileVisitOption.FOLLOW_LINKS)) {
+ return stream
+ .filter(Files::isRegularFile)
+ .map(Path::toFile)
+ .collect(Collectors.toCollection(HashSet::new));
+ }
+ }
+
private static final Map timeSuffixes;
private static final Map byteSuffixes;
@@ -415,7 +587,7 @@ public static File createDirectory(String root, String namePrefix) throws IOExce
dir = new File(root, namePrefix + "-" + UUID.randomUUID());
Files.createDirectories(dir.toPath());
} catch (IOException | SecurityException e) {
- logger.error("Failed to create directory {}", e, MDC.of(LogKeys.PATH$.MODULE$, dir));
+ logger.error("Failed to create directory {}", e, MDC.of(LogKeys.PATH, dir));
dir = null;
}
}
@@ -435,4 +607,154 @@ public static void readFully(ReadableByteChannel channel, ByteBuffer dst) throws
}
}
+ /**
+ * Read len bytes exactly, otherwise throw exceptions.
+ */
+ public static void readFully(InputStream in, byte[] arr, int off, int len) throws IOException {
+ if (in == null || len < 0 || (off < 0 || off > arr.length - len)) {
+ throw new IllegalArgumentException("Invalid input argument");
+ }
+ if (len != in.readNBytes(arr, off, len)) {
+ throw new EOFException("Fail to read " + len + " bytes.");
+ }
+ }
+
+ /**
+ * Copy the content of a URL into a file.
+ */
+ public static void copyURLToFile(URL url, File file) throws IOException {
+ if (url == null || file == null || (file.exists() && file.isDirectory())) {
+ throw new IllegalArgumentException("Invalid input " + url + " or " + file);
+ }
+ Files.createDirectories(file.getParentFile().toPath());
+ try (InputStream in = url.openStream()) {
+ Files.copy(in, file.toPath(), StandardCopyOption.REPLACE_EXISTING);
+ }
+ }
+
+ public static String join(List arr, String sep) {
+ if (arr == null) return "";
+ StringJoiner joiner = new StringJoiner(sep == null ? "" : sep);
+ for (Object a : arr) {
+ joiner.add(a == null ? "" : a.toString());
+ }
+ return joiner.toString();
+ }
+
+ public static String stackTraceToString(Throwable t) {
+ if (t == null) {
+ return "";
+ }
+
+ ByteArrayOutputStream out = new ByteArrayOutputStream();
+ try (PrintWriter writer = new PrintWriter(out)) {
+ t.printStackTrace(writer);
+ writer.flush();
+ }
+ return out.toString(StandardCharsets.UTF_8);
+ }
+
+ public static int checkedCast(long value) {
+ if (value > Integer.MAX_VALUE || value < Integer.MIN_VALUE) {
+ throw new IllegalArgumentException("Cannot cast to integer.");
+ }
+ return (int) value;
+ }
+
+ /** Return true if the content of the files are equal or they both don't exist */
+ public static boolean contentEquals(File file1, File file2) throws IOException {
+ if (file1 == null && file2 != null || file1 != null && file2 == null) {
+ return false;
+ } else if (file1 == null && file2 == null || !file1.exists() && !file2.exists()) {
+ return true;
+ } else if (!file1.exists() || !file2.exists()) {
+ return false;
+ } else if (file1.isDirectory() || file2.isDirectory()) {
+ throw new IllegalArgumentException("Input is not a file: %s or %s".formatted(file1, file2));
+ } else if (file1.length() != file2.length()) {
+ return false;
+ } else {
+ Path path1 = file1.toPath();
+ Path path2 = file2.toPath();
+ return Files.isSameFile(path1, path2) || Files.mismatch(path1, path2) == -1L;
+ }
+ }
+
+ public static String toString(InputStream in) throws IOException {
+ return new String(in.readAllBytes(), StandardCharsets.UTF_8);
+ }
+
+ /**
+ * Indicates whether Spark is currently running unit tests.
+ */
+ public static boolean isTesting() {
+ return System.getenv("SPARK_TESTING") != null || System.getProperty("spark.testing") != null;
+ }
+
+ /**
+ * The `os.name` system property.
+ */
+ public static String osName = System.getProperty("os.name");
+
+ /**
+ * The `os.version` system property.
+ */
+ public static String osVersion = System.getProperty("os.version");
+
+ /**
+ * The `java.version` system property.
+ */
+ public static String javaVersion = Runtime.version().toString();
+
+ /**
+ * The `os.arch` system property.
+ */
+ public static String osArch = System.getProperty("os.arch");
+
+ /**
+ * Whether the underlying operating system is Windows.
+ */
+ public static boolean isWindows = osName.regionMatches(true, 0, "Windows", 0, 7);
+
+ /**
+ * Whether the underlying operating system is Mac OS X.
+ */
+ public static boolean isMac = osName.regionMatches(true, 0, "Mac OS X", 0, 8);
+
+ /**
+ * Whether the underlying operating system is Mac OS X and processor is Apple Silicon.
+ */
+ public static boolean isMacOnAppleSilicon = isMac && osArch.equals("aarch64");
+
+ /**
+ * Whether the underlying operating system is Linux.
+ */
+ public static boolean isLinux = osName.regionMatches(true, 0, "Linux", 0, 5);
+
+ /**
+ * Whether the underlying operating system is UNIX.
+ */
+ public static boolean isUnix = Stream.of("AIX", "HP-UX", "Irix", "Linux", "Mac OS X", "Solaris",
+ "SunOS", "FreeBSD", "OpenBSD", "NetBSD")
+ .anyMatch(prefix -> osName.regionMatches(true, 0, prefix, 0, prefix.length()));
+
+ /**
+ * Throws IllegalArgumentException with the given message if the check is false.
+ * Keep this clone of CommandBuilderUtils.checkArgument synced with the original.
+ */
+ public static void checkArgument(boolean check, String msg, Object... args) {
+ if (!check) {
+ throw new IllegalArgumentException(String.format(msg, args));
+ }
+ }
+
+ /**
+ * Throws IllegalStateException with the given message if the check is false.
+ * Keep this clone of CommandBuilderUtils.checkState synced with the original.
+ */
+ public static void checkState(boolean check, String msg, Object... args) {
+ if (!check) {
+ throw new IllegalStateException(String.format(msg, args));
+ }
+ }
}
diff --git a/common/utils/src/main/java/org/apache/spark/unsafe/array/ByteArrayUtils.java b/common/utils-java/src/main/java/org/apache/spark/unsafe/array/ByteArrayUtils.java
similarity index 100%
rename from common/utils/src/main/java/org/apache/spark/unsafe/array/ByteArrayUtils.java
rename to common/utils-java/src/main/java/org/apache/spark/unsafe/array/ByteArrayUtils.java
diff --git a/common/utils-java/src/main/java/org/apache/spark/util/Pair.java b/common/utils-java/src/main/java/org/apache/spark/util/Pair.java
new file mode 100644
index 0000000000000..bdcc01b49dcf4
--- /dev/null
+++ b/common/utils-java/src/main/java/org/apache/spark/util/Pair.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.util;
+
+/**
+ * An immutable pair of values. Note that the fields are intentionally designed to be `getLeft` and
+ * `getRight` instead of `left` and `right` in order to mitigate the migration burden
+ * from `org.apache.commons.lang3.tuple.Pair`.
+ */
+public record Pair(L getLeft, R getRight) {
+ public static Pair of(L left, R right) {
+ return new Pair<>(left, right);
+ }
+}
diff --git a/common/utils/src/main/resources/org/apache/spark/SparkLayout.json b/common/utils-java/src/main/resources/org/apache/spark/SparkLayout.json
similarity index 100%
rename from common/utils/src/main/resources/org/apache/spark/SparkLayout.json
rename to common/utils-java/src/main/resources/org/apache/spark/SparkLayout.json
diff --git a/common/utils/src/main/resources/org/apache/spark/log4j2-defaults.properties b/common/utils-java/src/main/resources/org/apache/spark/log4j2-defaults.properties
similarity index 100%
rename from common/utils/src/main/resources/org/apache/spark/log4j2-defaults.properties
rename to common/utils-java/src/main/resources/org/apache/spark/log4j2-defaults.properties
diff --git a/common/utils/src/main/resources/org/apache/spark/log4j2-json-layout.properties b/common/utils-java/src/main/resources/org/apache/spark/log4j2-json-layout.properties
similarity index 100%
rename from common/utils/src/main/resources/org/apache/spark/log4j2-json-layout.properties
rename to common/utils-java/src/main/resources/org/apache/spark/log4j2-json-layout.properties
diff --git a/connect-examples/server-library-example/common/src/main/protobuf/base.proto b/common/utils-java/src/test/java/org/apache/spark/util/CustomLogKeys.java
similarity index 75%
rename from connect-examples/server-library-example/common/src/main/protobuf/base.proto
rename to common/utils-java/src/test/java/org/apache/spark/util/CustomLogKeys.java
index 9d902a587ed37..cadacba7c5175 100644
--- a/connect-examples/server-library-example/common/src/main/protobuf/base.proto
+++ b/common/utils-java/src/test/java/org/apache/spark/util/CustomLogKeys.java
@@ -15,14 +15,10 @@
* limitations under the License.
*/
-syntax = 'proto3';
+package org.apache.spark.util;
-option java_multiple_files = true;
-option java_package = "org.apache.connect.examples.serverlibrary.proto";
+import org.apache.spark.internal.LogKey;
-message CustomTable {
- // Path to the custom table.
- string path = 1;
- // Name of the custom table.
- string name = 2;
+public enum CustomLogKeys implements LogKey {
+ CUSTOM_LOG_KEY
}
diff --git a/common/utils/src/test/java/org/apache/spark/util/PatternSparkLoggerSuite.java b/common/utils-java/src/test/java/org/apache/spark/util/PatternSparkLoggerSuite.java
similarity index 90%
rename from common/utils/src/test/java/org/apache/spark/util/PatternSparkLoggerSuite.java
rename to common/utils-java/src/test/java/org/apache/spark/util/PatternSparkLoggerSuite.java
index 6bfe595def1d4..7f8f3f93a8d46 100644
--- a/common/utils/src/test/java/org/apache/spark/util/PatternSparkLoggerSuite.java
+++ b/common/utils-java/src/test/java/org/apache/spark/util/PatternSparkLoggerSuite.java
@@ -90,12 +90,7 @@ String expectedPatternForMsgWithMDCValueIsNull(Level level) {
}
@Override
- String expectedPatternForScalaCustomLogKey(Level level) {
- return toRegexPattern(level, ".*: Scala custom log message.\n");
- }
-
- @Override
- String expectedPatternForJavaCustomLogKey(Level level) {
- return toRegexPattern(level, ".*: Java custom log message.\n");
+ String expectedPatternForCustomLogKey(Level level) {
+ return toRegexPattern(level, ".*: Custom log message.\n");
}
}
diff --git a/common/utils/src/test/java/org/apache/spark/util/SparkLoggerSuiteBase.java b/common/utils-java/src/test/java/org/apache/spark/util/SparkLoggerSuiteBase.java
similarity index 83%
rename from common/utils/src/test/java/org/apache/spark/util/SparkLoggerSuiteBase.java
rename to common/utils-java/src/test/java/org/apache/spark/util/SparkLoggerSuiteBase.java
index 186088ede1d0b..d86fe12c89243 100644
--- a/common/utils/src/test/java/org/apache/spark/util/SparkLoggerSuiteBase.java
+++ b/common/utils-java/src/test/java/org/apache/spark/util/SparkLoggerSuiteBase.java
@@ -22,11 +22,9 @@
import java.nio.file.Files;
import java.util.List;
-import org.apache.commons.lang3.tuple.Pair;
import org.apache.logging.log4j.Level;
import org.junit.jupiter.api.Test;
-import org.apache.spark.internal.LogKey;
import org.apache.spark.internal.LogKeys;
import org.apache.spark.internal.MDC;
import org.apache.spark.internal.SparkLogger;
@@ -73,23 +71,20 @@ private void checkLogOutput(Level level, Runnable func, ExpectedResult result) {
private final String basicMsgWithEscapeChar =
"This is a log message\nThis is a new line \t other msg";
- private final MDC executorIDMDC = MDC.of(LogKeys.EXECUTOR_ID$.MODULE$, "1");
+ private final MDC executorIDMDC = MDC.of(LogKeys.EXECUTOR_ID, "1");
private final String msgWithMDC = "Lost executor {}.";
private final MDC[] mdcs = new MDC[] {
- MDC.of(LogKeys.EXECUTOR_ID$.MODULE$, "1"),
- MDC.of(LogKeys.REASON$.MODULE$, "the shuffle data is too large")};
+ MDC.of(LogKeys.EXECUTOR_ID, "1"),
+ MDC.of(LogKeys.REASON, "the shuffle data is too large")};
private final String msgWithMDCs = "Lost executor {}, reason: {}";
private final MDC[] emptyMDCs = new MDC[0];
- private final MDC executorIDMDCValueIsNull = MDC.of(LogKeys.EXECUTOR_ID$.MODULE$, null);
+ private final MDC executorIDMDCValueIsNull = MDC.of(LogKeys.EXECUTOR_ID, null);
- private final MDC scalaCustomLogMDC =
- MDC.of(CustomLogKeys.CUSTOM_LOG_KEY$.MODULE$, "Scala custom log message.");
-
- private final MDC javaCustomLogMDC =
- MDC.of(JavaCustomLogKeys.CUSTOM_LOG_KEY, "Java custom log message.");
+ private final MDC customLogMDC =
+ MDC.of(CustomLogKeys.CUSTOM_LOG_KEY, "Custom log message.");
// test for basic message (without any mdc)
abstract String expectedPatternForBasicMsg(Level level);
@@ -118,10 +113,7 @@ String expectedPatternForMsgWithEmptyMDCsAndException(Level level) {
abstract String expectedPatternForMsgWithMDCValueIsNull(Level level);
// test for scala custom LogKey
- abstract String expectedPatternForScalaCustomLogKey(Level level);
-
- // test for java custom LogKey
- abstract String expectedPatternForJavaCustomLogKey(Level level);
+ abstract String expectedPatternForCustomLogKey(Level level);
@Test
public void testBasicMsg() {
@@ -241,34 +233,14 @@ public void testLoggerWithMDCValueIsNull() {
}
@Test
- public void testLoggerWithScalaCustomLogKey() {
- Runnable errorFn = () -> logger().error("{}", scalaCustomLogMDC);
- Runnable warnFn = () -> logger().warn("{}", scalaCustomLogMDC);
- Runnable infoFn = () -> logger().info("{}", scalaCustomLogMDC);
- List.of(
- Pair.of(Level.ERROR, errorFn),
- Pair.of(Level.WARN, warnFn),
- Pair.of(Level.INFO, infoFn)).forEach(pair ->
- checkLogOutput(pair.getLeft(), pair.getRight(), this::expectedPatternForScalaCustomLogKey));
- }
-
- @Test
- public void testLoggerWithJavaCustomLogKey() {
- Runnable errorFn = () -> logger().error("{}", javaCustomLogMDC);
- Runnable warnFn = () -> logger().warn("{}", javaCustomLogMDC);
- Runnable infoFn = () -> logger().info("{}", javaCustomLogMDC);
+ public void testLoggerWithCustomLogKey() {
+ Runnable errorFn = () -> logger().error("{}", customLogMDC);
+ Runnable warnFn = () -> logger().warn("{}", customLogMDC);
+ Runnable infoFn = () -> logger().info("{}", customLogMDC);
List.of(
Pair.of(Level.ERROR, errorFn),
Pair.of(Level.WARN, warnFn),
Pair.of(Level.INFO, infoFn)).forEach(pair ->
- checkLogOutput(pair.getLeft(), pair.getRight(), this::expectedPatternForJavaCustomLogKey));
+ checkLogOutput(pair.getLeft(), pair.getRight(), this::expectedPatternForCustomLogKey));
}
}
-
-class JavaCustomLogKeys {
- // Custom `LogKey` must be `implements LogKey`
- public static class CUSTOM_LOG_KEY implements LogKey { }
-
- // Singleton
- public static final CUSTOM_LOG_KEY CUSTOM_LOG_KEY = new CUSTOM_LOG_KEY();
-}
diff --git a/common/utils/src/test/java/org/apache/spark/util/StructuredSparkLoggerSuite.java b/common/utils-java/src/test/java/org/apache/spark/util/StructuredSparkLoggerSuite.java
similarity index 88%
rename from common/utils/src/test/java/org/apache/spark/util/StructuredSparkLoggerSuite.java
rename to common/utils-java/src/test/java/org/apache/spark/util/StructuredSparkLoggerSuite.java
index 1fab167adfeb0..88ac8ea34710a 100644
--- a/common/utils/src/test/java/org/apache/spark/util/StructuredSparkLoggerSuite.java
+++ b/common/utils-java/src/test/java/org/apache/spark/util/StructuredSparkLoggerSuite.java
@@ -24,7 +24,6 @@
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.BeforeAll;
-import org.apache.spark.internal.Logging$;
import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;
@@ -33,13 +32,13 @@ public class StructuredSparkLoggerSuite extends SparkLoggerSuiteBase {
// Enable Structured Logging before running the tests
@BeforeAll
public static void setup() {
- Logging$.MODULE$.enableStructuredLogging();
+ SparkLoggerFactory.enableStructuredLogging();
}
// Disable Structured Logging after running the tests
@AfterAll
public static void teardown() {
- Logging$.MODULE$.disableStructuredLogging();
+ SparkLoggerFactory.disableStructuredLogging();
}
private static final SparkLogger LOGGER =
@@ -176,28 +175,14 @@ String expectedPatternForMsgWithMDCValueIsNull(Level level) {
}
@Override
- String expectedPatternForScalaCustomLogKey(Level level) {
+ String expectedPatternForCustomLogKey(Level level) {
return compactAndToRegexPattern(level, """
{
"ts": "",
"level": "",
- "msg": "Scala custom log message.",
+ "msg": "Custom log message.",
"context": {
- "custom_log_key": "Scala custom log message."
- },
- "logger": ""
- }""");
- }
-
- @Override
- String expectedPatternForJavaCustomLogKey(Level level) {
- return compactAndToRegexPattern(level, """
- {
- "ts": "",
- "level": "",
- "msg": "Java custom log message.",
- "context": {
- "custom_log_key": "Java custom log message."
+ "custom_log_key": "Custom log message."
},
"logger": ""
}""");
diff --git a/common/utils-java/src/test/resources/log4j2.properties b/common/utils-java/src/test/resources/log4j2.properties
new file mode 100644
index 0000000000000..cb38f5b55a0ba
--- /dev/null
+++ b/common/utils-java/src/test/resources/log4j2.properties
@@ -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.
+#
+
+rootLogger.level = info
+rootLogger.appenderRef.file.ref = ${sys:test.appender:-File}
+
+appender.file.type = File
+appender.file.name = File
+appender.file.fileName = target/unit-tests.log
+appender.file.layout.type = JsonTemplateLayout
+appender.file.layout.eventTemplateUri = classpath:org/apache/spark/SparkLayout.json
+
+# Structured Logging Appender
+appender.structured.type = File
+appender.structured.name = structured
+appender.structured.fileName = target/structured.log
+appender.structured.layout.type = JsonTemplateLayout
+appender.structured.layout.eventTemplateUri = classpath:org/apache/spark/SparkLayout.json
+
+# Pattern Logging Appender
+appender.pattern.type = File
+appender.pattern.name = pattern
+appender.pattern.fileName = target/pattern.log
+appender.pattern.layout.type = PatternLayout
+appender.pattern.layout.pattern = %d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n%ex
+
+# Custom loggers
+logger.structured_logging.name = org.apache.spark.util.StructuredLoggingSuite
+logger.structured_logging.level = trace
+logger.structured_logging.appenderRefs = structured
+logger.structured_logging.appenderRef.structured.ref = structured
+
+logger.pattern_logging.name = org.apache.spark.util.PatternLoggingSuite
+logger.pattern_logging.level = trace
+logger.pattern_logging.appenderRefs = pattern
+logger.pattern_logging.appenderRef.pattern.ref = pattern
+
+logger.structured_logger.name = org.apache.spark.util.StructuredSparkLoggerSuite
+logger.structured_logger.level = trace
+logger.structured_logger.appenderRefs = structured
+logger.structured_logger.appenderRef.structured.ref = structured
+
+logger.pattern_logger.name = org.apache.spark.util.PatternSparkLoggerSuite
+logger.pattern_logger.level = trace
+logger.pattern_logger.appenderRefs = pattern
+logger.pattern_logger.appenderRef.pattern.ref = pattern
diff --git a/common/utils/pom.xml b/common/utils/pom.xml
index 44771938439ae..df3bc5adb10bd 100644
--- a/common/utils/pom.xml
+++ b/common/utils/pom.xml
@@ -39,6 +39,18 @@
org.apache.sparkspark-tags_${scala.binary.version}
+
+ org.apache.spark
+ spark-common-utils-java_${scala.binary.version}
+ ${project.version}
+
+
+ org.apache.spark
+ spark-common-utils-java_${scala.binary.version}
+ ${project.version}
+ test-jar
+ test
+ org.apache.xbeanxbean-asm9-shaded
@@ -51,14 +63,6 @@
com.fasterxml.jackson.modulejackson-module-scala_${scala.binary.version}
-
- org.apache.commons
- commons-text
-
-
- commons-io
- commons-io
- org.apache.ivyivy
diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json
index fe14f4e827938..167c460536ac9 100644
--- a/common/utils/src/main/resources/error/error-conditions.json
+++ b/common/utils/src/main/resources/error/error-conditions.json
@@ -90,6 +90,30 @@
],
"sqlState" : "42000"
},
+ "APPROX_TOP_K_MAX_ITEMS_TRACKED_EXCEEDS_LIMIT" : {
+ "message" : [
+ "The max items tracked `maxItemsTracked`() of `approx_top_k` should be less than or equal to ."
+ ],
+ "sqlState" : "22023"
+ },
+ "APPROX_TOP_K_MAX_ITEMS_TRACKED_LESS_THAN_K" : {
+ "message" : [
+ "The max items tracked `maxItemsTracked`() of `approx_top_k` should be greater than or equal to `k`()."
+ ],
+ "sqlState" : "22023"
+ },
+ "APPROX_TOP_K_NON_POSITIVE_ARG" : {
+ "message" : [
+ "The value of in `approx_top_k` must be a positive integer, but got ."
+ ],
+ "sqlState" : "22023"
+ },
+ "APPROX_TOP_K_NULL_ARG" : {
+ "message" : [
+ "The value of in `approx_top_k` cannot be NULL."
+ ],
+ "sqlState" : "22004"
+ },
"ARITHMETIC_OVERFLOW" : {
"message" : [
". If necessary set to \"false\" to bypass this error."
@@ -249,6 +273,24 @@
],
"sqlState" : "0A000"
},
+ "CANNOT_LOAD_CHECKPOINT_FILE_MANAGER" : {
+ "message" : [
+ "Error loading streaming checkpoint file manager for path=."
+ ],
+ "subClass" : {
+ "ERROR_LOADING_CLASS" : {
+ "message" : [
+ "Error instantiating streaming checkpoint file manager for path= with className=. msg=."
+ ]
+ },
+ "UNCATEGORIZED" : {
+ "message" : [
+ ""
+ ]
+ }
+ },
+ "sqlState" : "58030"
+ },
"CANNOT_LOAD_FUNCTION_CLASS" : {
"message" : [
"Cannot load class when registering the function , please make sure it is on the classpath."
@@ -853,6 +895,11 @@
"Please fit or load a model smaller than bytes."
]
},
+ "MODEL_SUMMARY_LOST" : {
+ "message" : [
+ "The model summary is lost because the cached model is offloaded."
+ ]
+ },
"UNSUPPORTED_EXCEPTION" : {
"message" : [
""
@@ -921,12 +968,24 @@
},
"sqlState" : "21S01"
},
+ "CYCLIC_FUNCTION_REFERENCE" : {
+ "message" : [
+ "Cyclic function reference detected: ."
+ ],
+ "sqlState" : "42887"
+ },
"DATAFLOW_GRAPH_NOT_FOUND" : {
"message" : [
"Dataflow graph with id could not be found"
],
"sqlState" : "KD011"
},
+ "DATATYPE_CANNOT_ORDER" : {
+ "message" : [
+ "Type does not support ordered operations."
+ ],
+ "sqlState" : "0A000"
+ },
"DATATYPE_MISMATCH" : {
"message" : [
"Cannot resolve due to data type mismatch:"
@@ -1435,6 +1494,12 @@
],
"sqlState" : "42711"
},
+ "DUPLICATE_VARIABLE_NAME_INSIDE_DECLARE" : {
+ "message" : [
+ "Found duplicate variable in the declare variable list. Please, remove one of them."
+ ],
+ "sqlState" : "42734"
+ },
"EMITTING_ROWS_OLDER_THAN_WATERMARK_NOT_ALLOWED" : {
"message" : [
"Previous node emitted a row with eventTime= which is older than current_watermark_value=",
@@ -1781,12 +1846,6 @@
],
"sqlState" : "39000"
},
- "FOUND_MULTIPLE_DATA_SOURCES" : {
- "message" : [
- "Detected multiple data sources with the name ''. Please check the data source isn't simultaneously registered and located in the classpath."
- ],
- "sqlState" : "42710"
- },
"GENERATED_COLUMN_WITH_DEFAULT_VALUE" : {
"message" : [
"A column cannot have both a default value and a generation expression but column has default value: () and generation expression: ()."
@@ -2708,6 +2767,34 @@
],
"sqlState" : "42000"
},
+ "INVALID_FLOW_QUERY_TYPE" : {
+ "message" : [
+ "Flow returns an invalid relation type."
+ ],
+ "subClass" : {
+ "BATCH_RELATION_FOR_STREAMING_TABLE" : {
+ "message" : [
+ "Streaming tables may only be defined by streaming relations, but the flow attempts to write a batch relation to the streaming table . Consider using the STREAM operator in Spark-SQL to convert the batch relation into a streaming relation, or populating the streaming table with an append once-flow instead."
+ ]
+ },
+ "STREAMING_RELATION_FOR_MATERIALIZED_VIEW" : {
+ "message" : [
+ "Materialized views may only be defined by a batch relation, but the flow attempts to write a streaming relation to the materialized view ."
+ ]
+ },
+ "STREAMING_RELATION_FOR_ONCE_FLOW" : {
+ "message" : [
+ " is an append once-flow that is defined by a streaming relation. Append once-flows may only be defined by or return a batch relation."
+ ]
+ },
+ "STREAMING_RELATION_FOR_PERSISTED_VIEW" : {
+ "message" : [
+ "Persisted views may only be defined by a batch relation, but the flow attempts to write a streaming relation to the persisted view ."
+ ]
+ }
+ },
+ "sqlState" : "42000"
+ },
"INVALID_FORMAT" : {
"message" : [
"The format is invalid: ."
@@ -3349,6 +3436,11 @@
"expects a string literal, but got ."
]
},
+ "TIME_UNIT" : {
+ "message" : [
+ "expects one of the units 'HOUR', 'MINUTE', 'SECOND', 'MILLISECOND', 'MICROSECOND', but got ''."
+ ]
+ },
"ZERO_INDEX" : {
"message" : [
"expects %1$, %2$ and so on, but got %0$."
@@ -3407,7 +3499,7 @@
},
"INVALID_RECURSIVE_CTE" : {
"message" : [
- "Invalid recursive definition found. Recursive queries must contain an UNION or an UNION ALL statement with 2 children. The first child needs to be the anchor term without any recursive references."
+ "Invalid recursive definition found. Recursive queries must contain an UNION or an UNION ALL statement with 2 children. The first child needs to be the anchor term without any recursive references. Any top level inner CTE must not contain self references."
],
"sqlState" : "42836"
},
@@ -3416,14 +3508,9 @@
"Invalid recursive reference found inside WITH RECURSIVE clause."
],
"subClass" : {
- "NUMBER" : {
- "message" : [
- "Multiple self-references to one recursive CTE are not allowed."
- ]
- },
"PLACE" : {
"message" : [
- "Recursive references cannot be used on the right side of left outer/semi/anti joins, on the left side of right outer joins, in full outer joins, in aggregates, and in subquery expressions."
+ "Recursive references cannot be used on the right side of left outer/semi/anti joins, on the left side of right outer joins, in full outer joins, in aggregates, window functions or sorts"
]
}
},
@@ -3551,6 +3638,12 @@
],
"sqlState" : "42K08"
},
+ "INVALID_SQL_FUNCTION_DATA_ACCESS" : {
+ "message" : [
+ "Cannot create a SQL function with CONTAINS SQL that accesses a table/view or a SQL function that reads SQL data. Please use READS SQL DATA instead."
+ ],
+ "sqlState" : "42K0E"
+ },
"INVALID_SQL_FUNCTION_PLAN_STRUCTURE" : {
"message" : [
"Invalid SQL function plan structure",
@@ -3975,18 +4068,25 @@
],
"sqlState" : "42K0L"
},
- "LABEL_ALREADY_EXISTS" : {
+ "LABEL_OR_FOR_VARIABLE_ALREADY_EXISTS" : {
"message" : [
- "The label org.objenesisobjenesis
- 3.3
+ 3.4com.github.jnr
@@ -666,7 +665,7 @@
org.apache.commonscommons-text
- 1.13.1
+ 1.14.0commons-lang
@@ -874,7 +873,7 @@
com.github.lubenzstd-jni
- 1.5.7-3
+ 1.5.7-4com.clearspring.analytics
@@ -1152,6 +1151,10 @@
org.apache.commonscommons-math3
+
+ org.scala-lang.modules
+ scala-collection-compat_${scala.binary.version}
+
@@ -1240,13 +1243,13 @@
net.bytebuddybyte-buddy
- 1.17.0
+ 1.17.6testnet.bytebuddybyte-buddy-agent
- 1.17.0
+ 1.17.6test
@@ -1623,19 +1626,6 @@
-
-
- org.codehaus.jackson
- jackson-core-asl
- ${codehaus.jackson.version}
- ${hive.jackson.scope}
-
-
- org.codehaus.jackson
- jackson-mapper-asl
- ${codehaus.jackson.version}
- ${hive.jackson.scope}
- ${hive.group}hive-beeline
@@ -2320,7 +2310,7 @@
org.apache.orcorc-format
- 1.1.0
+ 1.1.1${orc.classifier}${orc.deps.scope}
@@ -2463,11 +2453,6 @@
joda-time${joda.version}
-
- org.jodd
- jodd-core
- ${jodd.version}
- org.datanucleusdatanucleus-core
@@ -2631,7 +2616,7 @@
org.apache.maven.pluginsmaven-enforcer-plugin
- 3.5.0
+ 3.6.0enforce-versions
@@ -2685,14 +2670,14 @@
org.codehaus.mojoextra-enforcer-rules
- 1.9.0
+ 1.10.0org.codehaus.mojobuild-helper-maven-plugin
- 3.6.0
+ 3.6.1module-timestamp-property
@@ -2808,6 +2793,7 @@
--release${java.version}-Xlint:all,-serial,-path,-try
+ -proc:full
@@ -2829,7 +2815,7 @@
org.apache.maven.pluginsmaven-surefire-plugin
- 3.5.2
+ 3.5.3
@@ -2967,7 +2953,7 @@
org.apache.maven.pluginsmaven-clean-plugin
- 3.4.1
+ 3.5.0
diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala
index 5f1c0cbca0d0b..57ab8e196da91 100644
--- a/project/MimaExcludes.scala
+++ b/project/MimaExcludes.scala
@@ -39,7 +39,10 @@ object MimaExcludes {
ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.linalg.Vector.getSizeInBytes"),
// [SPARK-52221][SQL] Refactor SqlScriptingLocalVariableManager into more generic context manager
- ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.scripting.SqlScriptingExecution.withLocalVariableManager")
+ ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.scripting.SqlScriptingExecution.withLocalVariableManager"),
+
+ // [SPARK-53391][CORE] Remove unused PrimitiveKeyOpenHashMap
+ ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.collection.PrimitiveKeyOpenHashMap*")
)
// Default exclude rules
@@ -83,6 +86,13 @@ object MimaExcludes {
// SPARK-51267: Match local Spark Connect server logic between Python and Scala
ProblemFilters.exclude[MissingFieldProblem]("org.apache.spark.launcher.SparkLauncher.SPARK_LOCAL_REMOTE"),
+ // SPARK-53138: Split common/utils Java code into a new module common/utils-java
+ ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.QueryContext"),
+ ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.QueryContextType"),
+ ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.api.java.function.*"),
+ ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.network.util.ByteUnit"),
+ ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.network.util.JavaUtils"),
+
(problem: Problem) => problem match {
case MissingClassProblem(cls) => !cls.fullName.startsWith("org.sparkproject.jpmml") &&
!cls.fullName.startsWith("org.sparkproject.dmg.pmml")
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 77001e6bdf227..b1bbdad17759b 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -16,6 +16,7 @@
*/
import java.io._
+import java.lang.{Runtime => JRuntime}
import java.nio.charset.StandardCharsets.UTF_8
import java.nio.file.{Files, Paths, StandardCopyOption}
import java.util.Locale
@@ -58,10 +59,10 @@ object BuildCommons {
val allProjects@Seq(
core, graphx, mllib, mllibLocal, repl, networkCommon, networkShuffle, launcher, unsafe, tags, sketch, kvstore,
- commonUtils, variant, pipelines, _*
+ commonUtils, commonUtilsJava, variant, pipelines, _*
) = Seq(
"core", "graphx", "mllib", "mllib-local", "repl", "network-common", "network-shuffle", "launcher", "unsafe",
- "tags", "sketch", "kvstore", "common-utils", "variant", "pipelines"
+ "tags", "sketch", "kvstore", "common-utils", "common-utils-java", "variant", "pipelines"
).map(ProjectRef(buildLocation, _)) ++ sqlProjects ++ streamingProjects ++ connectProjects
val optionallyEnabledProjects@Seq(kubernetes, yarn,
@@ -99,6 +100,30 @@ object SparkBuild extends PomBuild {
import sbtunidoc.GenJavadocPlugin.autoImport._
import scala.collection.mutable.Map
+ lazy val checkJavaVersion = taskKey[Unit]("Check Java Version")
+ lazy val checkJavaVersionSettings: Seq[Setting[?]] = Seq(
+ checkJavaVersion := {
+ val currentVersion = JRuntime.version()
+ val currentVersionFeature = currentVersion.feature()
+ val currentVersionUpdate = currentVersion.update()
+ val minimumVersion = JRuntime.Version.parse(
+ SbtPomKeys.effectivePom.value.getProperties
+ .get("java.minimum.version").asInstanceOf[String])
+ val minimumVersionFeature = minimumVersion.feature()
+ val minimumVersionUpdate = minimumVersion.update()
+ val isCompatible = currentVersionFeature > minimumVersionFeature ||
+ (currentVersionFeature == minimumVersionFeature &&
+ currentVersionUpdate >= minimumVersionUpdate)
+ if (!isCompatible) {
+ throw new MessageOnlyException(
+ "The Java version used to build the project is outdated. " +
+ s"Please use Java $minimumVersion or later.")
+ }
+ },
+ (Compile / compile) := ((Compile / compile) dependsOn checkJavaVersion).value,
+ (Test / compile) := ((Test / compile) dependsOn checkJavaVersion).value
+ )
+
val projectsMap: Map[String, Seq[Setting[_]]] = Map.empty
override val profiles = {
@@ -261,7 +286,8 @@ object SparkBuild extends PomBuild {
val noLintOnCompile = sys.env.contains("NOLINT_ON_COMPILE") &&
!sys.env.get("NOLINT_ON_COMPILE").contains("false")
- lazy val sharedSettings = sparkGenjavadocSettings ++
+ lazy val sharedSettings = checkJavaVersionSettings ++
+ sparkGenjavadocSettings ++
compilerWarningSettings ++
(if (noLintOnCompile) Nil else enableScalaStyle) ++ Seq(
(Compile / exportJars) := true,
@@ -313,6 +339,7 @@ object SparkBuild extends PomBuild {
(Compile / javacOptions) ++= Seq(
"-encoding", UTF_8.name(),
"-g",
+ "-proc:full",
"--release", javaVersion.value
),
// This -target and Xlint:unchecked options cannot be set in the Compile configuration scope since
@@ -373,7 +400,7 @@ object SparkBuild extends PomBuild {
Seq(
spark, hive, hiveThriftServer, repl, networkCommon, networkShuffle, networkYarn,
unsafe, tags, tokenProviderKafka010, sqlKafka010, pipelines, connectCommon, connect,
- connectClient, variant, connectShims, profiler
+ connectClient, variant, connectShims, profiler, commonUtilsJava
).contains(x)
}
@@ -1078,6 +1105,7 @@ object ExcludedDependencies {
libraryDependencies ~= { libs => libs.filterNot(_.name == "groovy-all") },
excludeDependencies ++= Seq(
ExclusionRule(organization = "ch.qos.logback"),
+ ExclusionRule("org.slf4j", "slf4j-simple"),
ExclusionRule("javax.servlet", "javax.servlet-api"))
)
}
@@ -1516,6 +1544,9 @@ object CopyDependencies {
val fid = (LocalProject("connect") / assembly).value
val fidClient = (LocalProject("connect-client-jvm") / assembly).value
val fidProtobuf = (LocalProject("protobuf") / assembly).value
+ val noProvidedSparkJars: Boolean = sys.env.getOrElse("NO_PROVIDED_SPARK_JARS", "1") == "1" ||
+ sys.env.getOrElse("NO_PROVIDED_SPARK_JARS", "true")
+ .toLowerCase(Locale.getDefault()) == "true"
(Compile / dependencyClasspath).value.map(_.data)
.filter { jar => jar.isFile() }
@@ -1526,17 +1557,18 @@ object CopyDependencies {
destJar.delete()
}
- if (jar.getName.contains("spark-connect-common") &&
- !SbtPomKeys.profiles.value.contains("noshade-connect")) {
+ if (jar.getName.contains("spark-connect-common")) {
// Don't copy the spark connect common JAR as it is shaded in the spark connect.
} else if (jar.getName.contains("connect-client-jvm")) {
// Do not place Spark Connect client jars as it is not built-in.
- } else if (jar.getName.contains("spark-connect") &&
- !SbtPomKeys.profiles.value.contains("noshade-connect")) {
+ } else if (noProvidedSparkJars && jar.getName.contains("spark-avro")) {
+ // Do not place Spark Avro jars as it is not built-in.
+ } else if (jar.getName.contains("spark-connect")) {
Files.copy(fid.toPath, destJar.toPath)
- } else if (jar.getName.contains("spark-protobuf") &&
- !SbtPomKeys.profiles.value.contains("noshade-protobuf")) {
- Files.copy(fidProtobuf.toPath, destJar.toPath)
+ } else if (jar.getName.contains("spark-protobuf")) {
+ if (!noProvidedSparkJars) {
+ Files.copy(fidProtobuf.toPath, destJar.toPath)
+ }
} else {
Files.copy(jar.toPath(), destJar.toPath())
}
@@ -1651,6 +1683,7 @@ object TestSettings {
(Test / javaOptions) ++= System.getProperties.asScala.filter(_._1.startsWith("spark"))
.map { case (k,v) => s"-D$k=$v" }.toSeq,
(Test / javaOptions) += "-ea",
+ (Test / javaOptions) += s"-XX:ErrorFile=${baseDirectory.value}/target/hs_err_pid%p.log",
(Test / javaOptions) ++= {
val metaspaceSize = sys.env.get("METASPACE_SIZE").getOrElse("1300m")
val heapSize = sys.env.get("HEAP_SIZE").getOrElse("4g")
@@ -1722,7 +1755,7 @@ object TestSettings {
(Test / testOptions) += Tests.Argument(TestFrameworks.ScalaTest, "-W", "120", "300"),
(Test / testOptions) += Tests.Argument(TestFrameworks.JUnit, "-v", "-a"),
// Enable Junit testing.
- libraryDependencies += "com.github.sbt.junit" % "jupiter-interface" % "0.14.0" % "test",
+ libraryDependencies += "com.github.sbt.junit" % "jupiter-interface" % "0.15.0" % "test",
// `parallelExecutionInTest` controls whether test suites belonging to the same SBT project
// can run in parallel with one another. It does NOT control whether tests execute in parallel
// within the same JVM (which is controlled by `testForkedParallel`) or whether test cases
diff --git a/project/plugins.sbt b/project/plugins.sbt
index 579020e5af0ce..0e5ce79c8b567 100644
--- a/project/plugins.sbt
+++ b/project/plugins.sbt
@@ -41,6 +41,6 @@ addSbtPlugin("com.simplytyped" % "sbt-antlr4" % "0.8.3")
addSbtPlugin("com.github.sbt" % "sbt-pom-reader" % "2.4.0")
-addSbtPlugin("com.github.sbt.junit" % "sbt-jupiter-interface" % "0.14.0")
+addSbtPlugin("com.github.sbt.junit" % "sbt-jupiter-interface" % "0.15.0")
addSbtPlugin("com.thesamet" % "sbt-protoc" % "1.0.7")
diff --git a/python/docs/Makefile b/python/docs/Makefile
index eba8a3fffbcd1..045b03a1afd1b 100644
--- a/python/docs/Makefile
+++ b/python/docs/Makefile
@@ -22,7 +22,6 @@ SOURCEDIR ?= source
BUILDDIR ?= build
export PYTHONPATH=$(realpath ..):$(realpath ../lib/py4j-0.10.9.9-src.zip)
-export SPARK_ANSI_SQL_MODE=false
# Put it first so that "make" without argument is like "make help".
help:
diff --git a/python/docs/source/development/contributing.rst b/python/docs/source/development/contributing.rst
index fc2a1cc586040..4103a1e650930 100644
--- a/python/docs/source/development/contributing.rst
+++ b/python/docs/source/development/contributing.rst
@@ -129,8 +129,8 @@ If you are using Conda, the development environment can be set as follows.
.. code-block:: bash
- # Python 3.9+ is required
- conda create --name pyspark-dev-env python=3.9
+ # Python 3.10+ is required
+ conda create --name pyspark-dev-env python=3.10
conda activate pyspark-dev-env
pip install --upgrade -r dev/requirements.txt
@@ -145,7 +145,7 @@ Now, you can start developing and `running the tests `_.
pip
~~~
-With Python 3.9+, pip can be used as below to install and set up the development environment.
+With Python 3.10+, pip can be used as below to install and set up the development environment.
.. code-block:: bash
diff --git a/python/docs/source/development/debugging.rst b/python/docs/source/development/debugging.rst
index 0aa2426cf862d..9fa9a0c526cea 100644
--- a/python/docs/source/development/debugging.rst
+++ b/python/docs/source/development/debugging.rst
@@ -215,7 +215,7 @@ Python/Pandas UDF
~~~~~~~~~~~~~~~~~
PySpark provides remote `memory_profiler `_ for
-Python/Pandas UDFs. That can be used on editors with line numbers such as Jupyter notebooks. UDFs with iterators as inputs/outputs are not supported.
+Python/Pandas UDFs. That can be used on editors with line numbers such as Jupyter notebooks. UDFs that are generator functions are not supported.
SparkSession-based memory profiler can be enabled by setting the `Runtime SQL configuration `_
``spark.sql.pyspark.udf.profiler`` to ``memory``. An example on a Jupyter notebook is as shown below.
@@ -320,7 +320,7 @@ Python/Pandas UDF
~~~~~~~~~~~~~~~~~
PySpark provides remote `Python Profilers `_ for
-Python/Pandas UDFs. UDFs with iterators as inputs/outputs are not supported.
+Python/Pandas UDFs. UDFs that are generator functions are not supported.
SparkSession-based performance profiler can be enabled by setting the `Runtime SQL configuration `_
``spark.sql.pyspark.udf.profiler`` to ``perf``. An example is as shown below.
diff --git a/python/docs/source/getting_started/install.rst b/python/docs/source/getting_started/install.rst
index 5f445c8aa5242..82db489651ff9 100644
--- a/python/docs/source/getting_started/install.rst
+++ b/python/docs/source/getting_started/install.rst
@@ -30,7 +30,7 @@ and building from the source.
Python Versions Supported
-------------------------
-Python 3.9 and above.
+Python 3.10 and above.
Using PyPI
@@ -143,7 +143,7 @@ the same session as pyspark (you can install in several steps too).
.. code-block:: bash
- conda install -c conda-forge pyspark # can also add "python=3.9 some_package [etc.]" here
+ conda install -c conda-forge pyspark # can also add "python=3.10 some_package [etc.]" here
Note that `PySpark for conda `_ is maintained
separately by the community; while new versions generally get packaged quickly, the
@@ -225,8 +225,8 @@ Installable with ``pip install "pyspark[connect]"``.
========================== ================= ==========================
Package Supported version Note
========================== ================= ==========================
-`pandas` >=2.0.0 Required for Spark Connect
-`pyarrow` >=11.0.0 Required for Spark Connect
+`pandas` >=2.2.0 Required for Spark Connect
+`pyarrow` >=15.0.0 Required for Spark Connect
`grpcio` >=1.67.0 Required for Spark Connect
`grpcio-status` >=1.67.0 Required for Spark Connect
`googleapis-common-protos` >=1.65.0 Required for Spark Connect
@@ -241,8 +241,8 @@ Installable with ``pip install "pyspark[sql]"``.
========= ================= ======================
Package Supported version Note
========= ================= ======================
-`pandas` >=2.0.0 Required for Spark SQL
-`pyarrow` >=11.0.0 Required for Spark SQL
+`pandas` >=2.2.0 Required for Spark SQL
+`pyarrow` >=15.0.0 Required for Spark SQL
========= ================= ======================
Additional libraries that enhance functionality but are not included in the installation packages:
@@ -259,7 +259,7 @@ Installable with ``pip install "pyspark[pandas_on_spark]"``.
Package Supported version Note
========= ================= ================================
`pandas` >=2.2.0 Required for Pandas API on Spark
-`pyarrow` >=11.0.0 Required for Pandas API on Spark
+`pyarrow` >=15.0.0 Required for Pandas API on Spark
========= ================= ================================
Additional libraries that enhance functionality but are not included in the installation packages:
@@ -277,7 +277,7 @@ Installable with ``pip install "pyspark[ml]"``.
======= ================= ======================================
Package Supported version Note
======= ================= ======================================
-`numpy` >=1.21 Required for MLlib DataFrame-based API
+`numpy` >=1.22 Required for MLlib DataFrame-based API
======= ================= ======================================
Additional libraries that enhance functionality but are not included in the installation packages:
@@ -297,5 +297,22 @@ Installable with ``pip install "pyspark[mllib]"``.
======= ================= ==================
Package Supported version Note
======= ================= ==================
-`numpy` >=1.21 Required for MLlib
+`numpy` >=1.22 Required for MLlib
======= ================= ==================
+
+Declarative Pipelines
+^^^^^^^^^^^^^^^^^^^^^
+
+Installable with ``pip install "pyspark[pipelines]"``. Includes all dependencies for both Spark SQL and Spark Connect, because Declarative Pipelines is built on top of both.
+
+========================== ================= ===================================================
+Package Supported version Note
+========================== ================= ===================================================
+`pandas` >=2.2.0 Required for Spark Connect and Spark SQL
+`pyarrow` >=15.0.0 Required for Spark Connect and Spark SQL
+`grpcio` >=1.67.0 Required for Spark Connect
+`grpcio-status` >=1.67.0 Required for Spark Connect
+`googleapis-common-protos` >=1.65.0 Required for Spark Connect
+`pyyaml` >=3.11 Required for spark-pipelines command line interface
+`graphviz` >=0.20 Optional for Spark Connect
+========================== ================= ===================================================
diff --git a/python/docs/source/migration_guide/pyspark_upgrade.rst b/python/docs/source/migration_guide/pyspark_upgrade.rst
index f91e297539649..f7d80fe10786d 100644
--- a/python/docs/source/migration_guide/pyspark_upgrade.rst
+++ b/python/docs/source/migration_guide/pyspark_upgrade.rst
@@ -22,10 +22,16 @@ Upgrading PySpark
Upgrading from PySpark 4.0 to 4.1
---------------------------------
+* In Spark 4.1, Python 3.9 support was dropped in PySpark.
+* In Spark 4.1, the minimum supported version for PyArrow has been raised from 11.0.0 to 15.0.0 in PySpark.
+* In Spark 4.1, the minimum supported version for Pandas has been raised from 2.0.0 to 2.2.0 in PySpark.
+* In Spark 4.1, ``DataFrame['name']`` on Spark Connect Python Client no longer eagerly validate the column name. To restore the legacy behavior, set ``PYSPARK_VALIDATE_COLUMN_NAME_LEGACY`` environment variable to ``1``.
* In Spark 4.1, Arrow-optimized Python UDF supports UDT input / output instead of falling back to the regular UDF. To restore the legacy behavior, set ``spark.sql.execution.pythonUDF.arrow.legacy.fallbackOnUDT`` to ``true``.
-
+* In Spark 4.1, unnecessary conversion to pandas instances is removed when ``spark.sql.execution.pythonUDF.arrow.enabled`` is enabled. As a result, the type coercion changes when the produced output has a schema different from the specified schema. To restore the previous behavior, enable ``spark.sql.legacy.execution.pythonUDF.pandas.conversion.enabled``.
* In Spark 4.1, unnecessary conversion to pandas instances is removed when ``spark.sql.execution.pythonUDTF.arrow.enabled`` is enabled. As a result, the type coercion changes when the produced output has a schema different from the specified schema. To restore the previous behavior, enable ``spark.sql.legacy.execution.pythonUDTF.pandas.conversion.enabled``.
+* In Spark 4.1, the ``spark.sql.execution.pandas.convertToArrowArraySafely`` configuration is enabled by default. When this setting is enabled, PyArrow raises errors for unsafe conversions such as integer overflows, floating point truncation, and loss of precision. This change affects the return data serialization of arrow-enabled UDFs/pandas_udfs, and the creation of PySpark DataFrames. To restore the previous behavior, set the configuration to ``false``.
+* In Spark 4.1, pandas API on Spark works under ANSI mode when ``compute.ansi_mode_support`` is set to ``True`` (default). The safeguard option ``compute.fail_on_ansi_mode`` remains available, but it only takes effect when ``compute.ansi_mode_support`` is ``False``.
Upgrading from PySpark 3.5 to 4.0
---------------------------------
diff --git a/python/docs/source/reference/pyspark.errors.rst b/python/docs/source/reference/pyspark.errors.rst
index 4d2bf76127795..1204cccc8df5b 100644
--- a/python/docs/source/reference/pyspark.errors.rst
+++ b/python/docs/source/reference/pyspark.errors.rst
@@ -50,7 +50,6 @@ Classes
QueryContext
QueryContextType
QueryExecutionException
- RetriesExceeded
SessionNotSameException
SparkRuntimeException
SparkUpgradeException
diff --git a/python/docs/source/reference/pyspark.sql/data_types.rst b/python/docs/source/reference/pyspark.sql/data_types.rst
index d201be271a6a6..99f8c5bb87ef9 100644
--- a/python/docs/source/reference/pyspark.sql/data_types.rst
+++ b/python/docs/source/reference/pyspark.sql/data_types.rst
@@ -31,6 +31,7 @@ Data Types
ByteType
DataType
DateType
+ TimeType
DecimalType
DoubleType
FloatType
diff --git a/python/docs/source/reference/pyspark.sql/functions.rst b/python/docs/source/reference/pyspark.sql/functions.rst
index a3a2e11daf2e1..7bec529407667 100644
--- a/python/docs/source/reference/pyspark.sql/functions.rst
+++ b/python/docs/source/reference/pyspark.sql/functions.rst
@@ -253,6 +253,7 @@ Date and Timestamp Functions
convert_timezone
curdate
current_date
+ current_time
current_timestamp
current_timezone
date_add
@@ -279,6 +280,7 @@ Date and Timestamp Functions
make_date
make_dt_interval
make_interval
+ make_time
make_timestamp
make_timestamp_ltz
make_timestamp_ntz
@@ -297,7 +299,9 @@ Date and Timestamp Functions
timestamp_micros
timestamp_millis
timestamp_seconds
+ time_trunc
to_date
+ to_time
to_timestamp
to_timestamp_ltz
to_timestamp_ntz
@@ -308,7 +312,9 @@ Date and Timestamp Functions
try_make_timestamp
try_make_timestamp_ltz
try_make_timestamp_ntz
+ try_to_time
try_to_timestamp
+ try_to_date
unix_date
unix_micros
unix_millis
@@ -643,6 +649,8 @@ UDF, UDTF and UDT
.. autosummary::
:toctree: api/
+ arrow_udf
+ arrow_udtf
call_udf
pandas_udf
udf
diff --git a/python/docs/source/tutorial/pandas_on_spark/options.rst b/python/docs/source/tutorial/pandas_on_spark/options.rst
index 74197a5d99750..25e9ceb5171d6 100644
--- a/python/docs/source/tutorial/pandas_on_spark/options.rst
+++ b/python/docs/source/tutorial/pandas_on_spark/options.rst
@@ -324,10 +324,10 @@ compute.fail_on_ansi_mode True 'compute.fail_on_ansi_mo
an exception if the underlying Spark is working with
ANSI mode enabled and the option
'compute.ansi_mode_support' is False.
-compute.ansi_mode_support False 'compute.ansi_mode_support' sets whether or not to
+compute.ansi_mode_support True 'compute.ansi_mode_support' sets whether or not to
support the ANSI mode of the underlying Spark. If
False, pandas API on Spark may hit unexpected results
- or errors. The default is False.
+ or errors. The default is True.
plotting.max_rows 1000 'plotting.max_rows' sets the visual limit on top-n-
based plots such as `plot.bar` and `plot.pie`. If it
is set to 1000, the first 1000 data points will be
diff --git a/python/docs/source/tutorial/pandas_on_spark/typehints.rst b/python/docs/source/tutorial/pandas_on_spark/typehints.rst
index 23126664d78a6..7045ce7f75987 100644
--- a/python/docs/source/tutorial/pandas_on_spark/typehints.rst
+++ b/python/docs/source/tutorial/pandas_on_spark/typehints.rst
@@ -62,7 +62,7 @@ it as a Spark schema. As an example, you can specify the return type hint as bel
Notice that the function ``pandas_div`` actually takes and outputs a pandas DataFrame instead of
pandas-on-Spark :class:`DataFrame`. So, technically the correct types should be of pandas.
-With Python 3.9+, you can specify the type hints by using pandas instances as follows:
+With Python 3.10+, you can specify the type hints by using pandas instances as follows:
.. code-block:: python
diff --git a/python/docs/source/tutorial/sql/arrow_pandas.rst b/python/docs/source/tutorial/sql/arrow_pandas.rst
index ffff59f136cbe..3bef50874d7ff 100644
--- a/python/docs/source/tutorial/sql/arrow_pandas.rst
+++ b/python/docs/source/tutorial/sql/arrow_pandas.rst
@@ -375,6 +375,12 @@ fallback for type mismatches, leading to potential ambiguity and data loss. Addi
and tuples to strings can yield ambiguous results. Arrow Python UDFs, on the other hand, leverage Arrow's
capabilities to standardize type coercion and address these issues effectively.
+A note on Arrow Python UDF type coercion: In Spark 4.1, unnecessary conversion to pandas instances is removed in the serializer
+when ``spark.sql.execution.pythonUDF.arrow.enabled`` is enabled. As a result, the type coercion changes
+when the produced output has a schema different from the specified schema. To restore the previous behavior,
+enable ``spark.sql.legacy.execution.pythonUDF.pandas.conversion.enabled``.
+The behavior difference is summarized in the tables `here `__.
+
Usage Notes
-----------
@@ -434,7 +440,7 @@ working with timestamps in ``pandas_udf``\s to get the best performance, see
Recommended Pandas and PyArrow Versions
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
-For usage with pyspark.sql, the minimum supported versions of Pandas is 2.0.0 and PyArrow is 11.0.0.
+For usage with pyspark.sql, the minimum supported versions of Pandas is 2.2.0 and PyArrow is 11.0.0.
Higher versions may be used, however, compatibility and data correctness can not be guaranteed and should
be verified by the user.
diff --git a/python/docs/source/tutorial/sql/arrow_python_udtf.rst b/python/docs/source/tutorial/sql/arrow_python_udtf.rst
new file mode 100644
index 0000000000000..3e933fea722f0
--- /dev/null
+++ b/python/docs/source/tutorial/sql/arrow_python_udtf.rst
@@ -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.
+
+======================================================
+Vectorized Python User-defined Table Functions (UDTFs)
+======================================================
+
+Spark 4.1 introduces the Vectorized Python user-defined table function (UDTF), a new type of user-defined table-valued function.
+It can be used via the ``@arrow_udtf`` decorator.
+Unlike scalar functions that return a single result value from each call, each UDTF is invoked in
+the ``FROM`` clause of a query and returns an entire table as output.
+Unlike the traditional Python UDTF that evaluates row by row, the Vectorized Python UDTF lets you directly operate on top of Apache Arrow arrays and column batches.
+This allows you to leverage vectorized operations and improve the performance of your UDTF.
+
+Vectorized Python UDTF Interface
+--------------------------------
+
+.. currentmodule:: pyspark.sql.functions
+
+.. code-block:: python
+
+ class NameYourArrowPythonUDTF:
+
+ def __init__(self) -> None:
+ """
+ Initializes the user-defined table function (UDTF). This is optional.
+
+ This method serves as the default constructor and is called once when the
+ UDTF is instantiated on the executor side.
+
+ Any class fields assigned in this method will be available for subsequent
+ calls to the `eval`, `terminate` and `cleanup` methods.
+
+ Notes
+ -----
+ - You cannot create or reference the Spark session within the UDTF. Any
+ attempt to do so will result in a serialization error.
+ """
+ ...
+
+ def eval(self, *args: Any) -> Iterator[pa.RecordBatch | pa.Table]:
+ """
+ Evaluates the function using the given input arguments.
+
+ This method is required and must be implemented.
+
+ Argument Mapping:
+ - Each provided scalar expression maps to exactly one value in the
+ `*args` list with type `pa.Array`.
+ - Each provided table argument maps to a `pa.RecordBatch` object containing
+ the columns in the order they appear in the provided input table,
+ and with the names computed by the query analyzer.
+
+ This method is called on every batch of input rows, and can produce zero or more
+ output pyarrow record batches or pyarrow tables. Each element in the output tuple
+ corresponds to one column specified in the return type of the UDTF.
+
+ Parameters
+ ----------
+ *args : Any
+ Arbitrary positional arguments representing the input to the UDTF.
+
+ Yields
+ ------
+ iterator
+ An iterator of `pa.RecordBatch` or `pa.Table` objects representing a batch of rows
+ in the UDTF result table. Yield as many times as needed to produce multiple batches.
+
+ Notes
+ -----
+ - UDTFs can instead accept keyword arguments during the function call if needed.
+ - The `eval` method can raise a `SkipRestOfInputTableException` to indicate that the
+ UDTF wants to skip consuming all remaining rows from the current partition of the
+ input table. This will cause the UDTF to proceed directly to the `terminate` method.
+ - The `eval` method can raise any other exception to indicate that the UDTF should be
+ aborted entirely. This will cause the UDTF to skip the `terminate` method and proceed
+ directly to the `cleanup` method, and then the exception will be propagated to the
+ query processor causing the invoking query to fail.
+
+ Examples
+ --------
+ This `eval` method takes a table argument and returns an arrow record batch for each input batch.
+
+ >>> def eval(self, batch: pa.RecordBatch):
+ ... yield batch
+
+ This `eval` method takes a table argument and returns a pyarrow table for each input batch.
+
+ >>> def eval(self, batch: pa.RecordBatch):
+ ... yield pa.table({"x": batch.column(0), "y": batch.column(1)})
+
+ This `eval` method takes both table and scalar arguments and returns a pyarrow table for each input batch.
+
+ >>> def eval(self, batch: pa.RecordBatch, x: pa.Array):
+ ... yield pa.table({"x": x, "y": batch.column(0)})
+ """
+ ...
+
+ def terminate(self) -> Iterator[pa.RecordBatch | pa.Table]:
+ """
+ Called when the UDTF has successfully processed all input rows.
+
+ This method is optional to implement and is useful for performing any
+ finalization operations after the UDTF has finished processing
+ all rows. It can also be used to yield additional rows if needed.
+ Table functions that consume all rows in the entire input partition
+ and then compute and return the entire output table can do so from
+ this method as well (please be mindful of memory usage when doing
+ this).
+
+ If any exceptions occur during input row processing, this method
+ won't be called.
+
+ Yields
+ ------
+ iterator
+ An iterator of `pa.RecordBatch` or `pa.Table` objects representing a batch of rows
+ in the UDTF result table. Yield as many times as needed to produce multiple batches.
+
+ Examples
+ --------
+ >>> def terminate(self) -> Iterator[pa.RecordBatch | pa.Table]:
+ >>> yield pa.table({"x": pa.array([1, 2, 3])})
+ """
+ ...
+
+ def cleanup(self) -> None:
+ """
+ Invoked after the UDTF completes processing input rows.
+
+ This method is optional to implement and is useful for final cleanup
+ regardless of whether the UDTF processed all input rows successfully
+ or was aborted due to exceptions.
+
+ Examples
+ --------
+ >>> def cleanup(self) -> None:
+ >>> self.conn.close()
+ """
+ ...
+
+Defining the Output Schema
+--------------------------
+
+The return type of the UDTF defines the schema of the table it outputs.
+You can specify it in the ``@arrow_udtf`` decorator.
+
+It must be either a ``StructType``:
+
+.. code-block:: python
+
+ @arrow_udtf(returnType=StructType().add("c1", StringType()).add("c2", IntegerType()))
+ class YourArrowPythonUDTF:
+ ...
+
+or a DDL string representing a struct type:
+
+.. code-block:: python
+
+ @arrow_udtf(returnType="c1 string, c2 int")
+ class YourArrowPythonUDTF:
+ ...
+
+Emitting Output Rows
+--------------------
+
+The `eval` and `terminate` methods then emit zero or more output batches conforming to this schema by
+yielding ``pa.RecordBatch`` or ``pa.Table`` objects.
+
+.. code-block:: python
+
+ @arrow_udtf(returnType="c1 int, c2 int")
+ class YourArrowPythonUDTF:
+ def eval(self, batch: pa.RecordBatch):
+ yield pa.table({"c1": batch.column(0), "c2": batch.column(1)})
+
+You can also yield multiple pyarrow tables in the `eval` method.
+
+.. code-block:: python
+
+ @arrow_udtf(returnType="c1 int")
+ class YourArrowPythonUDTF:
+ def eval(self, batch: pa.RecordBatch):
+ yield pa.table({"c1": batch.column(0)})
+ yield pa.table({"c1": batch.column(1)})
+
+You can also yield multiple pyarrow record batches in the `eval` method.
+
+.. code-block:: python
+
+ @arrow_udtf(returnType="c1 int")
+ class YourArrowPythonUDTF:
+ def eval(self, batch: pa.RecordBatch):
+ new_batch = pa.record_batch(
+ {"c1": batch.column(0).slice(0, len(batch) // 2)})
+ yield new_batch
+
+
+Usage Examples
+--------------
+
+Here's how to use these UDTFs in DataFrame:
+
+.. code-block:: python
+
+ import pyarrow as pa
+ from pyspark.sql.functions import arrow_udtf
+
+ @arrow_udtf(returnType="c1 string")
+ class MyArrowPythonUDTF:
+ def eval(self, batch: pa.RecordBatch):
+ yield pa.table({"c1": batch.column("value")})
+
+ df = spark.range(10).selectExpr("id", "cast(id as string) as value")
+ MyArrowPythonUDTF(df.asTable()).show()
+
+ # Register the UDTF
+ spark.udtf.register("my_arrow_udtf", MyArrowPythonUDTF)
+
+ # Use in SQL queries
+ df = spark.sql("""
+ SELECT * FROM my_arrow_udtf(TABLE(SELECT id, cast(id as string) as value FROM range(10)))
+ """)
diff --git a/python/docs/source/tutorial/sql/index.rst b/python/docs/source/tutorial/sql/index.rst
index d1b67f7eeb909..c2f2c57baadaf 100644
--- a/python/docs/source/tutorial/sql/index.rst
+++ b/python/docs/source/tutorial/sql/index.rst
@@ -24,6 +24,7 @@ Spark SQL
:maxdepth: 2
arrow_pandas
+ arrow_python_udtf
python_udtf
python_data_source
type_conversions
diff --git a/python/docs/source/tutorial/sql/python_data_source.rst b/python/docs/source/tutorial/sql/python_data_source.rst
index 22b2a0b5f3c7b..41b76c95d5806 100644
--- a/python/docs/source/tutorial/sql/python_data_source.rst
+++ b/python/docs/source/tutorial/sql/python_data_source.rst
@@ -520,4 +520,6 @@ The following example demonstrates how to implement a basic Data Source using Ar
Usage Notes
-----------
-- During Data Source resolution, built-in and Scala/Java Data Sources take precedence over Python Data Sources with the same name; to explicitly use a Python Data Source, make sure its name does not conflict with the other Data Sources.
+- During Data Source resolution, built-in and Scala/Java Data Sources take precedence over Python Data Sources with the same name; to explicitly use a Python Data Source, make sure its name does not conflict with the other non-Python Data Sources.
+- It is allowed to register multiple Python Data Sources with the same name. Later registrations will overwrite earlier ones.
+- To automatically register a data source, export it as ``DefaultSource`` in a top level module with name prefix ``pyspark_``. See `pyspark_huggingface `_ for an example.
diff --git a/python/docs/source/tutorial/sql/python_udtf.rst b/python/docs/source/tutorial/sql/python_udtf.rst
index ac394d9e5360f..a989848448c9d 100644
--- a/python/docs/source/tutorial/sql/python_udtf.rst
+++ b/python/docs/source/tutorial/sql/python_udtf.rst
@@ -351,7 +351,7 @@ or a DDL string representing a struct type:
.. code-block:: python
- c1: string
+ c1 string
Emitting Output Rows
--------------------
diff --git a/python/docs/source/user_guide/ansi_migration_guide.ipynb b/python/docs/source/user_guide/ansi_migration_guide.ipynb
new file mode 100644
index 0000000000000..5193b307aadf3
--- /dev/null
+++ b/python/docs/source/user_guide/ansi_migration_guide.ipynb
@@ -0,0 +1,196 @@
+{
+ "cells": [
+ {
+ "cell_type": "markdown",
+ "id": "4fa81d13",
+ "metadata": {},
+ "source": [
+ "# ANSI Migration Guide - Pandas API on Spark\n",
+ "ANSI mode is now on by default for Pandas API on Spark. This guide helps you understand the key behavior differences you’ll see.\n",
+ "In short, with ANSI mode on, Pandas API on Spark behavior matches native pandas in cases where Pandas API on Spark with ANSI off did not."
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "6e1c7952",
+ "metadata": {},
+ "source": [
+ "## Behavior Change\n",
+ "### String Number Comparison\n",
+ "**ANSI off:** Spark implicitly casts numbers and strings, so `1` and `'1'` are considered equal.\n",
+ "\n",
+ "**ANSI on:** behaves like pandas, `1 == '1'` is False."
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "69474e28-c1cd-40fe-8ec6-7373b56c4dee",
+ "metadata": {},
+ "source": [
+ "Examples are as shown below:\n",
+ "\n",
+ "```python\n",
+ ">>> pdf = pd.DataFrame({\"int\": [1, 2], \"str\": [\"1\", \"2\"]})\n",
+ ">>> psdf = ps.from_pandas(pdf)\n",
+ "\n",
+ "# ANSI on\n",
+ ">>> spark.conf.set(\"spark.sql.ansi.enabled\", True)\n",
+ ">>> psdf[\"int\"] == psdf[\"str\"]\n",
+ "0 False\n",
+ "1 False\n",
+ "dtype: bool\n",
+ "\n",
+ "# ANSI off\n",
+ ">>> spark.conf.set(\"spark.sql.ansi.enabled\", False)\n",
+ ">>> psdf[\"int\"] == psdf[\"str\"]\n",
+ "0 True\n",
+ "1 True\n",
+ "dtype: bool\n",
+ "\n",
+ "# Pandas\n",
+ ">>> pdf[\"int\"] == pdf[\"str\"]\n",
+ "0 False\n",
+ "1 False\n",
+ "dtype: bool\n",
+ "```"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "90a4ea8d",
+ "metadata": {},
+ "source": [
+ "### Strict Casting\n",
+ "**ANSI off:** invalid casts (e.g., `'a' → int`) quietly became NULL.\n",
+ "\n",
+ "**ANSI on:** the same casts raise errors."
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "b361febc-4435-4bd1-9ee1-4874413d770c",
+ "metadata": {},
+ "source": [
+ "Examples are as shown below:\n",
+ "\n",
+ "```python\n",
+ ">>> pdf = pd.DataFrame({\"str\": [\"a\"]})\n",
+ ">>> psdf = ps.from_pandas(pdf)\n",
+ "\n",
+ "# ANSI on\n",
+ ">>> spark.conf.set(\"spark.sql.ansi.enabled\", True)\n",
+ ">>> psdf[\"str\"].astype(int)\n",
+ "Traceback (most recent call last):\n",
+ "...\n",
+ "pyspark.errors.exceptions.captured.NumberFormatException: [CAST_INVALID_INPUT] ...\n",
+ "\n",
+ "# ANSI off\n",
+ ">>> spark.conf.set(\"spark.sql.ansi.enabled\", False)\n",
+ ">>> psdf[\"str\"].astype(int)\n",
+ "0 NaN\n",
+ "Name: str, dtype: float64\n",
+ "\n",
+ "# Pandas\n",
+ ">>> pdf[\"str\"].astype(int)\n",
+ "Traceback (most recent call last):\n",
+ "...\n",
+ "ValueError: invalid literal for int() with base 10: 'a'\n",
+ "```"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "e11583e2",
+ "metadata": {},
+ "source": [
+ "### MultiIndex.to_series Return\n",
+ "**ANSI off:** Each row is returned as an `ArrayType` value, e.g. `[1, red]`.\n",
+ "\n",
+ "**ANSI on:** Each row is returned as a `StructType` value, which appears as a tuple (e.g., `(1, red)`) if the Runtime SQL Configuration `spark.sql.execution.pandas.structHandlingMode` is set to `'row'`. Otherwise, the result may vary depending on whether Arrow is used. See more in the [Spark Runtime SQL Configuration docs](https://spark.apache.org/docs/latest/configuration.html#runtime-sql-configuration)."
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "4671a895-ed40-4bc4-b1bc-fa9fbb86cc18",
+ "metadata": {},
+ "source": [
+ "Examples are as shown below:\n",
+ "\n",
+ "```python\n",
+ ">>> arrays = [[1, 2], [\"red\", \"blue\"]]\n",
+ ">>> pidx = pd.MultiIndex.from_arrays(arrays, names=(\"number\", \"color\"))\n",
+ ">>> psidx = ps.from_pandas(pidx)\n",
+ "\n",
+ "# ANSI on\n",
+ ">>> spark.conf.set(\"spark.sql.ansi.enabled\", True)\n",
+ ">>> spark.conf.set(\"spark.sql.execution.pandas.structHandlingMode\", \"row\")\n",
+ ">>> psidx.to_series()\n",
+ "number color\n",
+ "1 red (1, red)\n",
+ "2 blue (2, blue)\n",
+ "dtype: object\n",
+ "\n",
+ "# ANSI off\n",
+ ">>> spark.conf.set(\"spark.sql.ansi.enabled\", False)\n",
+ ">>> psidx.to_series()\n",
+ "number color\n",
+ "1 red [1, red]\n",
+ "2 blue [2, blue]\n",
+ "dtype: object\n",
+ "\n",
+ "# Pandas\n",
+ ">>> pidx.to_series()\n",
+ "number color\n",
+ "1 red (1, red)\n",
+ "2 blue (2, blue)\n",
+ "dtype: object\n",
+ "```"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "fe146afd",
+ "metadata": {},
+ "source": [
+ "## Related Configurations\n",
+ "\n",
+ "### `spark.sql.ansi.enabled` (Spark config)\n",
+ "- Native Spark setting that controls ANSI mode. \n",
+ "- The most powerful config to control both SQL and pandas API behavior. \n",
+ "- If set to **False**, Spark reverts to the old behavior, and the other configs are not effective.\n",
+ "\n",
+ "### `compute.ansi_mode_support` (Pandas API on Spark option)\n",
+ "- Indicates whether ANSI mode is fully supported. \n",
+ "- Effective only when ANSI is enabled. \n",
+ "- If set to **False**, pandas API on Spark may hit unexpected results or errors. \n",
+ "- Default is **True**.\n",
+ "\n",
+ "### `compute.fail_on_ansi_mode` (Pandas API on Spark option)\n",
+ "- Controls whether pandas API on Spark fails immediately when ANSI mode is enabled. \n",
+ "- Effective only when ANSI is enabled and `compute.ansi_mode_support` is **False**. \n",
+ "- If set to **False**, forces pandas API on Spark to work with the old behavior even when ANSI is enabled."
+ ]
+ }
+ ],
+ "metadata": {
+ "kernelspec": {
+ "display_name": "Python 3 (ipykernel)",
+ "language": "python",
+ "name": "python3"
+ },
+ "language_info": {
+ "codemirror_mode": {
+ "name": "ipython",
+ "version": 3
+ },
+ "file_extension": ".py",
+ "mimetype": "text/x-python",
+ "name": "python",
+ "nbconvert_exporter": "python",
+ "pygments_lexer": "ipython3",
+ "version": "3.11.13"
+ }
+ },
+ "nbformat": 4,
+ "nbformat_minor": 5
+}
diff --git a/python/docs/source/user_guide/index.rst b/python/docs/source/user_guide/index.rst
index 1da4dd2e6eb13..e95a6ec0c64d8 100644
--- a/python/docs/source/user_guide/index.rst
+++ b/python/docs/source/user_guide/index.rst
@@ -33,5 +33,6 @@ Each of the below sections contains code-driven examples to help you get familia
udfandudtf
sql
loadandbehold
+ ansi_migration_guide
diff --git a/python/packaging/classic/setup.py b/python/packaging/classic/setup.py
index 51ab69c6e4cc6..fcb607a561b3d 100755
--- a/python/packaging/classic/setup.py
+++ b/python/packaging/classic/setup.py
@@ -150,9 +150,9 @@ def _supports_symlinks():
# binary format protocol with the Java version, see ARROW_HOME/format/* for specifications.
# Also don't forget to update python/docs/source/getting_started/install.rst,
# python/packaging/client/setup.py, and python/packaging/connect/setup.py
-_minimum_pandas_version = "2.0.0"
+_minimum_pandas_version = "2.2.0"
_minimum_numpy_version = "1.21"
-_minimum_pyarrow_version = "11.0.0"
+_minimum_pyarrow_version = "15.0.0"
_minimum_grpc_version = "1.67.0"
_minimum_googleapis_common_protos_version = "1.65.0"
_minimum_pyyaml_version = "3.11"
@@ -366,14 +366,21 @@ def run(self):
"grpcio-status>=%s" % _minimum_grpc_version,
"googleapis-common-protos>=%s" % _minimum_googleapis_common_protos_version,
"numpy>=%s" % _minimum_numpy_version,
+ ],
+ "pipelines": [
+ "pandas>=%s" % _minimum_pandas_version,
+ "pyarrow>=%s" % _minimum_pyarrow_version,
+ "numpy>=%s" % _minimum_numpy_version,
+ "grpcio>=%s" % _minimum_grpc_version,
+ "grpcio-status>=%s" % _minimum_grpc_version,
+ "googleapis-common-protos>=%s" % _minimum_googleapis_common_protos_version,
"pyyaml>=%s" % _minimum_pyyaml_version,
],
},
- python_requires=">=3.9",
+ python_requires=">=3.10",
classifiers=[
"Development Status :: 5 - Production/Stable",
"License :: OSI Approved :: Apache Software License",
- "Programming Language :: Python :: 3.9",
"Programming Language :: Python :: 3.10",
"Programming Language :: Python :: 3.11",
"Programming Language :: Python :: 3.12",
diff --git a/python/packaging/client/setup.py b/python/packaging/client/setup.py
index ac80e3a1ff585..d70c17465331d 100755
--- a/python/packaging/client/setup.py
+++ b/python/packaging/client/setup.py
@@ -133,9 +133,9 @@
# binary format protocol with the Java version, see ARROW_HOME/format/* for specifications.
# Also don't forget to update python/docs/source/getting_started/install.rst,
# python/packaging/classic/setup.py, and python/packaging/connect/setup.py
- _minimum_pandas_version = "2.0.0"
+ _minimum_pandas_version = "2.2.0"
_minimum_numpy_version = "1.21"
- _minimum_pyarrow_version = "11.0.0"
+ _minimum_pyarrow_version = "15.0.0"
_minimum_grpc_version = "1.67.0"
_minimum_googleapis_common_protos_version = "1.65.0"
_minimum_pyyaml_version = "3.11"
@@ -214,11 +214,10 @@
"numpy>=%s" % _minimum_numpy_version,
"pyyaml>=%s" % _minimum_pyyaml_version,
],
- python_requires=">=3.9",
+ python_requires=">=3.10",
classifiers=[
"Development Status :: 5 - Production/Stable",
"License :: OSI Approved :: Apache Software License",
- "Programming Language :: Python :: 3.9",
"Programming Language :: Python :: 3.10",
"Programming Language :: Python :: 3.11",
"Programming Language :: Python :: 3.12",
diff --git a/python/pyspark/errors/__init__.py b/python/pyspark/errors/__init__.py
index 98f8aa593371e..c20b057b74e61 100644
--- a/python/pyspark/errors/__init__.py
+++ b/python/pyspark/errors/__init__.py
@@ -46,7 +46,6 @@
PySparkAssertionError,
PySparkNotImplementedError,
PySparkPicklingError,
- RetriesExceeded,
PySparkKeyError,
QueryContext,
QueryContextType,
@@ -83,7 +82,6 @@
"PySparkAssertionError",
"PySparkNotImplementedError",
"PySparkPicklingError",
- "RetriesExceeded",
"PySparkKeyError",
"QueryContext",
"QueryContextType",
diff --git a/python/pyspark/errors/error-conditions.json b/python/pyspark/errors/error-conditions.json
index b9ed246753906..1da6b8e154784 100644
--- a/python/pyspark/errors/error-conditions.json
+++ b/python/pyspark/errors/error-conditions.json
@@ -174,6 +174,13 @@
"`` does not allow a Column in a list."
]
},
+ "CONFLICTING_PIPELINE_REFRESH_OPTIONS" : {
+ "message" : [
+ "--full-refresh-all option conflicts with ",
+ "The --full-refresh-all option performs a full refresh of all datasets, ",
+ "so specifying individual datasets with is not allowed."
+ ]
+ },
"CONNECT_URL_ALREADY_DEFINED": {
"message": [
"Only one Spark Connect client URL can be set; however, got a different URL [] from the existing []."
@@ -377,6 +384,11 @@
"The return type of the arrow-optimized Python UDTF should be of type 'pandas.DataFrame', but the '' method returned a value of type with value: ."
]
},
+ "INVALID_ARROW_UDTF_WITH_ANALYZE": {
+ "message": [
+ "The arrow UDTF '' is invalid. Arrow UDTFs do not support the 'analyze' method. Please remove the 'analyze' method from '' and specify a returnType instead."
+ ]
+ },
"INVALID_BROADCAST_OPERATION": {
"message": [
"Broadcast can only be in driver."
@@ -884,6 +896,11 @@
"No pipeline.yaml or pipeline.yml file provided in arguments or found in directory `` or readable ancestor directories."
]
},
+ "PIPELINE_SPEC_MISSING_REQUIRED_FIELD": {
+ "message": [
+ "Pipeline spec missing required field ``."
+ ]
+ },
"PIPELINE_SPEC_UNEXPECTED_FIELD": {
"message": [
"Pipeline spec field `` is unexpected."
@@ -970,14 +987,14 @@
"Columns do not match in their data type: ."
]
},
- "RETRIES_EXCEEDED": {
+ "REUSE_OBSERVATION": {
"message": [
- "The maximum number of retries has been exceeded."
+ "An Observation can be used with a DataFrame only once."
]
},
- "REUSE_OBSERVATION": {
+ "SCHEMA_MISMATCH_FOR_ARROW_PYTHON_UDF": {
"message": [
- "An Observation can be used with a DataFrame only once."
+ "Result vector from was not the required length: expected , got ."
]
},
"SCHEMA_MISMATCH_FOR_PANDAS_UDF": {
@@ -990,6 +1007,73 @@
"Cannot start a remote Spark session because there is a regular Spark session already running."
]
},
+ "SESSION_MUTATION_IN_DECLARATIVE_PIPELINE": {
+ "message": [
+ "Session mutation is not allowed in declarative pipelines."
+ ],
+ "sub_class": {
+ "SET_RUNTIME_CONF": {
+ "message": [
+ "Instead set configuration via the pipeline spec or use the 'spark_conf' argument in various decorators."
+ ]
+ },
+ "SET_CURRENT_CATALOG": {
+ "message": [
+ "Instead set catalog via the pipeline spec or the 'name' argument on the dataset decorators."
+ ]
+ },
+ "SET_CURRENT_DATABASE": {
+ "message": [
+ "Instead set database via the pipeline spec or the 'name' argument on the dataset decorators."
+ ]
+ },
+ "DROP_TEMP_VIEW": {
+ "message": [
+ "Instead remove the temporary view definition directly."
+ ]
+ },
+ "DROP_GLOBAL_TEMP_VIEW": {
+ "message": [
+ "Instead remove the temporary view definition directly."
+ ]
+ },
+ "CREATE_TEMP_VIEW": {
+ "message": [
+ "Instead use the @temporary_view decorator to define temporary views."
+ ]
+ },
+ "CREATE_OR_REPLACE_TEMP_VIEW": {
+ "message": [
+ "Instead use the @temporary_view decorator to define temporary views."
+ ]
+ },
+ "CREATE_GLOBAL_TEMP_VIEW": {
+ "message": [
+ "Instead use the @temporary_view decorator to define temporary views."
+ ]
+ },
+ "CREATE_OR_REPLACE_GLOBAL_TEMP_VIEW": {
+ "message": [
+ "Instead use the @temporary_view decorator to define temporary views."
+ ]
+ },
+ "REGISTER_UDF": {
+ "message": [
+ ""
+ ]
+ },
+ "REGISTER_JAVA_UDF": {
+ "message": [
+ ""
+ ]
+ },
+ "REGISTER_JAVA_UDAF": {
+ "message": [
+ ""
+ ]
+ }
+ }
+ },
"SESSION_NEED_CONN_STR_OR_BUILDER": {
"message": [
"Needs either connection string or channelBuilder (mutually exclusive) to create a new SparkSession."
@@ -1062,7 +1146,7 @@
},
"UDTF_ARROW_TYPE_CONVERSION_ERROR": {
"message": [
- "Cannot convert the output value of the input '' with type '' to the specified return type of the column: ''. Please check if the data types match and try again."
+ "PyArrow UDTF must return an iterator of pyarrow.Table or pyarrow.RecordBatch objects."
]
},
"UDTF_CONSTRUCTOR_INVALID_IMPLEMENTS_ANALYZE_METHOD": {
diff --git a/python/pyspark/errors/exceptions/base.py b/python/pyspark/errors/exceptions/base.py
index 4fa7ea92e34cb..b0455b64dabfb 100644
--- a/python/pyspark/errors/exceptions/base.py
+++ b/python/pyspark/errors/exceptions/base.py
@@ -350,13 +350,6 @@ class PySparkPicklingError(PySparkException, PicklingError):
"""
-class RetriesExceeded(PySparkException):
- """
- Represents an exception which is considered retriable, but retry limits
- were exceeded
- """
-
-
class PySparkKeyError(PySparkException, KeyError):
"""
Wrapper class for KeyError to support error classes.
diff --git a/python/pyspark/errors/tests/test_connect_errors_conversion.py b/python/pyspark/errors/tests/test_connect_errors_conversion.py
index 344af2ad62331..f80a6301d61b6 100644
--- a/python/pyspark/errors/tests/test_connect_errors_conversion.py
+++ b/python/pyspark/errors/tests/test_connect_errors_conversion.py
@@ -18,7 +18,7 @@
import unittest
-from pyspark.testing import should_test_connect, connect_requirement_message
+from pyspark.testing.utils import should_test_connect, connect_requirement_message
if should_test_connect:
from pyspark.errors.exceptions.connect import (
diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py
index a5fdaed0db2c4..f66fc762971b5 100644
--- a/python/pyspark/ml/classification.py
+++ b/python/pyspark/ml/classification.py
@@ -889,15 +889,14 @@ def summary(self) -> "LinearSVCTrainingSummary": # type: ignore[override]
Gets summary (accuracy/precision/recall, objective history, total iterations) of model
trained on the training set. An exception is thrown if `trainingSummary is None`.
"""
- if self.hasSummary:
- s = LinearSVCTrainingSummary(super(LinearSVCModel, self).summary)
- if is_remote():
- s.__source_transformer__ = self # type: ignore[attr-defined]
- return s
- else:
- raise RuntimeError(
- "No training summary available for this %s" % self.__class__.__name__
- )
+ return super().summary
+
+ @property
+ def _summaryCls(self) -> type:
+ return LinearSVCTrainingSummary
+
+ def _summary_dataset(self, train_dataset: DataFrame) -> DataFrame:
+ return train_dataset
def evaluate(self, dataset: DataFrame) -> "LinearSVCSummary":
"""
@@ -1577,29 +1576,6 @@ def interceptVector(self) -> Vector:
"""
return self._call_java("interceptVector")
- @property
- @since("2.0.0")
- def summary(self) -> "LogisticRegressionTrainingSummary":
- """
- Gets summary (accuracy/precision/recall, objective history, total iterations) of model
- trained on the training set. An exception is thrown if `trainingSummary is None`.
- """
- if self.hasSummary:
- s: LogisticRegressionTrainingSummary
- if self.numClasses <= 2:
- s = BinaryLogisticRegressionTrainingSummary(
- super(LogisticRegressionModel, self).summary
- )
- else:
- s = LogisticRegressionTrainingSummary(super(LogisticRegressionModel, self).summary)
- if is_remote():
- s.__source_transformer__ = self # type: ignore[attr-defined]
- return s
- else:
- raise RuntimeError(
- "No training summary available for this %s" % self.__class__.__name__
- )
-
def evaluate(self, dataset: DataFrame) -> "LogisticRegressionSummary":
"""
Evaluates the model on a test dataset.
@@ -1623,6 +1599,15 @@ def evaluate(self, dataset: DataFrame) -> "LogisticRegressionSummary":
s.__source_transformer__ = self # type: ignore[attr-defined]
return s
+ @property
+ def _summaryCls(self) -> type:
+ if self.numClasses <= 2:
+ return BinaryLogisticRegressionTrainingSummary
+ return LogisticRegressionTrainingSummary
+
+ def _summary_dataset(self, train_dataset: DataFrame) -> DataFrame:
+ return train_dataset
+
class LogisticRegressionSummary(_ClassificationSummary):
"""
@@ -2315,29 +2300,13 @@ def trees(self) -> List[DecisionTreeClassificationModel]:
return [DecisionTreeClassificationModel(m) for m in list(self._call_java("trees"))]
@property
- @since("3.1.0")
- def summary(self) -> "RandomForestClassificationTrainingSummary":
- """
- Gets summary (accuracy/precision/recall, objective history, total iterations) of model
- trained on the training set. An exception is thrown if `trainingSummary is None`.
- """
- if self.hasSummary:
- s: RandomForestClassificationTrainingSummary
- if self.numClasses <= 2:
- s = BinaryRandomForestClassificationTrainingSummary(
- super(RandomForestClassificationModel, self).summary
- )
- else:
- s = RandomForestClassificationTrainingSummary(
- super(RandomForestClassificationModel, self).summary
- )
- if is_remote():
- s.__source_transformer__ = self # type: ignore[attr-defined]
- return s
- else:
- raise RuntimeError(
- "No training summary available for this %s" % self.__class__.__name__
- )
+ def _summaryCls(self) -> type:
+ if self.numClasses <= 2:
+ return BinaryRandomForestClassificationTrainingSummary
+ return RandomForestClassificationTrainingSummary
+
+ def _summary_dataset(self, train_dataset: DataFrame) -> DataFrame:
+ return train_dataset
def evaluate(self, dataset: DataFrame) -> "RandomForestClassificationSummary":
"""
@@ -3372,17 +3341,14 @@ def summary( # type: ignore[override]
Gets summary (accuracy/precision/recall, objective history, total iterations) of model
trained on the training set. An exception is thrown if `trainingSummary is None`.
"""
- if self.hasSummary:
- s = MultilayerPerceptronClassificationTrainingSummary(
- super(MultilayerPerceptronClassificationModel, self).summary
- )
- if is_remote():
- s.__source_transformer__ = self # type: ignore[attr-defined]
- return s
- else:
- raise RuntimeError(
- "No training summary available for this %s" % self.__class__.__name__
- )
+ return super().summary
+
+ @property
+ def _summaryCls(self) -> type:
+ return MultilayerPerceptronClassificationTrainingSummary
+
+ def _summary_dataset(self, train_dataset: DataFrame) -> DataFrame:
+ return train_dataset
def evaluate(self, dataset: DataFrame) -> "MultilayerPerceptronClassificationSummary":
"""
@@ -4321,22 +4287,6 @@ def factors(self) -> Matrix:
"""
return self._call_java("factors")
- @since("3.1.0")
- def summary(self) -> "FMClassificationTrainingSummary":
- """
- Gets summary (accuracy/precision/recall, objective history, total iterations) of model
- trained on the training set. An exception is thrown if `trainingSummary is None`.
- """
- if self.hasSummary:
- s = FMClassificationTrainingSummary(super(FMClassificationModel, self).summary)
- if is_remote():
- s.__source_transformer__ = self # type: ignore[attr-defined]
- return s
- else:
- raise RuntimeError(
- "No training summary available for this %s" % self.__class__.__name__
- )
-
def evaluate(self, dataset: DataFrame) -> "FMClassificationSummary":
"""
Evaluates the model on a test dataset.
@@ -4356,6 +4306,21 @@ def evaluate(self, dataset: DataFrame) -> "FMClassificationSummary":
s.__source_transformer__ = self # type: ignore[attr-defined]
return s
+ @since("3.1.0")
+ def summary(self) -> "FMClassificationTrainingSummary":
+ """
+ Gets summary (accuracy/precision/recall, objective history, total iterations) of model
+ trained on the training set. An exception is thrown if `trainingSummary is None`.
+ """
+ return super().summary
+
+ @property
+ def _summaryCls(self) -> type:
+ return FMClassificationTrainingSummary
+
+ def _summary_dataset(self, train_dataset: DataFrame) -> DataFrame:
+ return train_dataset
+
class FMClassificationSummary(_BinaryClassificationSummary):
"""
diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py
index 7267ee2805987..0e26398de3c45 100644
--- a/python/pyspark/ml/clustering.py
+++ b/python/pyspark/ml/clustering.py
@@ -255,23 +255,6 @@ def gaussiansDF(self) -> DataFrame:
"""
return self._call_java("gaussiansDF")
- @property
- @since("2.1.0")
- def summary(self) -> "GaussianMixtureSummary":
- """
- Gets summary (cluster assignments, cluster sizes) of the model trained on the
- training set. An exception is thrown if no summary exists.
- """
- if self.hasSummary:
- s = GaussianMixtureSummary(super(GaussianMixtureModel, self).summary)
- if is_remote():
- s.__source_transformer__ = self # type: ignore[attr-defined]
- return s
- else:
- raise RuntimeError(
- "No training summary available for this %s" % self.__class__.__name__
- )
-
@since("3.0.0")
def predict(self, value: Vector) -> int:
"""
@@ -286,6 +269,10 @@ def predictProbability(self, value: Vector) -> Vector:
"""
return self._call_java("predictProbability", value)
+ @property
+ def _summaryCls(self) -> type:
+ return GaussianMixtureSummary
+
@inherit_doc
class GaussianMixture(
@@ -705,23 +692,6 @@ def numFeatures(self) -> int:
"""
return self._call_java("numFeatures")
- @property
- @since("2.1.0")
- def summary(self) -> KMeansSummary:
- """
- Gets summary (cluster assignments, cluster sizes) of the model trained on the
- training set. An exception is thrown if no summary exists.
- """
- if self.hasSummary:
- s = KMeansSummary(super(KMeansModel, self).summary)
- if is_remote():
- s.__source_transformer__ = self # type: ignore[attr-defined]
- return s
- else:
- raise RuntimeError(
- "No training summary available for this %s" % self.__class__.__name__
- )
-
@since("3.0.0")
def predict(self, value: Vector) -> int:
"""
@@ -729,6 +699,10 @@ def predict(self, value: Vector) -> int:
"""
return self._call_java("predict", value)
+ @property
+ def _summaryCls(self) -> type:
+ return KMeansSummary
+
@inherit_doc
class KMeans(JavaEstimator[KMeansModel], _KMeansParams, JavaMLWritable, JavaMLReadable["KMeans"]):
@@ -1055,23 +1029,6 @@ def numFeatures(self) -> int:
"""
return self._call_java("numFeatures")
- @property
- @since("2.1.0")
- def summary(self) -> "BisectingKMeansSummary":
- """
- Gets summary (cluster assignments, cluster sizes) of the model trained on the
- training set. An exception is thrown if no summary exists.
- """
- if self.hasSummary:
- s = BisectingKMeansSummary(super(BisectingKMeansModel, self).summary)
- if is_remote():
- s.__source_transformer__ = self # type: ignore[attr-defined]
- return s
- else:
- raise RuntimeError(
- "No training summary available for this %s" % self.__class__.__name__
- )
-
@since("3.0.0")
def predict(self, value: Vector) -> int:
"""
@@ -1079,6 +1036,10 @@ def predict(self, value: Vector) -> int:
"""
return self._call_java("predict", value)
+ @property
+ def _summaryCls(self) -> type:
+ return BisectingKMeansSummary
+
@inherit_doc
class BisectingKMeans(
diff --git a/python/pyspark/ml/connect/functions.py b/python/pyspark/ml/connect/functions.py
index 22ff32a5946de..8df40a400c5fe 100644
--- a/python/pyspark/ml/connect/functions.py
+++ b/python/pyspark/ml/connect/functions.py
@@ -58,7 +58,7 @@ def _test() -> None:
print("Not supported in no-GIL mode", file=sys.stderr)
sys.exit(0)
- from pyspark.testing import should_test_connect
+ from pyspark.testing.utils import should_test_connect
if not should_test_connect:
print("Skipping pyspark.ml.connect.functions doctests", file=sys.stderr)
diff --git a/python/pyspark/ml/connect/proto.py b/python/pyspark/ml/connect/proto.py
index 31f100859281a..7cffd32631ba5 100644
--- a/python/pyspark/ml/connect/proto.py
+++ b/python/pyspark/ml/connect/proto.py
@@ -70,8 +70,13 @@ class AttributeRelation(LogicalPlan):
could be a model or a summary. This attribute returns a DataFrame.
"""
- def __init__(self, ref_id: str, methods: List[pb2.Fetch.Method]) -> None:
- super().__init__(None)
+ def __init__(
+ self,
+ ref_id: str,
+ methods: List[pb2.Fetch.Method],
+ child: Optional["LogicalPlan"] = None,
+ ) -> None:
+ super().__init__(child)
self._ref_id = ref_id
self._methods = methods
@@ -79,4 +84,6 @@ def plan(self, session: "SparkConnectClient") -> pb2.Relation:
plan = self._create_proto_relation()
plan.ml_relation.fetch.obj_ref.CopyFrom(pb2.ObjectRef(id=self._ref_id))
plan.ml_relation.fetch.methods.extend(self._methods)
+ if self._child is not None:
+ plan.ml_relation.model_summary_dataset.CopyFrom(self._child.plan(session))
return plan
diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py
index 66d6dbd6a2678..ce97b98f6665c 100644
--- a/python/pyspark/ml/regression.py
+++ b/python/pyspark/ml/regression.py
@@ -479,22 +479,11 @@ def scale(self) -> float:
return self._call_java("scale")
@property
- @since("2.0.0")
- def summary(self) -> "LinearRegressionTrainingSummary":
- """
- Gets summary (residuals, MSE, r-squared ) of model on
- training set. An exception is thrown if
- `trainingSummary is None`.
- """
- if self.hasSummary:
- s = LinearRegressionTrainingSummary(super(LinearRegressionModel, self).summary)
- if is_remote():
- s.__source_transformer__ = self # type: ignore[attr-defined]
- return s
- else:
- raise RuntimeError(
- "No training summary available for this %s" % self.__class__.__name__
- )
+ def _summaryCls(self) -> type:
+ return LinearRegressionTrainingSummary
+
+ def _summary_dataset(self, train_dataset: DataFrame) -> DataFrame:
+ return train_dataset
def evaluate(self, dataset: DataFrame) -> "LinearRegressionSummary":
"""
@@ -2774,24 +2763,11 @@ def intercept(self) -> float:
return self._call_java("intercept")
@property
- @since("2.0.0")
- def summary(self) -> "GeneralizedLinearRegressionTrainingSummary":
- """
- Gets summary (residuals, deviance, p-values) of model on
- training set. An exception is thrown if
- `trainingSummary is None`.
- """
- if self.hasSummary:
- s = GeneralizedLinearRegressionTrainingSummary(
- super(GeneralizedLinearRegressionModel, self).summary
- )
- if is_remote():
- s.__source_transformer__ = self # type: ignore[attr-defined]
- return s
- else:
- raise RuntimeError(
- "No training summary available for this %s" % self.__class__.__name__
- )
+ def _summaryCls(self) -> type:
+ return GeneralizedLinearRegressionTrainingSummary
+
+ def _summary_dataset(self, train_dataset: DataFrame) -> DataFrame:
+ return train_dataset
def evaluate(self, dataset: DataFrame) -> "GeneralizedLinearRegressionSummary":
"""
diff --git a/python/pyspark/ml/tests/connect/test_connect_cache.py b/python/pyspark/ml/tests/connect/test_connect_cache.py
index 8d156a0f11e1d..b6c801f32eaf0 100644
--- a/python/pyspark/ml/tests/connect/test_connect_cache.py
+++ b/python/pyspark/ml/tests/connect/test_connect_cache.py
@@ -48,20 +48,27 @@ def test_delete_model(self):
"obj: class org.apache.spark.ml.classification.LinearSVCModel" in cache_info[0],
cache_info,
)
- assert model._java_obj._ref_count == 1
+ # the `model._summary` holds another ref to the remote model.
+ assert model._java_obj._ref_count == 2
+
+ model_size = spark.client._query_model_size(model._java_obj.ref_id)
+ assert isinstance(model_size, int) and model_size > 0
model2 = model.copy()
cache_info = spark.client._get_ml_cache_info()
self.assertEqual(len(cache_info), 1)
- assert model._java_obj._ref_count == 2
- assert model2._java_obj._ref_count == 2
+ assert model._java_obj._ref_count == 3
+ assert model2._java_obj._ref_count == 3
# explicitly delete the model
del model
cache_info = spark.client._get_ml_cache_info()
self.assertEqual(len(cache_info), 1)
- assert model2._java_obj._ref_count == 1
+ # Note the copied model 'model2' also holds the `_summary` object,
+ # and the `_summary` object holds another ref to the remote model.
+ # so the ref count is 2.
+ assert model2._java_obj._ref_count == 2
del model2
cache_info = spark.client._get_ml_cache_info()
@@ -99,7 +106,6 @@ def test_cleanup_ml_cache(self):
cache_info,
)
- # explicitly delete the model1
del model1
cache_info = spark.client._get_ml_cache_info()
diff --git a/python/pyspark/ml/tests/connect/test_parity_classification.py b/python/pyspark/ml/tests/connect/test_parity_classification.py
index 3c7e8ff71a2d6..7805546dba707 100644
--- a/python/pyspark/ml/tests/connect/test_parity_classification.py
+++ b/python/pyspark/ml/tests/connect/test_parity_classification.py
@@ -21,6 +21,8 @@
from pyspark.testing.connectutils import ReusedConnectTestCase
+# TODO(SPARK-52764): Re-enable this test after fixing the flakiness.
+@unittest.skip("Disabled due to flakiness, should be enabled after fixing the issue")
class ClassificationParityTests(ClassificationTestsMixin, ReusedConnectTestCase):
pass
diff --git a/python/pyspark/ml/tests/connect/test_parity_clustering.py b/python/pyspark/ml/tests/connect/test_parity_clustering.py
index 99714b0d6962b..bbfd2a2aea806 100644
--- a/python/pyspark/ml/tests/connect/test_parity_clustering.py
+++ b/python/pyspark/ml/tests/connect/test_parity_clustering.py
@@ -21,6 +21,8 @@
from pyspark.testing.connectutils import ReusedConnectTestCase
+# TODO(SPARK-52764): Re-enable this test after fixing the flakiness.
+@unittest.skip("Disabled due to flakiness, should be enabled after fixing the issue")
class ClusteringParityTests(ClusteringTestsMixin, ReusedConnectTestCase):
pass
diff --git a/python/pyspark/ml/tests/connect/test_parity_regression.py b/python/pyspark/ml/tests/connect/test_parity_regression.py
index 7c2743a938fa0..407280827076c 100644
--- a/python/pyspark/ml/tests/connect/test_parity_regression.py
+++ b/python/pyspark/ml/tests/connect/test_parity_regression.py
@@ -21,6 +21,8 @@
from pyspark.testing.connectutils import ReusedConnectTestCase
+# TODO(SPARK-52764): Re-enable this test after fixing the flakiness.
+@unittest.skip("Disabled due to flakiness, should be enabled after fixing the issue")
class RegressionParityTests(RegressionTestsMixin, ReusedConnectTestCase):
pass
diff --git a/python/pyspark/ml/tests/connect/test_parity_torch_distributor.py b/python/pyspark/ml/tests/connect/test_parity_torch_distributor.py
index fc0500b8e83a7..3061ecb60c1d9 100644
--- a/python/pyspark/ml/tests/connect/test_parity_torch_distributor.py
+++ b/python/pyspark/ml/tests/connect/test_parity_torch_distributor.py
@@ -21,8 +21,12 @@
from pyspark.util import is_remote_only
from pyspark.sql import SparkSession
-from pyspark.testing import should_test_connect, connect_requirement_message
-from pyspark.testing.utils import have_torch, torch_requirement_message
+from pyspark.testing.utils import (
+ have_torch,
+ torch_requirement_message,
+ should_test_connect,
+ connect_requirement_message,
+)
if not is_remote_only() and should_test_connect:
from pyspark.ml.torch.tests.test_distributor import (
diff --git a/python/pyspark/ml/tests/test_classification.py b/python/pyspark/ml/tests/test_classification.py
index 57e4c0ef86dc0..21bce70e8735b 100644
--- a/python/pyspark/ml/tests/test_classification.py
+++ b/python/pyspark/ml/tests/test_classification.py
@@ -55,6 +55,7 @@
MultilayerPerceptronClassificationTrainingSummary,
)
from pyspark.ml.regression import DecisionTreeRegressionModel
+from pyspark.sql import is_remote
from pyspark.testing.sqlutils import ReusedSQLTestCase
@@ -241,37 +242,45 @@ def test_binary_logistic_regression_summary(self):
model = lr.fit(df)
self.assertEqual(lr.uid, model.uid)
self.assertTrue(model.hasSummary)
- s = model.summary
- # test that api is callable and returns expected types
- self.assertTrue(isinstance(s.predictions, DataFrame))
- self.assertEqual(s.probabilityCol, "probability")
- self.assertEqual(s.labelCol, "label")
- self.assertEqual(s.featuresCol, "features")
- self.assertEqual(s.predictionCol, "prediction")
- objHist = s.objectiveHistory
- self.assertTrue(isinstance(objHist, list) and isinstance(objHist[0], float))
- self.assertGreater(s.totalIterations, 0)
- self.assertTrue(isinstance(s.labels, list))
- self.assertTrue(isinstance(s.truePositiveRateByLabel, list))
- self.assertTrue(isinstance(s.falsePositiveRateByLabel, list))
- self.assertTrue(isinstance(s.precisionByLabel, list))
- self.assertTrue(isinstance(s.recallByLabel, list))
- self.assertTrue(isinstance(s.fMeasureByLabel(), list))
- self.assertTrue(isinstance(s.fMeasureByLabel(1.0), list))
- self.assertTrue(isinstance(s.roc, DataFrame))
- self.assertAlmostEqual(s.areaUnderROC, 1.0, 2)
- self.assertTrue(isinstance(s.pr, DataFrame))
- self.assertTrue(isinstance(s.fMeasureByThreshold, DataFrame))
- self.assertTrue(isinstance(s.precisionByThreshold, DataFrame))
- self.assertTrue(isinstance(s.recallByThreshold, DataFrame))
- self.assertAlmostEqual(s.accuracy, 1.0, 2)
- self.assertAlmostEqual(s.weightedTruePositiveRate, 1.0, 2)
- self.assertAlmostEqual(s.weightedFalsePositiveRate, 0.0, 2)
- self.assertAlmostEqual(s.weightedRecall, 1.0, 2)
- self.assertAlmostEqual(s.weightedPrecision, 1.0, 2)
- self.assertAlmostEqual(s.weightedFMeasure(), 1.0, 2)
- self.assertAlmostEqual(s.weightedFMeasure(1.0), 1.0, 2)
+ def check_summary():
+ s = model.summary
+ # test that api is callable and returns expected types
+ self.assertTrue(isinstance(s.predictions, DataFrame))
+ self.assertEqual(s.probabilityCol, "probability")
+ self.assertEqual(s.labelCol, "label")
+ self.assertEqual(s.featuresCol, "features")
+ self.assertEqual(s.predictionCol, "prediction")
+ objHist = s.objectiveHistory
+ self.assertTrue(isinstance(objHist, list) and isinstance(objHist[0], float))
+ self.assertGreater(s.totalIterations, 0)
+ self.assertTrue(isinstance(s.labels, list))
+ self.assertTrue(isinstance(s.truePositiveRateByLabel, list))
+ self.assertTrue(isinstance(s.falsePositiveRateByLabel, list))
+ self.assertTrue(isinstance(s.precisionByLabel, list))
+ self.assertTrue(isinstance(s.recallByLabel, list))
+ self.assertTrue(isinstance(s.fMeasureByLabel(), list))
+ self.assertTrue(isinstance(s.fMeasureByLabel(1.0), list))
+ self.assertTrue(isinstance(s.roc, DataFrame))
+ self.assertAlmostEqual(s.areaUnderROC, 1.0, 2)
+ self.assertTrue(isinstance(s.pr, DataFrame))
+ self.assertTrue(isinstance(s.fMeasureByThreshold, DataFrame))
+ self.assertTrue(isinstance(s.precisionByThreshold, DataFrame))
+ self.assertTrue(isinstance(s.recallByThreshold, DataFrame))
+ self.assertAlmostEqual(s.accuracy, 1.0, 2)
+ self.assertAlmostEqual(s.weightedTruePositiveRate, 1.0, 2)
+ self.assertAlmostEqual(s.weightedFalsePositiveRate, 0.0, 2)
+ self.assertAlmostEqual(s.weightedRecall, 1.0, 2)
+ self.assertAlmostEqual(s.weightedPrecision, 1.0, 2)
+ self.assertAlmostEqual(s.weightedFMeasure(), 1.0, 2)
+ self.assertAlmostEqual(s.weightedFMeasure(1.0), 1.0, 2)
+
+ check_summary()
+ if is_remote():
+ self.spark.client._delete_ml_cache([model._java_obj._ref_id], evict_only=True)
+ check_summary()
+
+ s = model.summary
# test evaluation (with training dataset) produces a summary with same values
# one check is enough to verify a summary is returned, Scala version runs full test
sameSummary = model.evaluate(df)
@@ -292,31 +301,39 @@ def test_multiclass_logistic_regression_summary(self):
lr = LogisticRegression(maxIter=5, regParam=0.01, weightCol="weight", fitIntercept=False)
model = lr.fit(df)
self.assertTrue(model.hasSummary)
- s = model.summary
- # test that api is callable and returns expected types
- self.assertTrue(isinstance(s.predictions, DataFrame))
- self.assertEqual(s.probabilityCol, "probability")
- self.assertEqual(s.labelCol, "label")
- self.assertEqual(s.featuresCol, "features")
- self.assertEqual(s.predictionCol, "prediction")
- objHist = s.objectiveHistory
- self.assertTrue(isinstance(objHist, list) and isinstance(objHist[0], float))
- self.assertGreater(s.totalIterations, 0)
- self.assertTrue(isinstance(s.labels, list))
- self.assertTrue(isinstance(s.truePositiveRateByLabel, list))
- self.assertTrue(isinstance(s.falsePositiveRateByLabel, list))
- self.assertTrue(isinstance(s.precisionByLabel, list))
- self.assertTrue(isinstance(s.recallByLabel, list))
- self.assertTrue(isinstance(s.fMeasureByLabel(), list))
- self.assertTrue(isinstance(s.fMeasureByLabel(1.0), list))
- self.assertAlmostEqual(s.accuracy, 0.75, 2)
- self.assertAlmostEqual(s.weightedTruePositiveRate, 0.75, 2)
- self.assertAlmostEqual(s.weightedFalsePositiveRate, 0.25, 2)
- self.assertAlmostEqual(s.weightedRecall, 0.75, 2)
- self.assertAlmostEqual(s.weightedPrecision, 0.583, 2)
- self.assertAlmostEqual(s.weightedFMeasure(), 0.65, 2)
- self.assertAlmostEqual(s.weightedFMeasure(1.0), 0.65, 2)
+ def check_summary():
+ s = model.summary
+ # test that api is callable and returns expected types
+ self.assertTrue(isinstance(s.predictions, DataFrame))
+ self.assertEqual(s.probabilityCol, "probability")
+ self.assertEqual(s.labelCol, "label")
+ self.assertEqual(s.featuresCol, "features")
+ self.assertEqual(s.predictionCol, "prediction")
+ objHist = s.objectiveHistory
+ self.assertTrue(isinstance(objHist, list) and isinstance(objHist[0], float))
+ self.assertGreater(s.totalIterations, 0)
+ self.assertTrue(isinstance(s.labels, list))
+ self.assertTrue(isinstance(s.truePositiveRateByLabel, list))
+ self.assertTrue(isinstance(s.falsePositiveRateByLabel, list))
+ self.assertTrue(isinstance(s.precisionByLabel, list))
+ self.assertTrue(isinstance(s.recallByLabel, list))
+ self.assertTrue(isinstance(s.fMeasureByLabel(), list))
+ self.assertTrue(isinstance(s.fMeasureByLabel(1.0), list))
+ self.assertAlmostEqual(s.accuracy, 0.75, 2)
+ self.assertAlmostEqual(s.weightedTruePositiveRate, 0.75, 2)
+ self.assertAlmostEqual(s.weightedFalsePositiveRate, 0.25, 2)
+ self.assertAlmostEqual(s.weightedRecall, 0.75, 2)
+ self.assertAlmostEqual(s.weightedPrecision, 0.583, 2)
+ self.assertAlmostEqual(s.weightedFMeasure(), 0.65, 2)
+ self.assertAlmostEqual(s.weightedFMeasure(1.0), 0.65, 2)
+
+ check_summary()
+ if is_remote():
+ self.spark.client._delete_ml_cache([model._java_obj._ref_id], evict_only=True)
+ check_summary()
+
+ s = model.summary
# test evaluation (with training dataset) produces a summary with same values
# one check is enough to verify a summary is returned, Scala version runs full test
sameSummary = model.evaluate(df)
@@ -426,15 +443,21 @@ def test_linear_svc(self):
self.assertEqual(output.columns, expected_cols)
self.assertEqual(output.count(), 4)
- # model summary
- self.assertTrue(model.hasSummary)
- summary = model.summary()
- self.assertIsInstance(summary, LinearSVCSummary)
- self.assertIsInstance(summary, LinearSVCTrainingSummary)
- self.assertEqual(summary.labels, [0.0, 1.0])
- self.assertEqual(summary.accuracy, 0.5)
- self.assertEqual(summary.areaUnderROC, 0.75)
- self.assertEqual(summary.predictions.columns, expected_cols)
+ def check_summary():
+ # model summary
+ self.assertTrue(model.hasSummary)
+ summary = model.summary()
+ self.assertIsInstance(summary, LinearSVCSummary)
+ self.assertIsInstance(summary, LinearSVCTrainingSummary)
+ self.assertEqual(summary.labels, [0.0, 1.0])
+ self.assertEqual(summary.accuracy, 0.5)
+ self.assertEqual(summary.areaUnderROC, 0.75)
+ self.assertEqual(summary.predictions.columns, expected_cols)
+
+ check_summary()
+ if is_remote():
+ self.spark.client._delete_ml_cache([model._java_obj._ref_id], evict_only=True)
+ check_summary()
summary2 = model.evaluate(df)
self.assertIsInstance(summary2, LinearSVCSummary)
@@ -526,13 +549,20 @@ def test_factorization_machine(self):
# model summary
self.assertTrue(model.hasSummary)
- summary = model.summary()
- self.assertIsInstance(summary, FMClassificationSummary)
- self.assertIsInstance(summary, FMClassificationTrainingSummary)
- self.assertEqual(summary.labels, [0.0, 1.0])
- self.assertEqual(summary.accuracy, 0.25)
- self.assertEqual(summary.areaUnderROC, 0.5)
- self.assertEqual(summary.predictions.columns, expected_cols)
+
+ def check_summary():
+ summary = model.summary()
+ self.assertIsInstance(summary, FMClassificationSummary)
+ self.assertIsInstance(summary, FMClassificationTrainingSummary)
+ self.assertEqual(summary.labels, [0.0, 1.0])
+ self.assertEqual(summary.accuracy, 0.25)
+ self.assertEqual(summary.areaUnderROC, 0.5)
+ self.assertEqual(summary.predictions.columns, expected_cols)
+
+ check_summary()
+ if is_remote():
+ self.spark.client._delete_ml_cache([model._java_obj._ref_id], evict_only=True)
+ check_summary()
summary2 = model.evaluate(df)
self.assertIsInstance(summary2, FMClassificationSummary)
@@ -773,21 +803,27 @@ def test_binary_random_forest_classifier(self):
self.assertEqual(tree.transform(df).count(), 4)
self.assertEqual(tree.transform(df).columns, expected_cols)
- # model summary
- summary = model.summary
- self.assertTrue(isinstance(summary, BinaryRandomForestClassificationSummary))
- self.assertTrue(isinstance(summary, BinaryRandomForestClassificationTrainingSummary))
- self.assertEqual(summary.labels, [0.0, 1.0])
- self.assertEqual(summary.accuracy, 0.75)
- self.assertEqual(summary.areaUnderROC, 0.875)
- self.assertEqual(summary.predictions.columns, expected_cols)
+ def check_summary():
+ # model summary
+ summary = model.summary
+ self.assertTrue(isinstance(summary, BinaryRandomForestClassificationSummary))
+ self.assertTrue(isinstance(summary, BinaryRandomForestClassificationTrainingSummary))
+ self.assertEqual(summary.labels, [0.0, 1.0])
+ self.assertEqual(summary.accuracy, 0.75)
+ self.assertEqual(summary.areaUnderROC, 0.875)
+ self.assertEqual(summary.predictions.columns, expected_cols)
+
+ check_summary()
+ if is_remote():
+ self.spark.client._delete_ml_cache([model._java_obj._ref_id], evict_only=True)
+ check_summary()
summary2 = model.evaluate(df)
self.assertTrue(isinstance(summary2, BinaryRandomForestClassificationSummary))
self.assertFalse(isinstance(summary2, BinaryRandomForestClassificationTrainingSummary))
self.assertEqual(summary2.labels, [0.0, 1.0])
self.assertEqual(summary2.accuracy, 0.75)
- self.assertEqual(summary.areaUnderROC, 0.875)
+ self.assertEqual(summary2.areaUnderROC, 0.875)
self.assertEqual(summary2.predictions.columns, expected_cols)
# Model save & load
@@ -859,13 +895,19 @@ def test_multiclass_random_forest_classifier(self):
self.assertEqual(output.columns, expected_cols)
self.assertEqual(output.count(), 4)
- # model summary
- summary = model.summary
- self.assertTrue(isinstance(summary, RandomForestClassificationSummary))
- self.assertTrue(isinstance(summary, RandomForestClassificationTrainingSummary))
- self.assertEqual(summary.labels, [0.0, 1.0, 2.0])
- self.assertEqual(summary.accuracy, 0.5)
- self.assertEqual(summary.predictions.columns, expected_cols)
+ def check_summary():
+ # model summary
+ summary = model.summary
+ self.assertTrue(isinstance(summary, RandomForestClassificationSummary))
+ self.assertTrue(isinstance(summary, RandomForestClassificationTrainingSummary))
+ self.assertEqual(summary.labels, [0.0, 1.0, 2.0])
+ self.assertEqual(summary.accuracy, 0.5)
+ self.assertEqual(summary.predictions.columns, expected_cols)
+
+ check_summary()
+ if is_remote():
+ self.spark.client._delete_ml_cache([model._java_obj._ref_id], evict_only=True)
+ check_summary()
summary2 = model.evaluate(df)
self.assertTrue(isinstance(summary2, RandomForestClassificationSummary))
@@ -953,14 +995,20 @@ def test_mlp(self):
self.assertEqual(output.columns, expected_cols)
self.assertEqual(output.count(), 4)
- # model summary
- self.assertTrue(model.hasSummary)
- summary = model.summary()
- self.assertIsInstance(summary, MultilayerPerceptronClassificationSummary)
- self.assertIsInstance(summary, MultilayerPerceptronClassificationTrainingSummary)
- self.assertEqual(summary.labels, [0.0, 1.0])
- self.assertEqual(summary.accuracy, 0.75)
- self.assertEqual(summary.predictions.columns, expected_cols)
+ def check_summary():
+ # model summary
+ self.assertTrue(model.hasSummary)
+ summary = model.summary()
+ self.assertIsInstance(summary, MultilayerPerceptronClassificationSummary)
+ self.assertIsInstance(summary, MultilayerPerceptronClassificationTrainingSummary)
+ self.assertEqual(summary.labels, [0.0, 1.0])
+ self.assertEqual(summary.accuracy, 0.75)
+ self.assertEqual(summary.predictions.columns, expected_cols)
+
+ check_summary()
+ if is_remote():
+ self.spark.client._delete_ml_cache([model._java_obj._ref_id], evict_only=True)
+ check_summary()
summary2 = model.evaluate(df)
self.assertIsInstance(summary2, MultilayerPerceptronClassificationSummary)
diff --git a/python/pyspark/ml/tests/test_clustering.py b/python/pyspark/ml/tests/test_clustering.py
index 1b8eb73135a96..fbf012babcc3d 100644
--- a/python/pyspark/ml/tests/test_clustering.py
+++ b/python/pyspark/ml/tests/test_clustering.py
@@ -85,23 +85,39 @@ def test_kmeans(self):
self.assertTrue(np.allclose(model.predict(Vectors.dense(0.0, 5.0)), 1, atol=1e-4))
- # Model summary
- self.assertTrue(model.hasSummary)
- summary = model.summary
- self.assertTrue(isinstance(summary, KMeansSummary))
- self.assertEqual(summary.k, 2)
- self.assertEqual(summary.numIter, 2)
- self.assertEqual(summary.clusterSizes, [4, 2])
- self.assertTrue(np.allclose(summary.trainingCost, 1.35710375, atol=1e-4))
+ def check_summary():
+ # Model summary
+ self.assertTrue(model.hasSummary)
+ summary = model.summary
+ self.assertTrue(isinstance(summary, KMeansSummary))
+ self.assertEqual(summary.k, 2)
+ self.assertEqual(summary.numIter, 2)
+ self.assertEqual(summary.clusterSizes, [4, 2])
+ self.assertTrue(np.allclose(summary.trainingCost, 1.35710375, atol=1e-4))
- self.assertEqual(summary.featuresCol, "features")
- self.assertEqual(summary.predictionCol, "prediction")
+ self.assertEqual(summary.featuresCol, "features")
+ self.assertEqual(summary.predictionCol, "prediction")
- self.assertEqual(summary.cluster.columns, ["prediction"])
- self.assertEqual(summary.cluster.count(), 6)
+ self.assertEqual(summary.cluster.columns, ["prediction"])
+ self.assertEqual(summary.cluster.count(), 6)
- self.assertEqual(summary.predictions.columns, expected_cols)
- self.assertEqual(summary.predictions.count(), 6)
+ self.assertEqual(summary.predictions.columns, expected_cols)
+ self.assertEqual(summary.predictions.count(), 6)
+
+ # check summary before model offloading occurs
+ check_summary()
+
+ if is_remote():
+ self.spark.client._delete_ml_cache([model._java_obj._ref_id], evict_only=True)
+ # check summary "try_remote_call" path after model offloading occurs
+ self.assertEqual(model.summary.numIter, 2)
+
+ self.spark.client._delete_ml_cache([model._java_obj._ref_id], evict_only=True)
+ # check summary "invoke_remote_attribute_relation" path after model offloading occurs
+ self.assertEqual(model.summary.cluster.count(), 6)
+
+ self.spark.client._delete_ml_cache([model._java_obj._ref_id], evict_only=True)
+ check_summary()
# save & load
with tempfile.TemporaryDirectory(prefix="kmeans_model") as d:
@@ -112,6 +128,9 @@ def test_kmeans(self):
model.write().overwrite().save(d)
model2 = KMeansModel.load(d)
self.assertEqual(str(model), str(model2))
+ self.assertFalse(model2.hasSummary)
+ with self.assertRaisesRegex(Exception, "No training summary available"):
+ model2.summary
def test_bisecting_kmeans(self):
df = (
@@ -278,30 +297,36 @@ def test_gaussian_mixture(self):
self.assertEqual(output.columns, expected_cols)
self.assertEqual(output.count(), 6)
- # Model summary
- self.assertTrue(model.hasSummary)
- summary = model.summary
- self.assertTrue(isinstance(summary, GaussianMixtureSummary))
- self.assertEqual(summary.k, 2)
- self.assertEqual(summary.numIter, 2)
- self.assertEqual(len(summary.clusterSizes), 2)
- self.assertEqual(summary.clusterSizes, [3, 3])
- ll = summary.logLikelihood
- self.assertTrue(ll < 0, ll)
- self.assertTrue(np.allclose(ll, -1.311264553744033, atol=1e-4), ll)
-
- self.assertEqual(summary.featuresCol, "features")
- self.assertEqual(summary.predictionCol, "prediction")
- self.assertEqual(summary.probabilityCol, "probability")
-
- self.assertEqual(summary.cluster.columns, ["prediction"])
- self.assertEqual(summary.cluster.count(), 6)
-
- self.assertEqual(summary.predictions.columns, expected_cols)
- self.assertEqual(summary.predictions.count(), 6)
-
- self.assertEqual(summary.probability.columns, ["probability"])
- self.assertEqual(summary.predictions.count(), 6)
+ def check_summary():
+ # Model summary
+ self.assertTrue(model.hasSummary)
+ summary = model.summary
+ self.assertTrue(isinstance(summary, GaussianMixtureSummary))
+ self.assertEqual(summary.k, 2)
+ self.assertEqual(summary.numIter, 2)
+ self.assertEqual(len(summary.clusterSizes), 2)
+ self.assertEqual(summary.clusterSizes, [3, 3])
+ ll = summary.logLikelihood
+ self.assertTrue(ll < 0, ll)
+ self.assertTrue(np.allclose(ll, -1.311264553744033, atol=1e-4), ll)
+
+ self.assertEqual(summary.featuresCol, "features")
+ self.assertEqual(summary.predictionCol, "prediction")
+ self.assertEqual(summary.probabilityCol, "probability")
+
+ self.assertEqual(summary.cluster.columns, ["prediction"])
+ self.assertEqual(summary.cluster.count(), 6)
+
+ self.assertEqual(summary.predictions.columns, expected_cols)
+ self.assertEqual(summary.predictions.count(), 6)
+
+ self.assertEqual(summary.probability.columns, ["probability"])
+ self.assertEqual(summary.predictions.count(), 6)
+
+ check_summary()
+ if is_remote():
+ self.spark.client._delete_ml_cache([model._java_obj._ref_id], evict_only=True)
+ check_summary()
# save & load
with tempfile.TemporaryDirectory(prefix="gaussian_mixture") as d:
diff --git a/python/pyspark/ml/tests/test_pipeline.py b/python/pyspark/ml/tests/test_pipeline.py
index 892ce72e32bc4..fe12f9e504f1d 100644
--- a/python/pyspark/ml/tests/test_pipeline.py
+++ b/python/pyspark/ml/tests/test_pipeline.py
@@ -174,6 +174,7 @@ def test_clustering_pipeline(self):
self.assertEqual(str(model), str(model2))
self.assertEqual(str(model.stages), str(model2.stages))
+ @unittest.skip("Disabled due to flakiness, it might hang forever occasionally.")
def test_model_gc(self):
spark = self.spark
df1 = spark.createDataFrame(
@@ -201,6 +202,7 @@ def fit_transform_and_union(df1, df2):
output2 = fit_transform_and_union(df1, df2)
self.assertEqual(output2.count(), 13)
+ @unittest.skip("Disabled due to flakiness, it might hang forever occasionally.")
def test_model_training_summary_gc(self):
spark = self.spark
df1 = spark.createDataFrame(
@@ -228,6 +230,7 @@ def fit_predictions_and_union(df1, df2):
output2 = fit_predictions_and_union(df1, df2)
self.assertEqual(output2.count(), 13)
+ @unittest.skip("Disabled due to flakiness, it might hang forever occasionally.")
def test_model_testing_summary_gc(self):
spark = self.spark
df1 = spark.createDataFrame(
@@ -255,6 +258,7 @@ def fit_predictions_and_union(df1, df2):
output2 = fit_predictions_and_union(df1, df2)
self.assertEqual(output2.count(), 13)
+ @unittest.skip("Disabled due to flakiness, it might hang forever occasionally.")
def test_model_attr_df_gc(self):
spark = self.spark
df1 = (
diff --git a/python/pyspark/ml/tests/test_regression.py b/python/pyspark/ml/tests/test_regression.py
index 8638fb4d6078e..52688fdd63cf2 100644
--- a/python/pyspark/ml/tests/test_regression.py
+++ b/python/pyspark/ml/tests/test_regression.py
@@ -43,6 +43,7 @@
GBTRegressor,
GBTRegressionModel,
)
+from pyspark.sql import is_remote
from pyspark.testing.sqlutils import ReusedSQLTestCase
@@ -193,50 +194,58 @@ def test_linear_regression(self):
np.allclose(model.predict(Vectors.dense(0.0, 5.0)), 0.21249999999999963, atol=1e-4)
)
- # Model summary
- summary = model.summary
- self.assertTrue(isinstance(summary, LinearRegressionSummary))
- self.assertTrue(isinstance(summary, LinearRegressionTrainingSummary))
- self.assertEqual(summary.predictions.columns, expected_cols)
- self.assertEqual(summary.predictions.count(), 4)
- self.assertEqual(summary.residuals.columns, ["residuals"])
- self.assertEqual(summary.residuals.count(), 4)
-
- self.assertEqual(summary.degreesOfFreedom, 1)
- self.assertEqual(summary.numInstances, 4)
- self.assertEqual(summary.objectiveHistory, [0.0])
- self.assertTrue(
- np.allclose(
- summary.coefficientStandardErrors,
- [1.2859821149611763, 0.6248749874975031, 3.1645497310044184],
- atol=1e-4,
+ def check_summary():
+ # Model summary
+ summary = model.summary
+ self.assertTrue(isinstance(summary, LinearRegressionSummary))
+ self.assertTrue(isinstance(summary, LinearRegressionTrainingSummary))
+ self.assertEqual(summary.predictions.columns, expected_cols)
+ self.assertEqual(summary.predictions.count(), 4)
+ self.assertEqual(summary.residuals.columns, ["residuals"])
+ self.assertEqual(summary.residuals.count(), 4)
+
+ self.assertEqual(summary.degreesOfFreedom, 1)
+ self.assertEqual(summary.numInstances, 4)
+ self.assertEqual(summary.objectiveHistory, [0.0])
+ self.assertTrue(
+ np.allclose(
+ summary.coefficientStandardErrors,
+ [1.2859821149611763, 0.6248749874975031, 3.1645497310044184],
+ atol=1e-4,
+ )
)
- )
- self.assertTrue(
- np.allclose(
- summary.devianceResiduals, [-0.7424621202458727, 0.7875000000000003], atol=1e-4
+ self.assertTrue(
+ np.allclose(
+ summary.devianceResiduals, [-0.7424621202458727, 0.7875000000000003], atol=1e-4
+ )
)
- )
- self.assertTrue(
- np.allclose(
- summary.pValues,
- [0.7020630236843428, 0.8866003086182783, 0.9298746994547682],
- atol=1e-4,
+ self.assertTrue(
+ np.allclose(
+ summary.pValues,
+ [0.7020630236843428, 0.8866003086182783, 0.9298746994547682],
+ atol=1e-4,
+ )
)
- )
- self.assertTrue(
- np.allclose(
- summary.tValues,
- [0.5054502643838291, 0.1800360108036021, -0.11060025272186746],
- atol=1e-4,
+ self.assertTrue(
+ np.allclose(
+ summary.tValues,
+ [0.5054502643838291, 0.1800360108036021, -0.11060025272186746],
+ atol=1e-4,
+ )
)
- )
- self.assertTrue(np.allclose(summary.explainedVariance, 0.07997500000000031, atol=1e-4))
- self.assertTrue(np.allclose(summary.meanAbsoluteError, 0.4200000000000002, atol=1e-4))
- self.assertTrue(np.allclose(summary.meanSquaredError, 0.20212500000000005, atol=1e-4))
- self.assertTrue(np.allclose(summary.rootMeanSquaredError, 0.44958314025327956, atol=1e-4))
- self.assertTrue(np.allclose(summary.r2, 0.4427212572373862, atol=1e-4))
- self.assertTrue(np.allclose(summary.r2adj, -0.6718362282878414, atol=1e-4))
+ self.assertTrue(np.allclose(summary.explainedVariance, 0.07997500000000031, atol=1e-4))
+ self.assertTrue(np.allclose(summary.meanAbsoluteError, 0.4200000000000002, atol=1e-4))
+ self.assertTrue(np.allclose(summary.meanSquaredError, 0.20212500000000005, atol=1e-4))
+ self.assertTrue(
+ np.allclose(summary.rootMeanSquaredError, 0.44958314025327956, atol=1e-4)
+ )
+ self.assertTrue(np.allclose(summary.r2, 0.4427212572373862, atol=1e-4))
+ self.assertTrue(np.allclose(summary.r2adj, -0.6718362282878414, atol=1e-4))
+
+ check_summary()
+ if is_remote():
+ self.spark.client._delete_ml_cache([model._java_obj._ref_id], evict_only=True)
+ check_summary()
summary2 = model.evaluate(df)
self.assertTrue(isinstance(summary2, LinearRegressionSummary))
@@ -318,36 +327,43 @@ def test_generalized_linear_regression(self):
self.assertEqual(output.columns, expected_cols)
self.assertEqual(output.count(), 4)
- # Model summary
- self.assertTrue(model.hasSummary)
-
- summary = model.summary
- self.assertIsInstance(summary, GeneralizedLinearRegressionSummary)
- self.assertIsInstance(summary, GeneralizedLinearRegressionTrainingSummary)
- self.assertEqual(summary.numIterations, 1)
- self.assertEqual(summary.numInstances, 4)
- self.assertEqual(summary.rank, 3)
- self.assertTrue(
- np.allclose(
+ def check_summary():
+ # Model summary
+ self.assertTrue(model.hasSummary)
+
+ summary = model.summary
+ self.assertIsInstance(summary, GeneralizedLinearRegressionSummary)
+ self.assertIsInstance(summary, GeneralizedLinearRegressionTrainingSummary)
+ self.assertEqual(summary.numIterations, 1)
+ self.assertEqual(summary.numInstances, 4)
+ self.assertEqual(summary.rank, 3)
+ self.assertTrue(
+ np.allclose(
+ summary.tValues,
+ [0.3725037662281711, -0.49418209022924164, 2.6589353685797654],
+ atol=1e-4,
+ ),
summary.tValues,
- [0.3725037662281711, -0.49418209022924164, 2.6589353685797654],
- atol=1e-4,
- ),
- summary.tValues,
- )
- self.assertTrue(
- np.allclose(
+ )
+ self.assertTrue(
+ np.allclose(
+ summary.pValues,
+ [0.7729938686180984, 0.707802691825973, 0.22900885781807023],
+ atol=1e-4,
+ ),
summary.pValues,
- [0.7729938686180984, 0.707802691825973, 0.22900885781807023],
- atol=1e-4,
- ),
- summary.pValues,
- )
- self.assertEqual(summary.predictions.columns, expected_cols)
- self.assertEqual(summary.predictions.count(), 4)
- self.assertEqual(summary.residuals().columns, ["devianceResiduals"])
- self.assertEqual(summary.residuals().count(), 4)
+ )
+ self.assertEqual(summary.predictions.columns, expected_cols)
+ self.assertEqual(summary.predictions.count(), 4)
+ self.assertEqual(summary.residuals().columns, ["devianceResiduals"])
+ self.assertEqual(summary.residuals().count(), 4)
+ check_summary()
+ if is_remote():
+ self.spark.client._delete_ml_cache([model._java_obj._ref_id], evict_only=True)
+ check_summary()
+
+ summary = model.summary
summary2 = model.evaluate(df)
self.assertIsInstance(summary2, GeneralizedLinearRegressionSummary)
self.assertNotIsInstance(summary2, GeneralizedLinearRegressionTrainingSummary)
diff --git a/python/pyspark/ml/util.py b/python/pyspark/ml/util.py
index b86178a97c382..9072e88ca29fb 100644
--- a/python/pyspark/ml/util.py
+++ b/python/pyspark/ml/util.py
@@ -72,20 +72,6 @@
_logger = logging.getLogger("pyspark.ml.util")
-def try_remote_intermediate_result(f: FuncT) -> FuncT:
- """Mark the function/property that returns the intermediate result of the remote call.
- Eg, model.summary"""
-
- @functools.wraps(f)
- def wrapped(self: "JavaWrapper") -> Any:
- if is_remote() and "PYSPARK_NO_NAMESPACE_SHARE" not in os.environ:
- return f"{str(self._java_obj)}.{f.__name__}"
- else:
- return f(self)
-
- return cast(FuncT, wrapped)
-
-
def invoke_helper_attr(method: str, *args: Any) -> Any:
from pyspark.ml.wrapper import JavaWrapper
@@ -125,7 +111,12 @@ def invoke_remote_attribute_relation(
object_id = instance._java_obj # type: ignore
methods, obj_ref = _extract_id_methods(object_id)
methods.append(pb2.Fetch.Method(method=method, args=serialize(session.client, *args)))
- plan = AttributeRelation(obj_ref, methods)
+
+ if methods[0].method == "summary":
+ child = instance._summary_dataset._plan # type: ignore
+ else:
+ child = None
+ plan = AttributeRelation(obj_ref, methods, child=child)
# To delay the GC of the model, keep a reference to the source instance,
# might be a model or a summary.
@@ -204,6 +195,15 @@ def wrapped(self: "JavaEstimator", dataset: "ConnectDataFrame") -> Any:
_logger.warning(warning_msg)
remote_model_ref = RemoteModelRef(model_info.obj_ref.id)
model = self._create_model(remote_model_ref)
+ if isinstance(model, HasTrainingSummary):
+ summary_dataset = model._summary_dataset(dataset)
+
+ summary = model._summaryCls(f"{str(model._java_obj)}.summary") # type: ignore
+ summary._summary_dataset = summary_dataset
+ summary._remote_model_obj = model._java_obj # type: ignore
+ summary._remote_model_obj.add_ref()
+
+ model._summary = summary # type: ignore
if model.__class__.__name__ not in ["Bucketizer"]:
model._resetUid(self.uid)
return self._copyValues(model)
@@ -279,42 +279,69 @@ def try_remote_call(f: FuncT) -> FuncT:
@functools.wraps(f)
def wrapped(self: "JavaWrapper", name: str, *args: Any) -> Any:
if is_remote() and "PYSPARK_NO_NAMESPACE_SHARE" not in os.environ:
- # Launch a remote call if possible
+ from pyspark.errors.exceptions.connect import SparkException
import pyspark.sql.connect.proto as pb2
from pyspark.sql.connect.session import SparkSession
- from pyspark.ml.connect.util import _extract_id_methods
- from pyspark.ml.connect.serialize import serialize, deserialize
- from pyspark.ml.wrapper import JavaModel
session = SparkSession.getActiveSession()
- assert session is not None
- if self._java_obj == ML_CONNECT_HELPER_ID:
- obj_id = ML_CONNECT_HELPER_ID
- else:
- if isinstance(self, JavaModel):
- assert isinstance(self._java_obj, RemoteModelRef)
- obj_id = self._java_obj.ref_id
+
+ def remote_call() -> Any:
+ from pyspark.ml.connect.util import _extract_id_methods
+ from pyspark.ml.connect.serialize import serialize, deserialize
+ from pyspark.ml.wrapper import JavaModel
+
+ assert session is not None
+ if self._java_obj == ML_CONNECT_HELPER_ID:
+ obj_id = ML_CONNECT_HELPER_ID
else:
- # model summary
- obj_id = self._java_obj # type: ignore
- methods, obj_ref = _extract_id_methods(obj_id)
- methods.append(pb2.Fetch.Method(method=name, args=serialize(session.client, *args)))
- command = pb2.Command()
- command.ml_command.fetch.CopyFrom(
- pb2.Fetch(obj_ref=pb2.ObjectRef(id=obj_ref), methods=methods)
- )
- (_, properties, _) = session.client.execute_command(command)
- ml_command_result = properties["ml_command_result"]
- if ml_command_result.HasField("summary"):
- summary = ml_command_result.summary
- return summary
- elif ml_command_result.HasField("operator_info"):
- model_info = deserialize(properties)
- # get a new model ref id from the existing model,
- # it is up to the caller to build the model
- return model_info.obj_ref.id
- else:
- return deserialize(properties)
+ if isinstance(self, JavaModel):
+ assert isinstance(self._java_obj, RemoteModelRef)
+ obj_id = self._java_obj.ref_id
+ else:
+ # model summary
+ obj_id = self._java_obj # type: ignore
+ methods, obj_ref = _extract_id_methods(obj_id)
+ methods.append(pb2.Fetch.Method(method=name, args=serialize(session.client, *args)))
+ command = pb2.Command()
+ command.ml_command.fetch.CopyFrom(
+ pb2.Fetch(obj_ref=pb2.ObjectRef(id=obj_ref), methods=methods)
+ )
+ (_, properties, _) = session.client.execute_command(command)
+ ml_command_result = properties["ml_command_result"]
+ if ml_command_result.HasField("summary"):
+ summary = ml_command_result.summary
+ return summary
+ elif ml_command_result.HasField("operator_info"):
+ model_info = deserialize(properties)
+ # get a new model ref id from the existing model,
+ # it is up to the caller to build the model
+ return model_info.obj_ref.id
+ else:
+ return deserialize(properties)
+
+ try:
+ return remote_call()
+ except SparkException as e:
+ if e.getErrorClass() == "CONNECT_ML.MODEL_SUMMARY_LOST":
+ # the model summary is lost because the remote model was offloaded,
+ # send request to restore model.summary
+ create_summary_command = pb2.Command()
+ create_summary_command.ml_command.create_summary.CopyFrom(
+ pb2.MlCommand.CreateSummary(
+ model_ref=pb2.ObjectRef(
+ id=self._remote_model_obj.ref_id # type: ignore
+ ),
+ dataset=self._summary_dataset._plan.plan( # type: ignore
+ session.client # type: ignore
+ ),
+ )
+ )
+ session.client.execute_command(create_summary_command) # type: ignore
+
+ return remote_call()
+
+ # for other unexpected error, re-raise it.
+ raise
else:
return f(self, name, *args)
@@ -346,8 +373,11 @@ def wrapped(self: "JavaWrapper") -> Any:
except Exception:
return
- if in_remote and isinstance(self._java_obj, RemoteModelRef):
- self._java_obj.release_ref()
+ if in_remote:
+ if isinstance(self._java_obj, RemoteModelRef):
+ self._java_obj.release_ref()
+ if hasattr(self, "_remote_model_obj"):
+ self._remote_model_obj.release_ref()
return
else:
return f(self)
@@ -1076,17 +1106,32 @@ def hasSummary(self) -> bool:
Indicates whether a training summary exists for this model
instance.
"""
+ if is_remote() and "PYSPARK_NO_NAMESPACE_SHARE" not in os.environ:
+ return hasattr(self, "_summary")
return cast("JavaWrapper", self)._call_java("hasSummary")
@property
@since("2.1.0")
- @try_remote_intermediate_result
def summary(self) -> T:
"""
Gets summary of the model trained on the training set. An exception is thrown if
no summary exists.
"""
- return cast("JavaWrapper", self)._call_java("summary")
+ if is_remote() and "PYSPARK_NO_NAMESPACE_SHARE" not in os.environ:
+ if hasattr(self, "_summary"):
+ return self._summary
+ else:
+ raise RuntimeError(
+ "No training summary available for this %s" % self.__class__.__name__
+ )
+ return self._summaryCls(cast("JavaWrapper", self)._call_java("summary"))
+
+ @property
+ def _summaryCls(self) -> type:
+ raise NotImplementedError()
+
+ def _summary_dataset(self, train_dataset: "DataFrame") -> "DataFrame":
+ return self.transform(train_dataset) # type: ignore
class MetaAlgorithmReadWrite:
diff --git a/python/pyspark/pandas/base.py b/python/pyspark/pandas/base.py
index 01e23214d662d..a16a8902f9f06 100644
--- a/python/pyspark/pandas/base.py
+++ b/python/pyspark/pandas/base.py
@@ -42,6 +42,7 @@
from pyspark.pandas.spark.accessors import SparkIndexOpsMethods
from pyspark.pandas.typedef import extension_dtypes
from pyspark.pandas.utils import (
+ ansi_mode_context,
combine_frames,
same_anchor,
scol_for,
@@ -269,6 +270,14 @@ def wrapper(self: SeriesOrIndex, *args: Any) -> SeriesOrIndex:
return wrapper
+def _exclude_pd_np_operand(other: Any) -> None:
+ if isinstance(other, (pd.Series, pd.Index, pd.DataFrame, np.ndarray)):
+ raise TypeError(
+ f"Operand of type {type(other).__module__}.{type(other).__qualname__} "
+ f"is not supported for this operation. "
+ )
+
+
class IndexOpsMixin(object, metaclass=ABCMeta):
"""common ops mixin to support a unified interface / docs for Series / Index
@@ -313,16 +322,20 @@ def copy(self: IndexOpsLike) -> IndexOpsLike:
# arithmetic operators
def __neg__(self: IndexOpsLike) -> IndexOpsLike:
- return self._dtype_op.neg(self)
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ return self._dtype_op.neg(self)
def __add__(self, other: Any) -> SeriesOrIndex:
- return self._dtype_op.add(self, other)
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ return self._dtype_op.add(self, other)
def __sub__(self, other: Any) -> SeriesOrIndex:
- return self._dtype_op.sub(self, other)
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ return self._dtype_op.sub(self, other)
def __mul__(self, other: Any) -> SeriesOrIndex:
- return self._dtype_op.mul(self, other)
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ return self._dtype_op.mul(self, other)
def __truediv__(self, other: Any) -> SeriesOrIndex:
"""
@@ -342,22 +355,28 @@ def __truediv__(self, other: Any) -> SeriesOrIndex:
| -10 | null | -np.inf |
+-----------------------|---------|---------+
"""
- return self._dtype_op.truediv(self, other)
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ return self._dtype_op.truediv(self, other)
def __mod__(self, other: Any) -> SeriesOrIndex:
- return self._dtype_op.mod(self, other)
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ return self._dtype_op.mod(self, other)
def __radd__(self, other: Any) -> SeriesOrIndex:
- return self._dtype_op.radd(self, other)
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ return self._dtype_op.radd(self, other)
def __rsub__(self, other: Any) -> SeriesOrIndex:
- return self._dtype_op.rsub(self, other)
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ return self._dtype_op.rsub(self, other)
def __rmul__(self, other: Any) -> SeriesOrIndex:
- return self._dtype_op.rmul(self, other)
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ return self._dtype_op.rmul(self, other)
def __rtruediv__(self, other: Any) -> SeriesOrIndex:
- return self._dtype_op.rtruediv(self, other)
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ return self._dtype_op.rtruediv(self, other)
def __floordiv__(self, other: Any) -> SeriesOrIndex:
"""
@@ -377,68 +396,93 @@ def __floordiv__(self, other: Any) -> SeriesOrIndex:
| -10 | null | -np.inf |
+-----------------------|---------|---------+
"""
- return self._dtype_op.floordiv(self, other)
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ return self._dtype_op.floordiv(self, other)
def __rfloordiv__(self, other: Any) -> SeriesOrIndex:
- return self._dtype_op.rfloordiv(self, other)
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ return self._dtype_op.rfloordiv(self, other)
def __rmod__(self, other: Any) -> SeriesOrIndex:
- return self._dtype_op.rmod(self, other)
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ return self._dtype_op.rmod(self, other)
def __pow__(self, other: Any) -> SeriesOrIndex:
- return self._dtype_op.pow(self, other)
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ return self._dtype_op.pow(self, other)
def __rpow__(self, other: Any) -> SeriesOrIndex:
- return self._dtype_op.rpow(self, other)
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ return self._dtype_op.rpow(self, other)
def __abs__(self: IndexOpsLike) -> IndexOpsLike:
- return self._dtype_op.abs(self)
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ return self._dtype_op.abs(self)
# comparison operators
def __eq__(self, other: Any) -> SeriesOrIndex: # type: ignore[override]
# pandas always returns False for all items with dict and set.
- if isinstance(other, (dict, set)):
- return self != self
- else:
- return self._dtype_op.eq(self, other)
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ _exclude_pd_np_operand(other)
+ if isinstance(other, (dict, set)):
+ return self != self
+ else:
+ return self._dtype_op.eq(self, other)
def __ne__(self, other: Any) -> SeriesOrIndex: # type: ignore[override]
- return self._dtype_op.ne(self, other)
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ _exclude_pd_np_operand(other)
+ return self._dtype_op.ne(self, other)
def __lt__(self, other: Any) -> SeriesOrIndex:
- return self._dtype_op.lt(self, other)
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ _exclude_pd_np_operand(other)
+ return self._dtype_op.lt(self, other)
def __le__(self, other: Any) -> SeriesOrIndex:
- return self._dtype_op.le(self, other)
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ _exclude_pd_np_operand(other)
+ return self._dtype_op.le(self, other)
def __ge__(self, other: Any) -> SeriesOrIndex:
- return self._dtype_op.ge(self, other)
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ _exclude_pd_np_operand(other)
+ return self._dtype_op.ge(self, other)
def __gt__(self, other: Any) -> SeriesOrIndex:
- return self._dtype_op.gt(self, other)
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ _exclude_pd_np_operand(other)
+ return self._dtype_op.gt(self, other)
def __invert__(self: IndexOpsLike) -> IndexOpsLike:
- return self._dtype_op.invert(self)
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ return self._dtype_op.invert(self)
# `and`, `or`, `not` cannot be overloaded in Python,
# so use bitwise operators as boolean operators
def __and__(self, other: Any) -> SeriesOrIndex:
- return self._dtype_op.__and__(self, other)
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ return self._dtype_op.__and__(self, other)
def __or__(self, other: Any) -> SeriesOrIndex:
- return self._dtype_op.__or__(self, other)
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ return self._dtype_op.__or__(self, other)
def __rand__(self, other: Any) -> SeriesOrIndex:
- return self._dtype_op.rand(self, other)
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ return self._dtype_op.rand(self, other)
def __ror__(self, other: Any) -> SeriesOrIndex:
- return self._dtype_op.ror(self, other)
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ return self._dtype_op.ror(self, other)
def __xor__(self, other: Any) -> SeriesOrIndex:
- return self._dtype_op.xor(self, other)
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ return self._dtype_op.xor(self, other)
def __rxor__(self, other: Any) -> SeriesOrIndex:
- return self._dtype_op.rxor(self, other)
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ return self._dtype_op.rxor(self, other)
def __len__(self) -> int:
return len(self._psdf)
diff --git a/python/pyspark/pandas/config.py b/python/pyspark/pandas/config.py
index 290bee4340c0a..e89bcf835f8bd 100644
--- a/python/pyspark/pandas/config.py
+++ b/python/pyspark/pandas/config.py
@@ -24,7 +24,7 @@
from pyspark._globals import _NoValue, _NoValueType
from pyspark.sql.session import SparkSession
-from pyspark.pandas.utils import default_session, is_testing
+from pyspark.pandas.utils import default_session
__all__ = ["get_option", "set_option", "reset_option", "options", "option_context"]
@@ -287,7 +287,7 @@ def validate(self, v: Any) -> None:
"If False, pandas API on Spark may hit unexpected results or errors. "
"The default is False."
),
- default=is_testing(),
+ default=True,
types=bool,
),
Option(
diff --git a/python/pyspark/pandas/correlation.py b/python/pyspark/pandas/correlation.py
index da51dc2cc6126..9327b5eb71a45 100644
--- a/python/pyspark/pandas/correlation.py
+++ b/python/pyspark/pandas/correlation.py
@@ -19,7 +19,7 @@
from pyspark.sql import DataFrame as SparkDataFrame, functions as F
from pyspark.sql.window import Window
-from pyspark.pandas.utils import verify_temp_column_name
+from pyspark.pandas.utils import verify_temp_column_name, is_ansi_mode_enabled
CORRELATION_VALUE_1_COLUMN = "__correlation_value_1_input__"
@@ -60,6 +60,7 @@ def compute(sdf: SparkDataFrame, groupKeys: List[str], method: str) -> SparkData
.alias(CORRELATION_VALUE_2_COLUMN),
],
)
+ spark_session = sdf.sparkSession
if method in ["pearson", "spearman"]:
# convert values to avg ranks for spearman correlation
@@ -125,16 +126,20 @@ def compute(sdf: SparkDataFrame, groupKeys: List[str], method: str) -> SparkData
)
)
+ if is_ansi_mode_enabled(spark_session):
+ corr_expr = F.try_divide(
+ F.covar_samp(CORRELATION_VALUE_1_COLUMN, CORRELATION_VALUE_2_COLUMN),
+ F.stddev_samp(CORRELATION_VALUE_1_COLUMN)
+ * F.stddev_samp(CORRELATION_VALUE_2_COLUMN),
+ )
+ else:
+ corr_expr = F.corr(CORRELATION_VALUE_1_COLUMN, CORRELATION_VALUE_2_COLUMN)
+
sdf = sdf.groupby(groupKeys).agg(
- F.corr(CORRELATION_VALUE_1_COLUMN, CORRELATION_VALUE_2_COLUMN).alias(
- CORRELATION_CORR_OUTPUT_COLUMN
+ corr_expr.alias(CORRELATION_CORR_OUTPUT_COLUMN),
+ F.count(F.when(~F.isnull(CORRELATION_VALUE_1_COLUMN), 1)).alias(
+ CORRELATION_COUNT_OUTPUT_COLUMN
),
- F.count(
- F.when(
- ~F.isnull(CORRELATION_VALUE_1_COLUMN),
- 1,
- )
- ).alias(CORRELATION_COUNT_OUTPUT_COLUMN),
)
return sdf
@@ -219,6 +224,42 @@ def compute(sdf: SparkDataFrame, groupKeys: List[str], method: str) -> SparkData
F.col(CORRELATION_VALUE_2_COLUMN) == F.col(CORRELATION_VALUE_Y_COLUMN)
)
+ if is_ansi_mode_enabled(spark_session):
+ corr_expr = F.try_divide(
+ F.col(CORRELATION_KENDALL_P_COLUMN) - F.col(CORRELATION_KENDALL_Q_COLUMN),
+ F.sqrt(
+ (
+ F.col(CORRELATION_KENDALL_P_COLUMN)
+ + F.col(CORRELATION_KENDALL_Q_COLUMN)
+ + F.col(CORRELATION_KENDALL_T_COLUMN)
+ )
+ * (
+ F.col(CORRELATION_KENDALL_P_COLUMN)
+ + F.col(CORRELATION_KENDALL_Q_COLUMN)
+ + F.col(CORRELATION_KENDALL_U_COLUMN)
+ )
+ ),
+ )
+ else:
+ corr_expr = (
+ F.col(CORRELATION_KENDALL_P_COLUMN) - F.col(CORRELATION_KENDALL_Q_COLUMN)
+ ) / F.sqrt(
+ (
+ (
+ F.col(CORRELATION_KENDALL_P_COLUMN)
+ + F.col(CORRELATION_KENDALL_Q_COLUMN)
+ + (F.col(CORRELATION_KENDALL_T_COLUMN))
+ )
+ )
+ * (
+ (
+ F.col(CORRELATION_KENDALL_P_COLUMN)
+ + F.col(CORRELATION_KENDALL_Q_COLUMN)
+ + (F.col(CORRELATION_KENDALL_U_COLUMN))
+ )
+ )
+ )
+
sdf = (
sdf.groupby(groupKeys)
.agg(
@@ -232,26 +273,7 @@ def compute(sdf: SparkDataFrame, groupKeys: List[str], method: str) -> SparkData
).otherwise(F.lit(0))
).alias(CORRELATION_COUNT_OUTPUT_COLUMN),
)
- .withColumn(
- CORRELATION_CORR_OUTPUT_COLUMN,
- (F.col(CORRELATION_KENDALL_P_COLUMN) - F.col(CORRELATION_KENDALL_Q_COLUMN))
- / F.sqrt(
- (
- (
- F.col(CORRELATION_KENDALL_P_COLUMN)
- + F.col(CORRELATION_KENDALL_Q_COLUMN)
- + (F.col(CORRELATION_KENDALL_T_COLUMN))
- )
- )
- * (
- (
- F.col(CORRELATION_KENDALL_P_COLUMN)
- + F.col(CORRELATION_KENDALL_Q_COLUMN)
- + (F.col(CORRELATION_KENDALL_U_COLUMN))
- )
- )
- ),
- )
+ .withColumn(CORRELATION_CORR_OUTPUT_COLUMN, corr_expr)
)
sdf = sdf.select(
diff --git a/python/pyspark/pandas/data_type_ops/base.py b/python/pyspark/pandas/data_type_ops/base.py
index b4a6b1abbcaf9..bb2ef8b09aead 100644
--- a/python/pyspark/pandas/data_type_ops/base.py
+++ b/python/pyspark/pandas/data_type_ops/base.py
@@ -23,6 +23,7 @@
import numpy as np
import pandas as pd
from pandas.api.types import CategoricalDtype
+from pandas.core.dtypes.common import is_numeric_dtype
from pyspark.sql import functions as F, Column as PySparkColumn
from pyspark.sql.types import (
@@ -52,6 +53,7 @@
extension_object_dtypes_available,
spark_type_to_pandas_dtype,
)
+from pyspark.pandas.utils import is_ansi_mode_enabled
if extension_dtypes_available:
from pandas import Int8Dtype, Int16Dtype, Int32Dtype, Int64Dtype
@@ -108,6 +110,33 @@ def transform_boolean_operand_to_numeric(
return operand
+def _should_return_all_false(left: IndexOpsLike, right: Any) -> bool:
+ """
+ Determine if binary comparison should short-circuit to all False,
+ based on incompatible dtypes: non-numeric vs. numeric (including bools).
+ """
+ from pyspark.pandas.base import IndexOpsMixin
+ from pandas.api.types import is_list_like # type: ignore[attr-defined]
+
+ def are_both_numeric(left_dtype: Dtype, right_dtype: Dtype) -> bool:
+ return is_numeric_dtype(left_dtype) and is_numeric_dtype(right_dtype)
+
+ left_dtype = left.dtype
+
+ if isinstance(right, IndexOpsMixin):
+ right_dtype = right.dtype
+ elif isinstance(right, (list, tuple)):
+ right_dtype = pd.Series(right).dtype
+ else:
+ assert not is_list_like(right), (
+ "Only ps.Series, ps.Index, list, tuple, or scalar is supported as the "
+ "right-hand operand."
+ )
+ right_dtype = pd.Series([right]).dtype
+
+ return left_dtype != right_dtype and not are_both_numeric(left_dtype, right_dtype)
+
+
def _as_categorical_type(
index_ops: IndexOpsLike, dtype: CategoricalDtype, spark_type: DataType
) -> IndexOpsLike:
@@ -392,6 +421,10 @@ def ge(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
raise TypeError(">= can not be applied to %s." % self.pretty_name)
def eq(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
+ if is_ansi_mode_enabled(left._internal.spark_frame.sparkSession):
+ if _should_return_all_false(left, right):
+ return left._with_new_scol(F.lit(False)).rename(None) # type: ignore[attr-defined]
+
if isinstance(right, (list, tuple)):
from pyspark.pandas.series import first_series, scol_for
from pyspark.pandas.frame import DataFrame
diff --git a/python/pyspark/pandas/data_type_ops/boolean_ops.py b/python/pyspark/pandas/data_type_ops/boolean_ops.py
index 765ec6a946347..d8fccb9d18847 100644
--- a/python/pyspark/pandas/data_type_ops/boolean_ops.py
+++ b/python/pyspark/pandas/data_type_ops/boolean_ops.py
@@ -19,9 +19,11 @@
from typing import Any, Union
import pandas as pd
-from pandas.api.types import CategoricalDtype
+from pandas.api.types import CategoricalDtype, is_integer_dtype # type: ignore[attr-defined]
+from pandas.core.dtypes.common import is_numeric_dtype
-from pyspark.pandas.base import column_op, IndexOpsMixin, numpy_column_op
+from pyspark.pandas.base import column_op, IndexOpsMixin
+from pyspark.pandas.config import get_option
from pyspark.pandas._typing import Dtype, IndexOpsLike, SeriesOrIndex
from pyspark.pandas.data_type_ops.base import (
DataTypeOps,
@@ -137,21 +139,13 @@ def mod(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
raise TypeError(
"Modulo can not be applied to %s and the given type." % self.pretty_name
)
- spark_session = left._internal.spark_frame.sparkSession
-
- def safe_mod(left_col: PySparkColumn, right_val: Any) -> PySparkColumn:
- if is_ansi_mode_enabled(spark_session):
- return F.when(F.lit(right_val == 0), F.lit(None)).otherwise(left_col % right_val)
- else:
- return left_col % right_val
-
if isinstance(right, numbers.Number):
left = transform_boolean_operand_to_numeric(left, spark_type=as_spark_type(type(right)))
- return numpy_column_op(safe_mod)(left, right)
+ return left % right
else:
assert isinstance(right, IndexOpsMixin)
left = transform_boolean_operand_to_numeric(left, spark_type=right.spark.data_type)
- return numpy_column_op(safe_mod)(left, right)
+ return left % right
def pow(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
_sanitize_list_like(right)
@@ -235,18 +229,7 @@ def rmod(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
_sanitize_list_like(right)
if isinstance(right, numbers.Number) and not isinstance(right, bool):
left = transform_boolean_operand_to_numeric(left, spark_type=as_spark_type(type(right)))
- spark_session = left._internal.spark_frame.sparkSession
-
- if is_ansi_mode_enabled(spark_session):
-
- def safe_rmod(left_col: PySparkColumn, right_val: Any) -> PySparkColumn:
- return F.when(left_col != 0, F.pmod(F.lit(right_val), left_col)).otherwise(
- F.lit(None)
- )
-
- return numpy_column_op(safe_rmod)(left, right)
- else:
- return right % left
+ return right % left
else:
raise TypeError(
"Modulo can not be applied to %s and the given type." % self.pretty_name
@@ -339,6 +322,13 @@ def astype(self, index_ops: IndexOpsLike, dtype: Union[str, type, Dtype]) -> Ind
),
)
else:
+ is_ansi = is_ansi_mode_enabled(index_ops._internal.spark_frame.sparkSession)
+ if is_ansi and get_option("compute.eager_check"):
+ if is_integer_dtype(dtype) and not isinstance(dtype, extension_dtypes):
+ if index_ops.hasnans:
+ raise ValueError(
+ "Cannot convert %s with missing values to integer" % self.pretty_name
+ )
return _as_other_type(index_ops, dtype, spark_type)
def neg(self, operand: IndexOpsLike) -> IndexOpsLike:
@@ -347,6 +337,24 @@ def neg(self, operand: IndexOpsLike) -> IndexOpsLike:
def abs(self, operand: IndexOpsLike) -> IndexOpsLike:
return operand
+ def eq(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
+ if is_ansi_mode_enabled(left._internal.spark_frame.sparkSession):
+ # Handle bool vs. non-bool numeric comparisons
+ left_is_bool = _is_boolean_type(left)
+ right_is_non_bool_numeric = is_numeric_dtype(right) and not _is_boolean_type(right)
+
+ if left_is_bool and right_is_non_bool_numeric:
+ if isinstance(right, numbers.Number):
+ left = transform_boolean_operand_to_numeric(
+ left, spark_type=as_spark_type(type(right))
+ )
+ else:
+ left = transform_boolean_operand_to_numeric(
+ left, spark_type=right.spark.data_type
+ )
+
+ return super().eq(left, right)
+
def lt(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
_sanitize_list_like(right)
return column_op(PySparkColumn.__lt__)(left, right)
diff --git a/python/pyspark/pandas/data_type_ops/num_ops.py b/python/pyspark/pandas/data_type_ops/num_ops.py
index 34d313af8232e..940eb0ef40af2 100644
--- a/python/pyspark/pandas/data_type_ops/num_ops.py
+++ b/python/pyspark/pandas/data_type_ops/num_ops.py
@@ -16,13 +16,15 @@
#
import numbers
-from typing import Any, Union
+from typing import Any, Union, Callable
import numpy as np
import pandas as pd
from pandas.api.types import ( # type: ignore[attr-defined]
is_bool_dtype,
is_integer_dtype,
+ is_float_dtype,
+ is_numeric_dtype,
CategoricalDtype,
is_list_like,
)
@@ -41,8 +43,9 @@
_sanitize_list_like,
_is_valid_for_logical_operator,
_is_boolean_type,
+ _should_return_all_false,
)
-from pyspark.pandas.typedef.typehints import extension_dtypes, pandas_on_spark_type
+from pyspark.pandas.typedef.typehints import extension_dtypes, pandas_on_spark_type, as_spark_type
from pyspark.pandas.utils import is_ansi_mode_enabled
from pyspark.sql import functions as F, Column as PySparkColumn
from pyspark.sql.types import (
@@ -69,6 +72,27 @@ def _non_fractional_astype(
return _as_other_type(index_ops, dtype, spark_type)
+def _cast_back_float(
+ expr: PySparkColumn, left_dtype: Union[str, type, Dtype], right: Any
+) -> PySparkColumn:
+ """
+ Cast the result expression back to the original float dtype if needed.
+
+ This function ensures pandas on Spark matches pandas behavior when performing
+ arithmetic operations involving float32 and numeric values. In such cases, under ANSI mode,
+ Spark implicitly widen float32 to float64, when the other operand is a numeric type
+ but not float32 (e.g., int, bool), which deviates from pandas behavior where the result
+ retains float32.
+ """
+ is_left_float = is_float_dtype(left_dtype)
+ is_right_numeric = isinstance(right, (int, float, bool)) or (
+ hasattr(right, "dtype") and is_numeric_dtype(right.dtype)
+ )
+ if is_left_float and is_right_numeric:
+ return expr.cast(as_spark_type(left_dtype))
+ return expr
+
+
class NumericOps(DataTypeOps):
"""The class for binary operations of numeric pandas-on-Spark objects."""
@@ -80,28 +104,51 @@ def add(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
_sanitize_list_like(right)
if not is_valid_operand_for_numeric_arithmetic(right):
raise TypeError("Addition can not be applied to given types.")
+ spark_session = left._internal.spark_frame.sparkSession
+ new_right = transform_boolean_operand_to_numeric(right, spark_type=left.spark.data_type)
- right = transform_boolean_operand_to_numeric(right, spark_type=left.spark.data_type)
- return column_op(PySparkColumn.__add__)(left, right)
+ def wrapped_add(lc: PySparkColumn, rc: Any) -> PySparkColumn:
+ expr = PySparkColumn.__add__(lc, rc)
+ if is_ansi_mode_enabled(spark_session):
+ expr = _cast_back_float(expr, left.dtype, right)
+ return expr
+
+ return column_op(wrapped_add)(left, new_right)
def sub(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
_sanitize_list_like(right)
if not is_valid_operand_for_numeric_arithmetic(right):
raise TypeError("Subtraction can not be applied to given types.")
+ spark_session = left._internal.spark_frame.sparkSession
+ new_right = transform_boolean_operand_to_numeric(right, spark_type=left.spark.data_type)
- right = transform_boolean_operand_to_numeric(right, spark_type=left.spark.data_type)
- return column_op(PySparkColumn.__sub__)(left, right)
+ def wrapped_sub(lc: PySparkColumn, rc: Any) -> PySparkColumn:
+ expr = PySparkColumn.__sub__(lc, rc)
+ if is_ansi_mode_enabled(spark_session):
+ expr = _cast_back_float(expr, left.dtype, right)
+ return expr
+
+ return column_op(wrapped_sub)(left, new_right)
def mod(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
_sanitize_list_like(right)
if not is_valid_operand_for_numeric_arithmetic(right):
raise TypeError("Modulo can not be applied to given types.")
+ spark_session = left._internal.spark_frame.sparkSession
- def mod(left: PySparkColumn, right: Any) -> PySparkColumn:
- return ((left % right) + right) % right
+ def mod(left_op: PySparkColumn, right_op: Any) -> PySparkColumn:
+ if is_ansi_mode_enabled(spark_session):
+ expr = F.when(F.lit(right_op == 0), F.lit(None)).otherwise(
+ ((left_op % right_op) + right_op) % right_op
+ )
+ expr = _cast_back_float(expr, left.dtype, right)
+ else:
+ expr = ((left_op % right_op) + right_op) % right_op
+ return expr
- right = transform_boolean_operand_to_numeric(right, spark_type=left.spark.data_type)
- return column_op(mod)(left, right)
+ new_right = transform_boolean_operand_to_numeric(right, spark_type=left.spark.data_type)
+
+ return column_op(mod)(left, new_right)
def pow(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
_sanitize_list_like(right)
@@ -122,22 +169,46 @@ def radd(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
_sanitize_list_like(right)
if not isinstance(right, numbers.Number):
raise TypeError("Addition can not be applied to given types.")
- right = transform_boolean_operand_to_numeric(right)
- return column_op(PySparkColumn.__radd__)(left, right)
+ spark_session = left._internal.spark_frame.sparkSession
+ new_right = transform_boolean_operand_to_numeric(right)
+
+ def wrapped_radd(lc: PySparkColumn, rc: Any) -> PySparkColumn:
+ expr = PySparkColumn.__radd__(lc, rc)
+ if is_ansi_mode_enabled(spark_session):
+ expr = _cast_back_float(expr, left.dtype, right)
+ return expr
+
+ return column_op(wrapped_radd)(left, new_right)
def rsub(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
_sanitize_list_like(right)
if not isinstance(right, numbers.Number):
raise TypeError("Subtraction can not be applied to given types.")
- right = transform_boolean_operand_to_numeric(right)
- return column_op(PySparkColumn.__rsub__)(left, right)
+ spark_session = left._internal.spark_frame.sparkSession
+ new_right = transform_boolean_operand_to_numeric(right)
+
+ def wrapped_rsub(lc: PySparkColumn, rc: Any) -> PySparkColumn:
+ expr = PySparkColumn.__rsub__(lc, rc)
+ if is_ansi_mode_enabled(spark_session):
+ expr = _cast_back_float(expr, left.dtype, right)
+ return expr
+
+ return column_op(wrapped_rsub)(left, new_right)
def rmul(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
_sanitize_list_like(right)
if not isinstance(right, numbers.Number):
raise TypeError("Multiplication can not be applied to given types.")
- right = transform_boolean_operand_to_numeric(right)
- return column_op(PySparkColumn.__rmul__)(left, right)
+ spark_session = left._internal.spark_frame.sparkSession
+ new_right = transform_boolean_operand_to_numeric(right)
+
+ def wrapped_rmul(lc: PySparkColumn, rc: Any) -> PySparkColumn:
+ expr = PySparkColumn.__mul__(lc, rc)
+ if is_ansi_mode_enabled(spark_session):
+ expr = _cast_back_float(expr, left.dtype, right)
+ return expr
+
+ return column_op(wrapped_rmul)(left, new_right)
def rpow(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
_sanitize_list_like(right)
@@ -154,12 +225,22 @@ def rmod(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
_sanitize_list_like(right)
if not isinstance(right, numbers.Number):
raise TypeError("Modulo can not be applied to given types.")
+ spark_session = left._internal.spark_frame.sparkSession
- def rmod(left: PySparkColumn, right: Any) -> PySparkColumn:
- return ((right % left) + left) % left
+ new_right = transform_boolean_operand_to_numeric(right)
- right = transform_boolean_operand_to_numeric(right)
- return column_op(rmod)(left, right)
+ def safe_rmod(left_op: PySparkColumn, right_op: Any) -> PySparkColumn:
+ if is_ansi_mode_enabled(spark_session):
+ # Java-style modulo -> Python-style modulo
+ result = F.when(
+ left_op != 0, ((F.lit(right_op) % left_op) + left_op) % left_op
+ ).otherwise(F.lit(None))
+ result = _cast_back_float(result, left.dtype, right)
+ return result
+ else:
+ return ((right_op % left_op) + left_op) % left_op
+
+ return column_op(safe_rmod)(left, new_right)
def neg(self, operand: IndexOpsLike) -> IndexOpsLike:
return operand._with_new_scol(-operand.spark.column, field=operand._internal.data_fields[0])
@@ -173,7 +254,16 @@ def eq(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
# We can directly use `super().eq` when given object is list, tuple, dict or set.
if not isinstance(right, IndexOpsMixin) and is_list_like(right):
return super().eq(left, right)
- return pyspark_column_op("__eq__", left, right, fillna=False)
+ else:
+ if is_ansi_mode_enabled(left._internal.spark_frame.sparkSession):
+ if _should_return_all_false(left, right):
+ left_scol = left._with_new_scol(F.lit(False))
+ return left_scol.rename(None) # type: ignore[attr-defined]
+ if _is_boolean_type(right): # numeric vs. bool
+ right = transform_boolean_operand_to_numeric(
+ right, spark_type=left.spark.data_type
+ )
+ return pyspark_column_op("__eq__", left, right, fillna=False)
def ne(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
_sanitize_list_like(right)
@@ -271,13 +361,22 @@ def floordiv(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
_sanitize_list_like(right)
if not is_valid_operand_for_numeric_arithmetic(right):
raise TypeError("Floor division can not be applied to given types.")
+ spark_session = left._internal.spark_frame.sparkSession
+ use_try_divide = is_ansi_mode_enabled(spark_session)
+
+ def fallback_div(x: PySparkColumn, y: PySparkColumn) -> PySparkColumn:
+ return x.__div__(y)
+
+ safe_div: Callable[[PySparkColumn, PySparkColumn], PySparkColumn] = (
+ F.try_divide if use_try_divide else fallback_div
+ )
def floordiv(left: PySparkColumn, right: Any) -> PySparkColumn:
return F.when(F.lit(right is np.nan), np.nan).otherwise(
F.when(
F.lit(right != 0) | F.lit(right).isNull(),
F.floor(left.__div__(right)),
- ).otherwise(F.lit(np.inf).__div__(left))
+ ).otherwise(safe_div(F.lit(np.inf), left))
)
right = transform_boolean_operand_to_numeric(right, spark_type=left.spark.data_type)
@@ -334,56 +433,76 @@ def mul(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
if not is_valid_operand_for_numeric_arithmetic(right):
raise TypeError("Multiplication can not be applied to given types.")
- right = transform_boolean_operand_to_numeric(right, spark_type=left.spark.data_type)
+ spark_session = left._internal.spark_frame.sparkSession
+ new_right = transform_boolean_operand_to_numeric(right, spark_type=left.spark.data_type)
- return column_op(PySparkColumn.__mul__)(left, right)
+ def wrapped_mul(lc: PySparkColumn, rc: Any) -> PySparkColumn:
+ expr = PySparkColumn.__mul__(lc, rc)
+ if is_ansi_mode_enabled(spark_session):
+ expr = _cast_back_float(expr, left.dtype, right)
+ return expr
+
+ return column_op(wrapped_mul)(left, new_right)
def truediv(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
_sanitize_list_like(right)
if not is_valid_operand_for_numeric_arithmetic(right):
raise TypeError("True division can not be applied to given types.")
spark_session = left._internal.spark_frame.sparkSession
- right = transform_boolean_operand_to_numeric(right, spark_type=left.spark.data_type)
+ new_right = transform_boolean_operand_to_numeric(right, spark_type=left.spark.data_type)
+ left_dtype = left.dtype
- def truediv(left: PySparkColumn, right: Any) -> PySparkColumn:
+ def truediv(lc: PySparkColumn, rc: Any) -> PySparkColumn:
if is_ansi_mode_enabled(spark_session):
- return F.when(
- F.lit(right == 0),
- F.when(left < 0, F.lit(float("-inf")))
- .when(left > 0, F.lit(float("inf")))
+ expr = F.when(
+ F.lit(rc == 0),
+ F.when(lc < 0, F.lit(float("-inf")))
+ .when(lc > 0, F.lit(float("inf")))
.otherwise(F.lit(np.nan)),
- ).otherwise(left / right)
+ ).otherwise(lc / rc)
else:
- return F.when(
- F.lit(right != 0) | F.lit(right).isNull(),
- left.__div__(right),
+ expr = F.when(
+ F.lit(rc != 0) | F.lit(rc).isNull(),
+ lc.__div__(rc),
).otherwise(
- F.when(F.lit(left == np.inf) | F.lit(left == -np.inf), left).otherwise(
- F.lit(np.inf).__div__(left)
+ F.when(F.lit(lc == np.inf) | F.lit(lc == -np.inf), lc).otherwise(
+ F.lit(np.inf).__div__(lc)
)
)
+ return _cast_back_float(expr, left_dtype, right)
- return numpy_column_op(truediv)(left, right)
+ return numpy_column_op(truediv)(left, new_right)
def floordiv(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
_sanitize_list_like(right)
if not is_valid_operand_for_numeric_arithmetic(right):
raise TypeError("Floor division can not be applied to given types.")
+ spark_session = left._internal.spark_frame.sparkSession
+ use_try_divide = is_ansi_mode_enabled(spark_session)
+ left_dtype = left.dtype
- def floordiv(left: PySparkColumn, right: Any) -> PySparkColumn:
- return F.when(F.lit(right is np.nan), np.nan).otherwise(
+ def fallback_div(x: PySparkColumn, y: PySparkColumn) -> PySparkColumn:
+ return x.__div__(y)
+
+ safe_div: Callable[[PySparkColumn, PySparkColumn], PySparkColumn] = (
+ F.try_divide if use_try_divide else fallback_div
+ )
+
+ def floordiv(lc: PySparkColumn, rc: Any) -> PySparkColumn:
+ expr = F.when(F.lit(rc is np.nan), np.nan).otherwise(
F.when(
- F.lit(right != 0) | F.lit(right).isNull(),
- F.floor(left.__div__(right)),
+ F.lit(rc != 0) | F.lit(rc).isNull(),
+ F.floor(lc.__div__(rc)),
).otherwise(
- F.when(F.lit(left == np.inf) | F.lit(left == -np.inf), left).otherwise(
- F.lit(np.inf).__div__(left)
+ F.when(F.lit(lc == np.inf) | F.lit(lc == -np.inf), lc).otherwise(
+ safe_div(F.lit(np.inf), lc)
)
)
)
+ return _cast_back_float(expr, left_dtype, right)
- right = transform_boolean_operand_to_numeric(right, spark_type=left.spark.data_type)
- return numpy_column_op(floordiv)(left, right)
+ new_right = transform_boolean_operand_to_numeric(right, spark_type=left.spark.data_type)
+ return numpy_column_op(floordiv)(left, new_right)
def rtruediv(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
_sanitize_list_like(right)
diff --git a/python/pyspark/pandas/frame.py b/python/pyspark/pandas/frame.py
index a6f737d0d01bf..745551e8b8f9b 100644
--- a/python/pyspark/pandas/frame.py
+++ b/python/pyspark/pandas/frame.py
@@ -25,7 +25,7 @@
import inspect
import json
import types
-from functools import partial, reduce
+from functools import partial, reduce, wraps
import sys
from itertools import zip_longest, chain
from types import TracebackType
@@ -42,6 +42,7 @@
Sequence,
Tuple,
Type,
+ TypeVar,
Union,
cast,
no_type_check,
@@ -109,9 +110,11 @@
from pyspark.pandas.spark.accessors import SparkFrameMethods, CachedSparkFrameMethods
from pyspark.pandas.utils import (
align_diff_frames,
+ ansi_mode_context,
column_labels_level,
combine_frames,
default_session,
+ is_ansi_mode_enabled,
is_name_like_tuple,
is_name_like_value,
is_testing,
@@ -168,6 +171,18 @@
)
+FuncT = TypeVar("FuncT", bound=Callable[..., Any])
+
+
+def with_ansi_mode_context(f: FuncT) -> FuncT:
+ @wraps(f)
+ def _with_ansi_mode_context(self: "DataFrame", *args: Any, **kwargs: Any) -> Any:
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ return f(self, *args, **kwargs)
+
+ return cast(FuncT, _with_ansi_mode_context)
+
+
_flex_doc_FRAME = """
Get {desc} of dataframe and other, element-wise (binary operator `{op_name}`).
@@ -739,6 +754,7 @@ def axes(self) -> List:
"""
return [self.index, self.columns]
+ @with_ansi_mode_context
def _reduce_for_stat_function(
self,
sfun: Callable[["Series"], PySparkColumn],
@@ -869,6 +885,7 @@ def _psser_for(self, label: Label) -> "Series":
"""
return self._pssers[label]
+ @with_ansi_mode_context
def _apply_series_op(
self,
op: Callable[["Series"], Union["Series", PySparkColumn]],
@@ -883,6 +900,7 @@ def _apply_series_op(
return DataFrame(internal)
# Arithmetic Operators
+ @with_ansi_mode_context
def _map_series_op(self, op: str, other: Any) -> "DataFrame":
from pyspark.pandas.base import IndexOpsMixin
@@ -1506,6 +1524,7 @@ def aggregate(self, func: Union[List[str], Dict[Name, List[str]]]) -> "DataFrame
agg = aggregate
+ @with_ansi_mode_context
def corr(self, method: str = "pearson", min_periods: Optional[int] = None) -> "DataFrame":
"""
Compute pairwise correlation of columns, excluding NA/null values.
@@ -1725,6 +1744,7 @@ def corr(self, method: str = "pearson", min_periods: Optional[int] = None) -> "D
)
)
+ @with_ansi_mode_context
def corrwith(
self, other: DataFrameOrSeries, axis: Axis = 0, drop: bool = False, method: str = "pearson"
) -> "Series":
@@ -8383,6 +8403,7 @@ def nsmallest(
by_scols = self._prepare_sort_by_scols(columns)
return self._sort(by=by_scols, ascending=True, na_position="last", keep=keep).head(n=n)
+ @with_ansi_mode_context
def isin(self, values: Union[List, Dict]) -> "DataFrame":
"""
Whether each element in the DataFrame is contained in values.
@@ -8456,7 +8477,13 @@ def isin(self, values: Union[List, Dict]) -> "DataFrame":
)
for label in self._internal.column_labels:
- scol = self._internal.spark_column_for(label).isin([F.lit(v) for v in values])
+ if is_ansi_mode_enabled(self._internal.spark_frame.sparkSession):
+ col_type = self._internal.spark_type_for(label)
+ scol = self._internal.spark_column_for(label).isin(
+ [F.lit(v).try_cast(col_type) for v in values]
+ )
+ else:
+ scol = self._internal.spark_column_for(label).isin([F.lit(v) for v in values])
scol = F.coalesce(scol, F.lit(False))
data_spark_columns.append(scol.alias(self._internal.spark_column_name_for(label)))
else:
@@ -10425,6 +10452,7 @@ def reindex_like(self, other: "DataFrame", copy: bool = True) -> "DataFrame":
else:
raise TypeError("other must be a pandas-on-Spark DataFrame")
+ @with_ansi_mode_context
def melt(
self,
id_vars: Optional[Union[Name, List[Name]]] = None,
@@ -10610,12 +10638,35 @@ def melt(
else:
var_name = [var_name] # type: ignore[list-item]
+ use_cast = is_ansi_mode_enabled(self._internal.spark_frame.sparkSession)
+ string_cast_required_type = None
+ if use_cast:
+ field_by_label = {
+ label: field
+ for label, field in zip(self._internal.column_labels, self._internal.data_fields)
+ }
+
+ value_col_types = [field_by_label[label].spark_type for label in value_vars]
+ # If any value column is of StringType, cast all value columns to StringType to avoid
+ # ANSI mode errors during explode - mixing strings and integers.
+ string_cast_required_type = (
+ StringType() if any(isinstance(t, StringType) for t in value_col_types) else None
+ )
+
pairs = F.explode(
F.array(
*[
F.struct(
*[F.lit(c).alias(name) for c, name in zip(label, var_name)],
- *[self._internal.spark_column_for(label).alias(value_name)],
+ *[
+ (
+ self._internal.spark_column_for(label).cast(
+ string_cast_required_type
+ )
+ if use_cast and string_cast_required_type is not None
+ else self._internal.spark_column_for(label)
+ ).alias(value_name)
+ ],
)
for label in column_labels
if label in value_vars
@@ -13778,32 +13829,12 @@ def _test() -> None:
import uuid
from pyspark.sql import SparkSession
import pyspark.pandas.frame
- from pyspark.testing.utils import is_ansi_mode_test
os.chdir(os.environ["SPARK_HOME"])
globs = pyspark.pandas.frame.__dict__.copy()
globs["ps"] = pyspark.pandas
- if is_ansi_mode_test:
- del pyspark.pandas.frame.DataFrame.add.__doc__
- del pyspark.pandas.frame.DataFrame.div.__doc__
- del pyspark.pandas.frame.DataFrame.floordiv.__doc__
- del pyspark.pandas.frame.DataFrame.melt.__doc__
- del pyspark.pandas.frame.DataFrame.mod.__doc__
- del pyspark.pandas.frame.DataFrame.mul.__doc__
- del pyspark.pandas.frame.DataFrame.pow.__doc__
- del pyspark.pandas.frame.DataFrame.radd.__doc__
- del pyspark.pandas.frame.DataFrame.rdiv.__doc__
- del pyspark.pandas.frame.DataFrame.rfloordiv.__doc__
- del pyspark.pandas.frame.DataFrame.rmod.__doc__
- del pyspark.pandas.frame.DataFrame.rmul.__doc__
- del pyspark.pandas.frame.DataFrame.rpow.__doc__
- del pyspark.pandas.frame.DataFrame.rsub.__doc__
- del pyspark.pandas.frame.DataFrame.rtruediv.__doc__
- del pyspark.pandas.frame.DataFrame.sub.__doc__
- del pyspark.pandas.frame.DataFrame.truediv.__doc__
-
spark = (
SparkSession.builder.master("local[4]").appName("pyspark.pandas.frame tests").getOrCreate()
)
diff --git a/python/pyspark/pandas/generic.py b/python/pyspark/pandas/generic.py
index 1244ee2d88aac..dd9cc311f2742 100644
--- a/python/pyspark/pandas/generic.py
+++ b/python/pyspark/pandas/generic.py
@@ -156,7 +156,7 @@ def cummin(self: FrameLike, skipna: bool = True) -> FrameLike:
Returns a DataFrame or Series of the same size containing the cumulative minimum.
.. note:: the current implementation of cummin uses Spark's Window without
- specifying partition specification. This leads to moveing all data into a
+ specifying partition specification. This leads to moving all data into a
single partition in a single machine and could cause serious
performance degradation. Avoid this method with very large datasets.
@@ -216,7 +216,7 @@ def cummax(self: FrameLike, skipna: bool = True) -> FrameLike:
Returns a DataFrame or Series of the same size containing the cumulative maximum.
.. note:: the current implementation of cummax uses Spark's Window without
- specifying partition specification. This leads to moveing all data into a
+ specifying partition specification. This leads to moving all data into a
single partition in a single machine and could cause serious
performance degradation. Avoid this method with very large datasets.
@@ -277,7 +277,7 @@ def cumsum(self: FrameLike, skipna: bool = True) -> FrameLike:
Returns a DataFrame or Series of the same size containing the cumulative sum.
.. note:: the current implementation of cumsum uses Spark's Window without
- specifying partition specification. This leads to moveing all data into a
+ specifying partition specification. This leads to moving all data into a
single partition in a single machine and could cause serious
performance degradation. Avoid this method with very large datasets.
@@ -338,7 +338,7 @@ def cumprod(self: FrameLike, skipna: bool = True) -> FrameLike:
Returns a DataFrame or Series of the same size containing the cumulative product.
.. note:: the current implementation of cumprod uses Spark's Window without
- specifying partition specification. This leads to moveing all data into a
+ specifying partition specification. This leads to moving all data into a
single partition in a single machine and could cause serious
performance degradation. Avoid this method with very large datasets.
@@ -3330,7 +3330,7 @@ def bfill(
Synonym for `DataFrame.fillna()` or `Series.fillna()` with ``method=`bfill```.
.. note:: the current implementation of 'bfill' uses Spark's Window
- without specifying partition specification. This leads to moveing all data into a
+ without specifying partition specification. This leads to moving all data into a
single partition in a single machine and could cause serious
performance degradation. Avoid this method with very large datasets.
@@ -3409,7 +3409,7 @@ def ffill(
Synonym for `DataFrame.fillna()` or `Series.fillna()` with ``method=`ffill```.
.. note:: the current implementation of 'ffill' uses Spark's Window
- without specifying partition specification. This leads to moveing all data into a
+ without specifying partition specification. This leads to moving all data into a
single a partition in a single machine and could cause serious
performance degradation. Avoid this method with very large datasets.
@@ -3489,7 +3489,7 @@ def interpolate(
Fill NaN values using an interpolation method.
.. note:: the current implementation of interpolate uses Spark's Window without
- specifying partition specification. This leads to moveing all data into a
+ specifying partition specification. This leads to moving all data into a
single partition in a single machine and could cause serious
performance degradation. Avoid this method with very large datasets.
diff --git a/python/pyspark/pandas/groupby.py b/python/pyspark/pandas/groupby.py
index 93cbf6ff0a2c0..24a6dbe2e40f2 100644
--- a/python/pyspark/pandas/groupby.py
+++ b/python/pyspark/pandas/groupby.py
@@ -21,7 +21,7 @@
from abc import ABCMeta, abstractmethod
import inspect
from collections import defaultdict, namedtuple
-from functools import partial
+from functools import partial, wraps
from itertools import product
from typing import (
Any,
@@ -36,6 +36,7 @@
Set,
Tuple,
Type,
+ TypeVar,
Union,
cast,
TYPE_CHECKING,
@@ -85,6 +86,7 @@
)
from pyspark.pandas.utils import (
align_diff_frames,
+ ansi_mode_context,
is_name_like_tuple,
is_name_like_value,
name_like_string,
@@ -100,6 +102,18 @@
from pyspark.pandas.window import RollingGroupby, ExpandingGroupby, ExponentialMovingGroupby
+FuncT = TypeVar("FuncT", bound=Callable[..., Any])
+
+
+def with_ansi_mode_context(f: FuncT) -> FuncT:
+ @wraps(f)
+ def _with_ansi_mode_context(self: "GroupBy", *args: Any, **kwargs: Any) -> Any:
+ with ansi_mode_context(self._psdf._internal.spark_frame.sparkSession):
+ return f(self, *args, **kwargs)
+
+ return cast(FuncT, _with_ansi_mode_context)
+
+
# to keep it the same as pandas
NamedAgg = namedtuple("NamedAgg", ["column", "aggfunc"])
@@ -3940,6 +3954,7 @@ def describe(self) -> DataFrame:
# Cast columns to ``"float64"`` to match `pandas.DataFrame.groupby`.
return DataFrame(internal).astype("float64")
+ @with_ansi_mode_context
def corr(
self,
method: str = "pearson",
@@ -4595,7 +4610,6 @@ def _test() -> None:
import numpy
from pyspark.sql import SparkSession
import pyspark.pandas.groupby
- from pyspark.testing.utils import is_ansi_mode_test
os.chdir(os.environ["SPARK_HOME"])
@@ -4603,9 +4617,6 @@ def _test() -> None:
globs["np"] = numpy
globs["ps"] = pyspark.pandas
- if is_ansi_mode_test:
- del pyspark.pandas.groupby.DataFrameGroupBy.corr.__doc__
-
spark = (
SparkSession.builder.master("local[4]")
.appName("pyspark.pandas.groupby tests")
diff --git a/python/pyspark/pandas/indexes/base.py b/python/pyspark/pandas/indexes/base.py
index 2627baac8bc6f..17cb3bf3f74a7 100644
--- a/python/pyspark/pandas/indexes/base.py
+++ b/python/pyspark/pandas/indexes/base.py
@@ -62,6 +62,7 @@
from pyspark.pandas.series import Series, first_series
from pyspark.pandas.spark.accessors import SparkIndexMethods
from pyspark.pandas.utils import (
+ is_ansi_mode_enabled,
is_name_like_tuple,
is_name_like_value,
name_like_string,
@@ -922,13 +923,17 @@ def to_series(self, name: Optional[Name] = None) -> Series:
return result
else:
# MultiIndex
- def struct_to_array(scol: Column) -> Column:
- field_names = result._internal.spark_type_for(
- scol
- ).fieldNames() # type: ignore[attr-defined]
- return F.array([scol[field] for field in field_names])
+ if is_ansi_mode_enabled(self._internal.spark_frame.sparkSession):
+ return result
+ else:
+
+ def struct_to_array(scol: Column) -> Column:
+ field_names = result._internal.spark_type_for(
+ scol
+ ).fieldNames() # type: ignore[attr-defined]
+ return F.array([scol[field] for field in field_names])
- return result.spark.transform(struct_to_array)
+ return result.spark.transform(struct_to_array)
def to_frame(self, index: bool = True, name: Optional[Name] = None) -> DataFrame:
"""
diff --git a/python/pyspark/pandas/internal.py b/python/pyspark/pandas/internal.py
index 3f6831b600678..5739c66f1f421 100644
--- a/python/pyspark/pandas/internal.py
+++ b/python/pyspark/pandas/internal.py
@@ -1200,10 +1200,10 @@ def with_new_sdf(
:param spark_frame: the new Spark DataFrame
:param index_fields: the new InternalFields for the index columns.
- If None, the original dtyeps are used.
+ If None, the original dtypes are used.
:param data_columns: the new column names. If None, the original one is used.
:param data_fields: the new InternalFields for the data columns.
- If None, the original dtyeps are used.
+ If None, the original dtypes are used.
:return: the copied InternalFrame.
"""
if index_fields is None:
diff --git a/python/pyspark/pandas/namespace.py b/python/pyspark/pandas/namespace.py
index 0073c3851065a..a5f2b5ee4f6aa 100644
--- a/python/pyspark/pandas/namespace.py
+++ b/python/pyspark/pandas/namespace.py
@@ -76,6 +76,7 @@
from pyspark.pandas.utils import (
align_diff_frames,
default_session,
+ is_ansi_mode_enabled,
is_name_like_tuple,
is_name_like_value,
name_like_string,
@@ -3630,7 +3631,11 @@ def to_numeric(arg, errors="raise"):
"""
if isinstance(arg, Series):
if errors == "coerce":
- return arg._with_new_scol(arg.spark.column.cast("float"))
+ spark_session = arg._internal.spark_frame.sparkSession
+ if is_ansi_mode_enabled(spark_session):
+ return arg._with_new_scol(arg.spark.column.try_cast("float"))
+ else:
+ return arg._with_new_scol(arg.spark.column.cast("float"))
elif errors == "raise":
scol = arg.spark.column
scol_casted = scol.cast("float")
@@ -3874,7 +3879,6 @@ def _test() -> None:
from pyspark.sql import SparkSession
import pyspark.pandas.namespace
from pandas.util.version import Version
- from pyspark.testing.utils import is_ansi_mode_test
os.chdir(os.environ["SPARK_HOME"])
@@ -3888,10 +3892,6 @@ def _test() -> None:
globs["ps"] = pyspark.pandas
globs["sf"] = F
- if is_ansi_mode_test:
- del pyspark.pandas.namespace.melt.__doc__
- del pyspark.pandas.namespace.to_numeric.__doc__
-
spark = (
SparkSession.builder.master("local[4]")
.appName("pyspark.pandas.namespace tests")
diff --git a/python/pyspark/pandas/series.py b/python/pyspark/pandas/series.py
index f7f8c89ab2783..e5c8890b646c6 100644
--- a/python/pyspark/pandas/series.py
+++ b/python/pyspark/pandas/series.py
@@ -23,7 +23,7 @@
import inspect
import warnings
from collections.abc import Mapping
-from functools import partial, reduce
+from functools import partial, reduce, wraps
from typing import (
Any,
Callable,
@@ -36,6 +36,7 @@
Sequence,
Tuple,
Type,
+ TypeVar,
Union,
cast,
no_type_check,
@@ -103,7 +104,9 @@
from pyspark.pandas.missing.series import MissingPandasLikeSeries
from pyspark.pandas.plot import PandasOnSparkPlotAccessor
from pyspark.pandas.utils import (
+ ansi_mode_context,
combine_frames,
+ is_ansi_mode_enabled,
is_name_like_tuple,
is_name_like_value,
name_like_string,
@@ -142,6 +145,18 @@
# This pattern basically seeks the footer string from pandas'
REPR_PATTERN = re.compile(r"Length: (?P[0-9]+)")
+FuncT = TypeVar("FuncT", bound=Callable[..., Any])
+
+
+def with_ansi_mode_context(f: FuncT) -> FuncT:
+ @wraps(f)
+ def _with_ansi_mode_context(self: "Series", *args: Any, **kwargs: Any) -> Any:
+ with ansi_mode_context(self._internal.spark_frame.sparkSession):
+ return f(self, *args, **kwargs)
+
+ return cast(FuncT, _with_ansi_mode_context)
+
+
_flex_doc_SERIES = """
Return {desc} of series and other, element-wise (binary operator `{op_name}`).
@@ -369,6 +384,10 @@ class Series(Frame, IndexOpsMixin, Generic[T]):
pandas-on-Spark Series that corresponds to pandas Series logically. This holds Spark Column
internally.
+ .. versionchanged:: 4.1.0
+ Support construction from a pandas-on-Spark Series input, which can be used with
+ additional parameters index, dtype, and name for overriding the original value.
+
:ivar _internal: an internal immutable Frame to manage metadata.
:type _internal: InternalFrame
:ivar _psdf: Parent's pandas-on-Spark DataFrame
@@ -376,9 +395,10 @@ class Series(Frame, IndexOpsMixin, Generic[T]):
Parameters
----------
- data : array-like, dict, or scalar value, pandas Series
+ data : array-like, dict, or scalar value, pandas Series, pandas-on-Spark Series
Contains data stored in Series
- Note that if `data` is a pandas Series, other arguments should not be used.
+ Note that if `data` is a Series, index, dtype, or name can also be
+ specified to override the original value.
index : array-like or Index (1d)
Values must be hashable and have the same length as `data`.
Non-unique index values are allowed. Will default to
@@ -387,6 +407,8 @@ class Series(Frame, IndexOpsMixin, Generic[T]):
dict.
dtype : numpy.dtype or None
If None, dtype will be inferred
+ name : str, default None
+ The name to give to the Series.
copy : boolean, default False
Copy input data
"""
@@ -406,6 +428,24 @@ def __init__( # type: ignore[no-untyped-def]
self._anchor = data
self._col_label = index
+
+ elif isinstance(data, Series):
+ assert not copy
+ assert not fastpath
+
+ if name:
+ data = data.rename(name)
+
+ if index:
+ data = data.reindex(index)
+
+ if dtype:
+ data = data.astype(dtype)
+
+ anchor = DataFrame(data)
+ self._anchor = anchor
+ self._col_label = anchor._internal.column_labels[0]
+ object.__setattr__(anchor, "_psseries", {self._column_label: self})
else:
if isinstance(data, pd.Series):
assert index is None
@@ -2061,7 +2101,7 @@ def fillna(
"""Fill NA/NaN values.
.. note:: the current implementation of 'method' parameter in fillna uses Spark's Window
- without specifying partition specification. This leads to moveing all data into
+ without specifying partition specification. This leads to moving all data into
a single partition in a single machine and could cause serious
performance degradation. Avoid this method with very large datasets.
@@ -3310,7 +3350,7 @@ def autocorr(self, lag: int = 1) -> float:
the Series and its shifted self.
.. note:: the current implementation of rank uses Spark's Window without
- specifying partition specification. This leads to moveing all data into
+ specifying partition specification. This leads to moving all data into
a single partition in a single machine and could cause serious
performance degradation. Avoid this method with very large datasets.
@@ -3370,13 +3410,21 @@ def autocorr(self, lag: int = 1) -> float:
else:
lag_scol = F.lag(scol, lag).over(Window.orderBy(NATURAL_ORDER_COLUMN_NAME))
lag_col_name = verify_temp_column_name(sdf, "__autocorr_lag_tmp_col__")
- corr = (
- sdf.withColumn(lag_col_name, lag_scol)
- .select(F.corr(scol, F.col(lag_col_name)))
- .head()[0]
- )
+
+ sdf_lag = sdf.withColumn(lag_col_name, lag_scol)
+ if is_ansi_mode_enabled(sdf.sparkSession):
+ # Compute covariance between the original and lagged columns.
+ # If the covariance is None or zero (indicating no linear relationship),
+ # return NaN, otherwise, proceeding to compute correlation may raise
+ # DIVIDE_BY_ZERO under ANSI mode.
+ cov_value = sdf_lag.select(F.covar_samp(scol, F.col(lag_col_name))).head()[0]
+ if cov_value is None or cov_value == 0.0:
+ return np.nan
+ corr = sdf_lag.select(F.corr(scol, F.col(lag_col_name))).head()[0]
+
return np.nan if corr is None else corr
+ @with_ansi_mode_context
def corr(
self, other: "Series", method: str = "pearson", min_periods: Optional[int] = None
) -> float:
@@ -4066,7 +4114,7 @@ def rank(
assigned a rank that is the average of the ranks of those values.
.. note:: the current implementation of rank uses Spark's Window without
- specifying partition specification. This leads to moveing all data into
+ specifying partition specification. This leads to moving all data into
a single partition in a single machine and could cause serious
performance degradation. Avoid this method with very large datasets.
@@ -4244,7 +4292,7 @@ def diff(self, periods: int = 1) -> "Series":
DataFrame (default is the element in the same column of the previous row).
.. note:: the current implementation of diff uses Spark's Window without
- specifying partition specification. This leads to moveing all data into
+ specifying partition specification. This leads to moving all data into
a single partition in a single machine and could cause serious
performance degradation. Avoid this method with very large datasets.
@@ -4856,6 +4904,7 @@ def keys(self) -> "ps.Index":
return self.index
# TODO: introduce 'in_place'; fully support 'regex'
+ @with_ansi_mode_context
def replace(
self,
to_replace: Optional[Union[Any, List, Tuple, Dict]] = None,
@@ -5081,33 +5130,68 @@ def replace(
)
)
to_replace = {k: v for k, v in zip(to_replace, value)}
+
+ spark_session = self._internal.spark_frame.sparkSession
+ ansi_mode = is_ansi_mode_enabled(spark_session)
+ col_type = self.spark.data_type
+
if isinstance(to_replace, dict):
is_start = True
if len(to_replace) == 0:
current = self.spark.column
else:
for to_replace_, value in to_replace.items():
- cond = (
- (F.isnan(self.spark.column) | self.spark.column.isNull())
- if pd.isna(to_replace_)
- else (self.spark.column == F.lit(to_replace_))
- )
+ if pd.isna(to_replace_):
+ if ansi_mode and isinstance(col_type, NumericType):
+ cond = F.isnan(self.spark.column) | self.spark.column.isNull()
+ else:
+ cond = self.spark.column.isNull()
+ else:
+ to_replace_lit = (
+ F.lit(to_replace_).try_cast(col_type)
+ if ansi_mode
+ else F.lit(to_replace_)
+ )
+ cond = self.spark.column == to_replace_lit
+ value_expr = F.lit(value).try_cast(col_type) if ansi_mode else F.lit(value)
if is_start:
- current = F.when(cond, value)
+ current = F.when(cond, value_expr)
is_start = False
else:
- current = current.when(cond, value)
+ current = current.when(cond, value_expr)
current = current.otherwise(self.spark.column)
else:
if regex:
# to_replace must be a string
cond = self.spark.column.rlike(cast(str, to_replace))
else:
- cond = self.spark.column.isin(to_replace)
+ if ansi_mode:
+ to_replace_values = (
+ [to_replace]
+ if not is_list_like(to_replace) or isinstance(to_replace, str)
+ else to_replace
+ )
+ to_replace_values = cast(List[Any], to_replace_values)
+ literals = [F.lit(v).try_cast(col_type) for v in to_replace_values]
+ cond = self.spark.column.isin(literals)
+ else:
+ cond = self.spark.column.isin(to_replace)
# to_replace may be a scalar
if np.array(pd.isna(to_replace)).any():
- cond = cond | F.isnan(self.spark.column) | self.spark.column.isNull()
- current = F.when(cond, value).otherwise(self.spark.column)
+ if ansi_mode:
+ if isinstance(col_type, NumericType):
+ cond = cond | F.isnan(self.spark.column) | self.spark.column.isNull()
+ else:
+ cond = cond | self.spark.column.isNull()
+ else:
+ cond = cond | F.isnan(self.spark.column) | self.spark.column.isNull()
+
+ if ansi_mode:
+ value_expr = F.lit(value).try_cast(col_type)
+ current = F.when(cond, value_expr).otherwise(self.spark.column.try_cast(col_type))
+
+ else:
+ current = F.when(cond, value).otherwise(self.spark.column)
return self._with_new_scol(current) # TODO: dtype?
@@ -5486,7 +5570,7 @@ def pct_change(self, periods: int = 1) -> "Series":
Percentage change between the current and a prior element.
.. note:: the current implementation of this API uses Spark's Window without
- specifying partition specification. This leads to moveing all data into
+ specifying partition specification. This leads to moving all data into
a single partition in a single machine and could cause serious
performance degradation. Avoid this method with very large datasets.
@@ -7336,16 +7420,12 @@ def _test() -> None:
import sys
from pyspark.sql import SparkSession
import pyspark.pandas.series
- from pyspark.testing.utils import is_ansi_mode_test
os.chdir(os.environ["SPARK_HOME"])
globs = pyspark.pandas.series.__dict__.copy()
globs["ps"] = pyspark.pandas
- if is_ansi_mode_test:
- del pyspark.pandas.series.Series.autocorr.__doc__
-
spark = (
SparkSession.builder.master("local[4]").appName("pyspark.pandas.series tests").getOrCreate()
)
diff --git a/python/pyspark/pandas/strings.py b/python/pyspark/pandas/strings.py
index dc1544d8be39d..05d678d123f47 100644
--- a/python/pyspark/pandas/strings.py
+++ b/python/pyspark/pandas/strings.py
@@ -18,12 +18,14 @@
"""
String functions on pandas-on-Spark Series
"""
+from functools import wraps
from typing import (
Any,
Callable,
Dict,
List,
Optional,
+ TypeVar,
Union,
cast,
no_type_check,
@@ -32,13 +34,25 @@
import numpy as np
import pandas as pd
-from pyspark.pandas.utils import is_ansi_mode_enabled
+from pyspark.pandas.utils import ansi_mode_context, is_ansi_mode_enabled
from pyspark.sql.types import StringType, BinaryType, ArrayType, LongType, MapType
from pyspark.sql import functions as F
from pyspark.sql.functions import pandas_udf
import pyspark.pandas as ps
+FuncT = TypeVar("FuncT", bound=Callable[..., Any])
+
+
+def with_ansi_mode_context(f: FuncT) -> FuncT:
+ @wraps(f)
+ def _with_ansi_mode_context(self: "StringMethods", *args: Any, **kwargs: Any) -> Any:
+ with ansi_mode_context(self._data._internal.spark_frame.sparkSession):
+ return f(self, *args, **kwargs)
+
+ return cast(FuncT, _with_ansi_mode_context)
+
+
class StringMethods:
"""String methods for pandas-on-Spark Series"""
@@ -1890,6 +1904,7 @@ def pandas_slice_replace(s) -> ps.Series[str]: # type: ignore[no-untyped-def]
return self._data.pandas_on_spark.transform_batch(pandas_slice_replace)
+ @with_ansi_mode_context
def split(
self, pat: Optional[str] = None, n: int = -1, expand: bool = False
) -> Union["ps.Series", "ps.DataFrame"]:
@@ -2052,6 +2067,7 @@ def pudf(s: pd.Series) -> pd.Series:
else:
return psser
+ @with_ansi_mode_context
def rsplit(
self, pat: Optional[str] = None, n: int = -1, expand: bool = False
) -> Union["ps.Series", "ps.DataFrame"]:
diff --git a/python/pyspark/pandas/supported_api_gen.py b/python/pyspark/pandas/supported_api_gen.py
index f2a73cb1c1adf..595c11c559d05 100644
--- a/python/pyspark/pandas/supported_api_gen.py
+++ b/python/pyspark/pandas/supported_api_gen.py
@@ -38,7 +38,7 @@
MAX_MISSING_PARAMS_SIZE = 5
COMMON_PARAMETER_SET = {"kwargs", "args", "cls"}
MODULE_GROUP_MATCH = [(pd, ps), (pdw, psw), (pdg, psg)]
-PANDAS_LATEST_VERSION = "2.2.3"
+PANDAS_LATEST_VERSION = "2.3.2"
RST_HEADER = """
=====================
@@ -108,7 +108,9 @@ def _check_pandas_version() -> None:
"""
Check if the installed pandas version matches the expected version.
"""
- if LooseVersion(pd.__version__) != LooseVersion(PANDAS_LATEST_VERSION):
+ # Work around pandas version string issue,
+ # see https://github.com/pandas-dev/pandas/issues/61579.
+ if LooseVersion(pd.__version__.split("+")[0]) != LooseVersion(PANDAS_LATEST_VERSION):
msg = (
f"Warning: pandas {PANDAS_LATEST_VERSION} is required; your version is {pd.__version__}"
)
diff --git a/python/pyspark/pandas/tests/computation/test_binary_ops.py b/python/pyspark/pandas/tests/computation/test_binary_ops.py
index 3c9b7293d5d53..1f76064cd6df1 100644
--- a/python/pyspark/pandas/tests/computation/test_binary_ops.py
+++ b/python/pyspark/pandas/tests/computation/test_binary_ops.py
@@ -23,7 +23,6 @@
from pyspark import pandas as ps
from pyspark.testing.pandasutils import PandasOnSparkTestCase
from pyspark.testing.sqlutils import SQLTestUtils
-from pyspark.testing.utils import is_ansi_mode_test, ansi_mode_not_supported_message
# This file contains test cases for 'Binary operator functions'
@@ -122,8 +121,7 @@ def test_divide_by_zero_behavior(self):
dtype=np.float32,
)
psdf = ps.from_pandas(pdf)
- # TODO(SPARK-52332): Fix promotion from float32 to float64 during division
- self.assert_eq(psdf["a"] / psdf["b"], (pdf["a"] / pdf["b"]).astype(np.float64))
+ self.assert_eq(psdf["a"] / psdf["b"], pdf["a"] / pdf["b"])
# np.float64
pdf = pd.DataFrame(
@@ -195,6 +193,11 @@ def test_binary_operator_truediv(self):
self.assert_eq(psdf["a"] / psdf["b"], pdf["a"] / pdf["b"])
+ pser = pd.Series([1.1, 2.2, 3.3], dtype=np.float32)
+ psser = ps.from_pandas(pser)
+ self.assert_eq(psser / 1, pser / 1)
+ self.assert_eq(psser / 0, pser / 0)
+
# Negative
psdf = ps.DataFrame({"a": ["x"], "b": [1]})
@@ -208,7 +211,15 @@ def test_binary_operator_truediv(self):
self.assertRaisesRegex(TypeError, ks_err_msg, lambda: 1 / psdf["a"])
def test_binary_operator_floordiv(self):
- psdf = ps.DataFrame({"a": ["x"], "b": [1]})
+ pdf = pd.DataFrame({"a": ["x"], "b": [1], "c": [1.0], "d": [0]})
+ psdf = ps.from_pandas(pdf)
+ self.assert_eq(pdf["b"] // 0, psdf["b"] // 0)
+ self.assert_eq(pdf["c"] // 0, psdf["c"] // 0)
+ self.assert_eq(pdf["d"] // 0, psdf["d"] // 0)
+
+ pser = pd.Series([1.1, 2.2, 3.3], dtype=np.float32)
+ psser = ps.from_pandas(pser)
+ self.assert_eq(psser // 1, pser // 1)
ks_err_msg = "Floor division can not be applied to strings"
self.assertRaisesRegex(TypeError, ks_err_msg, lambda: psdf["a"] // psdf["b"])
@@ -221,10 +232,12 @@ def test_binary_operator_floordiv(self):
def test_binary_operator_mod(self):
# Positive
- pdf = pd.DataFrame({"a": [3], "b": [2]})
+ pdf = pd.DataFrame({"a": [3], "b": [2], "c": [0]})
psdf = ps.from_pandas(pdf)
self.assert_eq(psdf["a"] % psdf["b"], pdf["a"] % pdf["b"])
+ self.assert_eq(psdf["a"] % 0, pdf["a"] % 0)
+ self.assert_eq(1 % psdf["c"], 1 % pdf["c"])
# Negative
psdf = ps.DataFrame({"a": ["x"], "b": [1]})
diff --git a/python/pyspark/pandas/tests/computation/test_corr.py b/python/pyspark/pandas/tests/computation/test_corr.py
index ad982717e9a4c..49cb84ca22e04 100644
--- a/python/pyspark/pandas/tests/computation/test_corr.py
+++ b/python/pyspark/pandas/tests/computation/test_corr.py
@@ -22,11 +22,9 @@
from pyspark import pandas as ps
from pyspark.testing.pandasutils import PandasOnSparkTestCase, SPARK_CONF_ARROW_ENABLED
from pyspark.testing.sqlutils import SQLTestUtils
-from pyspark.testing.utils import is_ansi_mode_test, ansi_mode_not_supported_message
class FrameCorrMixin:
- @unittest.skipIf(is_ansi_mode_test, ansi_mode_not_supported_message)
def test_dataframe_corr(self):
pdf = pd.DataFrame(
index=[
diff --git a/python/pyspark/pandas/tests/computation/test_describe.py b/python/pyspark/pandas/tests/computation/test_describe.py
index f88eac7942d17..8df07f1945d24 100644
--- a/python/pyspark/pandas/tests/computation/test_describe.py
+++ b/python/pyspark/pandas/tests/computation/test_describe.py
@@ -26,6 +26,12 @@
class FrameDescribeMixin:
+ @classmethod
+ def setUpClass(cls):
+ super(FrameDescribeMixin, cls).setUpClass()
+ # Some nanosecond->microsecond conversions throw loss of precision errors
+ cls.spark.conf.set("spark.sql.execution.pandas.convertToArrowArraySafely", "false")
+
@property
def pdf(self):
return pd.DataFrame(
diff --git a/python/pyspark/pandas/tests/computation/test_missing_data.py b/python/pyspark/pandas/tests/computation/test_missing_data.py
index dfecaf4be20b0..c777499aea85f 100644
--- a/python/pyspark/pandas/tests/computation/test_missing_data.py
+++ b/python/pyspark/pandas/tests/computation/test_missing_data.py
@@ -23,7 +23,6 @@
from pyspark import pandas as ps
from pyspark.testing.pandasutils import PandasOnSparkTestCase
from pyspark.testing.sqlutils import SQLTestUtils
-from pyspark.testing.utils import is_ansi_mode_test, ansi_mode_not_supported_message
# This file contains test cases for 'Missing data handling'
@@ -274,7 +273,6 @@ def test_fillna(self):
pdf.fillna({("x", "a"): -1, ("x", "b"): -2, ("y", "c"): -5}),
)
- @unittest.skipIf(is_ansi_mode_test, ansi_mode_not_supported_message)
def test_replace(self):
pdf = pd.DataFrame(
{
diff --git a/python/pyspark/pandas/tests/data_type_ops/test_as_type.py b/python/pyspark/pandas/tests/data_type_ops/test_as_type.py
index 379d055d585d3..76debd10848a4 100644
--- a/python/pyspark/pandas/tests/data_type_ops/test_as_type.py
+++ b/python/pyspark/pandas/tests/data_type_ops/test_as_type.py
@@ -32,19 +32,17 @@
class AsTypeTestsMixin:
- """Unit tests for arithmetic operations of numeric data types.
-
- A few test cases are disabled because pandas-on-Spark returns float64 whereas pandas
- returns float32.
- The underlying reason is the respective Spark operations return DoubleType always.
- """
+ """Unit tests for arithmetic operations of numeric data types."""
def test_astype(self):
pdf, psdf = self.pdf, self.psdf
+ int_types = [int, np.int32, np.int16, np.int8]
+ cat_type = CategoricalDtype(categories=[2, 1, 3])
+ other_types = [float, np.float32, bool, str, "category", cat_type]
for col in self.numeric_df_cols:
pser, psser = pdf[col], psdf[col]
- for int_type in [int, np.int32, np.int16, np.int8]:
+ for int_type in int_types:
if not pser.hasnans:
self.assert_eq(pser.astype(int_type), psser.astype(int_type))
else:
@@ -54,14 +52,9 @@ def test_astype(self):
"values to integer" % psser._dtype_op.pretty_name,
lambda: psser.astype(int_type),
)
+ for other_type in other_types:
+ self.assert_eq(pser.astype(other_type), psser.astype(other_type))
- self.assert_eq(pser.astype(bool), psser.astype(bool))
- self.assert_eq(pser.astype(float), psser.astype(float))
- self.assert_eq(pser.astype(np.float32), psser.astype(np.float32))
- self.assert_eq(pser.astype(str), psser.astype(str))
- self.assert_eq(pser.astype("category"), psser.astype("category"))
- cat_type = CategoricalDtype(categories=[2, 1, 3])
- self.assert_eq(pser.astype(cat_type), psser.astype(cat_type))
if extension_object_dtypes_available and extension_float_dtypes_available:
pser = pd.Series(pd.Categorical([1.0, 2.0, 3.0]), dtype=pd.Float64Dtype())
psser = ps.from_pandas(pser)
diff --git a/python/pyspark/pandas/tests/data_type_ops/test_boolean_ops.py b/python/pyspark/pandas/tests/data_type_ops/test_boolean_ops.py
index 6969c527b5f74..ea4b23d537f9a 100644
--- a/python/pyspark/pandas/tests/data_type_ops/test_boolean_ops.py
+++ b/python/pyspark/pandas/tests/data_type_ops/test_boolean_ops.py
@@ -25,7 +25,6 @@
from pyspark import pandas as ps
from pyspark.pandas import option_context
from pyspark.testing.pandasutils import PandasOnSparkTestCase
-from pyspark.testing.utils import is_ansi_mode_test, ansi_mode_not_supported_message
from pyspark.pandas.tests.data_type_ops.testing_utils import OpsTestBase
from pyspark.pandas.typedef.typehints import (
extension_float_dtypes_available,
@@ -100,11 +99,11 @@ def test_mul(self):
else:
self.assertRaises(TypeError, lambda: b_psser * psser)
- @unittest.skipIf(is_ansi_mode_test, ansi_mode_not_supported_message)
def test_truediv(self):
pdf, psdf = self.pdf, self.psdf
b_pser, b_psser = pdf["bool"], psdf["bool"]
+ self.assert_eq(b_pser / 0, b_psser / 0)
self.assert_eq(b_pser / 1, b_psser / 1)
self.assert_eq(b_pser / 0.1, b_psser / 0.1)
self.assert_eq(b_pser / b_pser.astype(int), b_psser / b_psser.astype(int))
@@ -116,7 +115,6 @@ def test_truediv(self):
for col in self.non_numeric_df_cols:
self.assertRaises(TypeError, lambda: b_psser / psdf[col])
- @unittest.skipIf(is_ansi_mode_test, ansi_mode_not_supported_message)
def test_floordiv(self):
pdf, psdf = self.pdf, self.psdf
@@ -124,6 +122,7 @@ def test_floordiv(self):
# float is always returned in pandas-on-Spark
self.assert_eq((b_pser // 1).astype("float"), b_psser // 1)
+ self.assert_eq((b_pser // 0).astype("float"), b_psser // 0)
# in pandas, 1 // 0.1 = 9.0; in pandas-on-Spark, 1 // 0.1 = 10.0
# self.assert_eq(b_pser // 0.1, b_psser // 0.1)
@@ -141,6 +140,7 @@ def test_mod(self):
pdf, psdf = self.pdf, self.psdf
b_pser, b_psser = pdf["bool"], psdf["bool"]
+ self.assert_eq(b_pser % 0, b_psser % 0)
self.assert_eq(b_pser % 1, b_psser % 1)
self.assert_eq(b_pser % 0.1, b_psser % 0.1)
self.assert_eq(b_pser % b_pser.astype(float), b_psser % b_psser.astype(float))
@@ -238,16 +238,13 @@ def test_rpow(self):
def test_rmod(self):
psdf = self.psdf
+ pdf = self.pdf
b_psser = psdf["bool"]
- # 1 % False is 0.0 in pandas
- self.assert_eq(pd.Series([0, 0, None], dtype=float, name="bool"), 1 % b_psser)
- # 0.1 / True is 0.1 in pandas
- self.assert_eq(
- pd.Series([0.10000000000000009, 0.10000000000000009, None], dtype=float, name="bool"),
- 0.1 % b_psser,
- check_exact=False, # [0.1, 0.1, nan] for pandas-on-Spark
- )
+ b_pser = pdf["bool"]
+ self.assert_eq(1 % b_pser.astype(float), 1 % b_psser)
+ # # Allow float precision diff: pandas: 0.10000000000000009; pandas on spark: 0.1
+ self.assert_eq(0.1 % b_pser, 0.1 % b_psser, almost=True)
self.assertRaises(TypeError, lambda: datetime.date(1994, 1, 1) % b_psser)
self.assertRaises(TypeError, lambda: True % b_psser)
diff --git a/python/pyspark/pandas/tests/data_type_ops/test_date_ops.py b/python/pyspark/pandas/tests/data_type_ops/test_date_ops.py
index c2b29ee8a1d35..a584497832b25 100644
--- a/python/pyspark/pandas/tests/data_type_ops/test_date_ops.py
+++ b/python/pyspark/pandas/tests/data_type_ops/test_date_ops.py
@@ -208,31 +208,37 @@ def test_eq(self):
pdf, psdf = self.date_pdf, self.date_psdf
self.assert_eq(pdf["this"] == pdf["that"], psdf["this"] == psdf["that"])
self.assert_eq(pdf["this"] == pdf["this"], psdf["this"] == psdf["this"])
+ self.assertRaises(TypeError, lambda: psdf["this"] == pdf["this"])
def test_ne(self):
pdf, psdf = self.date_pdf, self.date_psdf
self.assert_eq(pdf["this"] != pdf["that"], psdf["this"] != psdf["that"])
self.assert_eq(pdf["this"] != pdf["this"], psdf["this"] != psdf["this"])
+ self.assertRaises(TypeError, lambda: psdf["this"] != pdf["this"])
def test_lt(self):
pdf, psdf = self.date_pdf, self.date_psdf
self.assert_eq(pdf["this"] < pdf["that"], psdf["this"] < psdf["that"])
self.assert_eq(pdf["this"] < pdf["this"], psdf["this"] < psdf["this"])
+ self.assertRaises(TypeError, lambda: psdf["this"] < pdf["this"])
def test_le(self):
pdf, psdf = self.date_pdf, self.date_psdf
self.assert_eq(pdf["this"] <= pdf["that"], psdf["this"] <= psdf["that"])
self.assert_eq(pdf["this"] <= pdf["this"], psdf["this"] <= psdf["this"])
+ self.assertRaises(TypeError, lambda: psdf["this"] <= pdf["this"])
def test_gt(self):
pdf, psdf = self.date_pdf, self.date_psdf
self.assert_eq(pdf["this"] > pdf["that"], psdf["this"] > psdf["that"])
self.assert_eq(pdf["this"] > pdf["this"], psdf["this"] > psdf["this"])
+ self.assertRaises(TypeError, lambda: psdf["this"] > pdf["this"])
def test_ge(self):
pdf, psdf = self.date_pdf, self.date_psdf
self.assert_eq(pdf["this"] >= pdf["that"], psdf["this"] >= psdf["that"])
self.assert_eq(pdf["this"] >= pdf["this"], psdf["this"] >= psdf["this"])
+ self.assertRaises(TypeError, lambda: psdf["this"] >= pdf["this"])
class DateOpsTests(
diff --git a/python/pyspark/pandas/tests/data_type_ops/test_num_arithmetic.py b/python/pyspark/pandas/tests/data_type_ops/test_num_arithmetic.py
index 0201dd6fa1f5f..798a49ae121ed 100644
--- a/python/pyspark/pandas/tests/data_type_ops/test_num_arithmetic.py
+++ b/python/pyspark/pandas/tests/data_type_ops/test_num_arithmetic.py
@@ -20,8 +20,8 @@
import pandas as pd
from pyspark import pandas as ps
+from pyspark.testing.utils import is_ansi_mode_test
from pyspark.testing.pandasutils import PandasOnSparkTestCase
-from pyspark.testing.utils import is_ansi_mode_test, ansi_mode_not_supported_message
from pyspark.pandas.tests.data_type_ops.testing_utils import OpsTestBase
@@ -41,14 +41,12 @@ def float_pser(self):
def float_psser(self):
return ps.from_pandas(self.float_pser)
- @unittest.skipIf(is_ansi_mode_test, ansi_mode_not_supported_message)
def test_add(self):
pdf, psdf = self.pdf, self.psdf
for col in self.numeric_df_cols:
pser, psser = pdf[col], psdf[col]
self.assert_eq(pser + pser, psser + psser, check_exact=False)
self.assert_eq(pser + 1, psser + 1, check_exact=False)
- # self.assert_eq(pser + 0.1, psser + 0.1)
self.assert_eq(pser + pser.astype(bool), psser + psser.astype(bool), check_exact=False)
self.assert_eq(pser + True, psser + True, check_exact=False)
self.assert_eq(pser + False, psser + False, check_exact=False)
@@ -59,14 +57,15 @@ def test_add(self):
else:
self.assertRaises(TypeError, lambda: psser + psdf[n_col])
- @unittest.skipIf(is_ansi_mode_test, ansi_mode_not_supported_message)
+ if is_ansi_mode_test and not col.startswith("decimal"):
+ self.assert_eq(pser + 0.1, psser + 0.1)
+
def test_sub(self):
pdf, psdf = self.pdf, self.psdf
for col in self.numeric_df_cols:
pser, psser = pdf[col], psdf[col]
self.assert_eq(pser - pser, psser - psser, check_exact=False)
self.assert_eq(pser - 1, psser - 1, check_exact=False)
- # self.assert_eq(pser - 0.1, psser - 0.1)
self.assert_eq(pser - pser.astype(bool), psser - psser.astype(bool), check_exact=False)
self.assert_eq(pser - True, psser - True, check_exact=False)
self.assert_eq(pser - False, psser - False, check_exact=False)
@@ -77,6 +76,9 @@ def test_sub(self):
else:
self.assertRaises(TypeError, lambda: psser - psdf[n_col])
+ if is_ansi_mode_test and not col.startswith("decimal"):
+ self.assert_eq(pser - 0.1, psser - 0.1)
+
class ArithmeticTests(
ArithmeticTestsMixin,
diff --git a/python/pyspark/pandas/tests/data_type_ops/test_num_mod.py b/python/pyspark/pandas/tests/data_type_ops/test_num_mod.py
index 5e4b6f46f4333..b1961437a96eb 100644
--- a/python/pyspark/pandas/tests/data_type_ops/test_num_mod.py
+++ b/python/pyspark/pandas/tests/data_type_ops/test_num_mod.py
@@ -22,7 +22,6 @@
from pyspark import pandas as ps
from pyspark.testing.pandasutils import PandasOnSparkTestCase
-from pyspark.testing.utils import is_ansi_mode_test, ansi_mode_not_supported_message
from pyspark.pandas.tests.data_type_ops.testing_utils import OpsTestBase
@@ -35,7 +34,6 @@ def float_pser(self):
def float_psser(self):
return ps.from_pandas(self.float_pser)
- @unittest.skipIf(is_ansi_mode_test, ansi_mode_not_supported_message)
def test_mod(self):
pdf, psdf = self.pdf, self.psdf
for col in self.numeric_df_cols:
@@ -43,6 +41,9 @@ def test_mod(self):
self.assert_eq(pser % pser, psser % psser, check_exact=False)
self.assert_eq(pser % pser.astype(bool), psser % psser.astype(bool), check_exact=False)
self.assert_eq(pser % True, psser % True, check_exact=False)
+ self.assert_eq(pser % 1, psser % 1, check_exact=False)
+ if not col.startswith("decimal"):
+ self.assert_eq(pser % 0, psser % 0, check_exact=False)
if col in ["int", "int32"]:
self.assert_eq(
pd.Series([np.nan, np.nan, np.nan], dtype=float, name=col), psser % False
diff --git a/python/pyspark/pandas/tests/data_type_ops/test_num_mul_div.py b/python/pyspark/pandas/tests/data_type_ops/test_num_mul_div.py
index 3c58e4655125e..8a719fe9fd88c 100644
--- a/python/pyspark/pandas/tests/data_type_ops/test_num_mul_div.py
+++ b/python/pyspark/pandas/tests/data_type_ops/test_num_mul_div.py
@@ -22,7 +22,6 @@
from pyspark import pandas as ps
from pyspark.testing.pandasutils import PandasOnSparkTestCase
-from pyspark.testing.utils import is_ansi_mode_test, ansi_mode_not_supported_message
from pyspark.pandas.tests.data_type_ops.testing_utils import OpsTestBase
@@ -35,7 +34,6 @@ def float_pser(self):
def float_psser(self):
return ps.from_pandas(self.float_pser)
- @unittest.skipIf(is_ansi_mode_test, ansi_mode_not_supported_message)
def test_mul(self):
pdf, psdf = self.pdf, self.psdf
for col in self.numeric_df_cols:
diff --git a/python/pyspark/pandas/tests/data_type_ops/test_num_ops.py b/python/pyspark/pandas/tests/data_type_ops/test_num_ops.py
index 03a794771a910..00fc04e362312 100644
--- a/python/pyspark/pandas/tests/data_type_ops/test_num_ops.py
+++ b/python/pyspark/pandas/tests/data_type_ops/test_num_ops.py
@@ -23,6 +23,7 @@
from pyspark import pandas as ps
from pyspark.pandas.config import option_context
from pyspark.testing.pandasutils import PandasOnSparkTestCase
+from pyspark.testing.utils import is_ansi_mode_test
from pyspark.pandas.tests.data_type_ops.testing_utils import OpsTestBase
from pyspark.pandas.typedef.typehints import (
extension_dtypes_available,
@@ -128,6 +129,18 @@ def test_invert(self):
else:
self.assertRaises(TypeError, lambda: ~psser)
+ def test_comparison_dtype_compatibility(self):
+ pdf = pd.DataFrame(
+ {"int": [1, 2], "bool": [True, False], "float": [0.1, 0.2], "str": ["1", "2"]}
+ )
+ psdf = ps.from_pandas(pdf)
+ self.assert_eq(pdf["int"] == pdf["bool"], psdf["int"] == psdf["bool"])
+ self.assert_eq(pdf["bool"] == pdf["int"], psdf["bool"] == psdf["int"])
+ self.assert_eq(pdf["int"] == pdf["float"], psdf["int"] == psdf["float"])
+ if is_ansi_mode_test: # TODO: match non-ansi behavior with pandas
+ self.assert_eq(pdf["int"] == pdf["str"], psdf["int"] == psdf["str"])
+ self.assert_eq(pdf["float"] == pdf["bool"], psdf["float"] == psdf["bool"])
+
def test_eq(self):
pdf, psdf = self.pdf, self.psdf
for col in self.numeric_df_cols:
diff --git a/python/pyspark/pandas/tests/data_type_ops/test_num_reverse.py b/python/pyspark/pandas/tests/data_type_ops/test_num_reverse.py
index 38341bce80308..a14177348cd00 100644
--- a/python/pyspark/pandas/tests/data_type_ops/test_num_reverse.py
+++ b/python/pyspark/pandas/tests/data_type_ops/test_num_reverse.py
@@ -22,7 +22,6 @@
from pyspark import pandas as ps
from pyspark.testing.pandasutils import PandasOnSparkTestCase
-from pyspark.testing.utils import is_ansi_mode_test, ansi_mode_not_supported_message
from pyspark.pandas.tests.data_type_ops.testing_utils import OpsTestBase
@@ -42,7 +41,6 @@ def float_pser(self):
def float_psser(self):
return ps.from_pandas(self.float_pser)
- @unittest.skipIf(is_ansi_mode_test, ansi_mode_not_supported_message)
def test_radd(self):
pdf, psdf = self.pdf, self.psdf
for col in self.numeric_df_cols:
@@ -55,7 +53,6 @@ def test_radd(self):
self.assertRaises(TypeError, lambda: datetime.date(1994, 1, 1) + psser)
self.assertRaises(TypeError, lambda: datetime.datetime(1994, 1, 1) + psser)
- @unittest.skipIf(is_ansi_mode_test, ansi_mode_not_supported_message)
def test_rsub(self):
pdf, psdf = self.pdf, self.psdf
for col in self.numeric_df_cols:
@@ -68,7 +65,6 @@ def test_rsub(self):
self.assertRaises(TypeError, lambda: datetime.date(1994, 1, 1) - psser)
self.assertRaises(TypeError, lambda: datetime.datetime(1994, 1, 1) - psser)
- @unittest.skipIf(is_ansi_mode_test, ansi_mode_not_supported_message)
def test_rmul(self):
pdf, psdf = self.pdf, self.psdf
for col in self.numeric_df_cols:
@@ -117,7 +113,6 @@ def test_rpow(self):
self.assertRaises(TypeError, lambda: datetime.date(1994, 1, 1) ** psser)
self.assertRaises(TypeError, lambda: datetime.datetime(1994, 1, 1) ** psser)
- @unittest.skipIf(is_ansi_mode_test, ansi_mode_not_supported_message)
def test_rmod(self):
pdf, psdf = self.pdf, self.psdf
for col in self.numeric_df_cols:
diff --git a/python/pyspark/pandas/tests/data_type_ops/test_string_ops.py b/python/pyspark/pandas/tests/data_type_ops/test_string_ops.py
index 9648ad9ab2c0f..9c3f4cf1dec4b 100644
--- a/python/pyspark/pandas/tests/data_type_ops/test_string_ops.py
+++ b/python/pyspark/pandas/tests/data_type_ops/test_string_ops.py
@@ -33,19 +33,19 @@
class StringOpsTestsMixin:
@property
- def bool_pdf(self):
+ def str_pdf(self):
return pd.DataFrame({"this": ["x", "y", "z"], "that": ["z", "y", "x"]})
@property
- def bool_non_numeric_pdf(self):
- return pd.concat([self.bool_pdf, self.non_numeric_pdf], axis=1)
+ def str_non_numeric_pdf(self):
+ return pd.concat([self.str_pdf, self.non_numeric_pdf], axis=1)
@property
- def bool_non_numeric_psdf(self):
- return ps.from_pandas(self.bool_non_numeric_pdf)
+ def str_non_numeric_psdf(self):
+ return ps.from_pandas(self.str_non_numeric_pdf)
def test_add(self):
- pdf, psdf = self.bool_non_numeric_pdf, self.bool_non_numeric_psdf
+ pdf, psdf = self.str_non_numeric_pdf, self.str_non_numeric_psdf
pser, psser = pdf["this"], psdf["this"]
other_pser, other_psser = pdf["that"], psdf["that"]
self.assert_eq(pser + "x", psser + "x")
@@ -188,42 +188,42 @@ def test_invert(self):
self.assertRaises(TypeError, lambda: ~self.psdf["string"])
def test_eq(self):
- pdf, psdf = self.bool_non_numeric_pdf, self.bool_non_numeric_psdf
+ pdf, psdf = self.str_non_numeric_pdf, self.str_non_numeric_psdf
pser, psser = pdf["this"], psdf["this"]
other_pser, other_psser = pdf["that"], psdf["that"]
self.assert_eq(pser == other_pser, psser == other_psser)
self.assert_eq(pser == pser, psser == psser)
def test_ne(self):
- pdf, psdf = self.bool_non_numeric_pdf, self.bool_non_numeric_psdf
+ pdf, psdf = self.str_non_numeric_pdf, self.str_non_numeric_psdf
pser, psser = pdf["this"], psdf["this"]
other_pser, other_psser = pdf["that"], psdf["that"]
self.assert_eq(pser != other_pser, psser != other_psser)
self.assert_eq(pser != pser, psser != psser)
def test_lt(self):
- pdf, psdf = self.bool_non_numeric_pdf, self.bool_non_numeric_psdf
+ pdf, psdf = self.str_non_numeric_pdf, self.str_non_numeric_psdf
pser, psser = pdf["this"], psdf["this"]
other_pser, other_psser = pdf["that"], psdf["that"]
self.assert_eq(pser < other_pser, psser < other_psser)
self.assert_eq(pser < pser, psser < psser)
def test_le(self):
- pdf, psdf = self.bool_non_numeric_pdf, self.bool_non_numeric_psdf
+ pdf, psdf = self.str_non_numeric_pdf, self.str_non_numeric_psdf
pser, psser = pdf["this"], psdf["this"]
other_pser, other_psser = pdf["that"], psdf["that"]
self.assert_eq(pser <= other_pser, psser <= other_psser)
self.assert_eq(pser <= pser, psser <= psser)
def test_gt(self):
- pdf, psdf = self.bool_non_numeric_pdf, self.bool_non_numeric_psdf
+ pdf, psdf = self.str_non_numeric_pdf, self.str_non_numeric_psdf
pser, psser = pdf["this"], psdf["this"]
other_pser, other_psser = pdf["that"], psdf["that"]
self.assert_eq(pser > other_pser, psser > other_psser)
self.assert_eq(pser > pser, psser > psser)
def test_ge(self):
- pdf, psdf = self.bool_non_numeric_pdf, self.bool_non_numeric_psdf
+ pdf, psdf = self.str_non_numeric_pdf, self.str_non_numeric_psdf
pser, psser = pdf["this"], psdf["this"]
other_pser, other_psser = pdf["that"], psdf["that"]
self.assert_eq(pser >= other_pser, psser >= other_psser)
diff --git a/python/pyspark/pandas/tests/data_type_ops/test_udt_ops.py b/python/pyspark/pandas/tests/data_type_ops/test_udt_ops.py
index 60b4153198a34..f4f833ea9cf55 100644
--- a/python/pyspark/pandas/tests/data_type_ops/test_udt_ops.py
+++ b/python/pyspark/pandas/tests/data_type_ops/test_udt_ops.py
@@ -130,6 +130,26 @@ def test_from_to_pandas(self):
self.assert_eq(pser, psser._to_pandas())
self.assert_eq(ps.from_pandas(pser), psser)
+ def test_with_first_null(self):
+ lst = [None, None, None, SparseVector(1, {0: 0.1})]
+ pser = pd.Series(lst)
+ psser = ps.Series(lst)
+ self.assert_eq(pser, psser._to_pandas())
+ self.assert_eq(ps.from_pandas(pser), psser)
+
+ lst2 = [SparseVector(1, {0: 0.1}), None, None, None]
+ pdf = pd.DataFrame({"a": lst, "b": lst2})
+ psdf = ps.DataFrame({"a": lst, "b": lst2})
+ self.assert_eq(pdf, psdf._to_pandas())
+ self.assert_eq(ps.from_pandas(pdf), psdf)
+
+ def test_with_all_null(self):
+ lst = [None, None, None, None]
+ pser = pd.Series(lst, dtype=object)
+ psser = ps.Series(lst, dtype=object)
+ self.assert_eq(pser, psser._to_pandas())
+ self.assert_eq(ps.from_pandas(pser), psser)
+
def test_isnull(self):
self.assert_eq(self.pser.isnull(), self.psser.isnull())
diff --git a/python/pyspark/pandas/tests/data_type_ops/testing_utils.py b/python/pyspark/pandas/tests/data_type_ops/testing_utils.py
index 089b929db1b33..17ac2bc5c4747 100644
--- a/python/pyspark/pandas/tests/data_type_ops/testing_utils.py
+++ b/python/pyspark/pandas/tests/data_type_ops/testing_utils.py
@@ -41,6 +41,12 @@
class OpsTestBase:
"""The test base for arithmetic operations of different data types."""
+ @classmethod
+ def setUpClass(cls):
+ super(OpsTestBase, cls).setUpClass()
+ # Some nanosecond->microsecond conversions throw loss of precision errors
+ cls.spark.conf.set("spark.sql.execution.pandas.convertToArrowArraySafely", "false")
+
@property
def numeric_pdf(self):
dtypes = [np.int32, int, np.float32, float]
diff --git a/python/pyspark/pandas/tests/diff_frames_ops/test_corrwith.py b/python/pyspark/pandas/tests/diff_frames_ops/test_corrwith.py
index b6474ba9f0ecd..1bd274b45a742 100644
--- a/python/pyspark/pandas/tests/diff_frames_ops/test_corrwith.py
+++ b/python/pyspark/pandas/tests/diff_frames_ops/test_corrwith.py
@@ -20,11 +20,9 @@
import numpy as np
from pyspark import pandas as ps
-from pyspark.loose_version import LooseVersion
from pyspark.pandas.config import set_option, reset_option
from pyspark.testing.pandasutils import PandasOnSparkTestCase
from pyspark.testing.sqlutils import SQLTestUtils
-from pyspark.testing.utils import is_ansi_mode_test, ansi_mode_not_supported_message
class DiffFramesCorrWithMixin:
@@ -82,7 +80,6 @@ def tearDownClass(cls):
reset_option("compute.ops_on_diff_frames")
super().tearDownClass()
- @unittest.skipIf(is_ansi_mode_test, ansi_mode_not_supported_message)
def test_corrwith(self):
df1 = ps.DataFrame({"A": [1, np.nan, 7, 8], "X": [5, 8, np.nan, 3], "C": [10, 4, 9, 3]})
df2 = ps.DataFrame({"A": [5, 3, 6, 4], "B": [11, 2, 4, 3], "C": [4, 3, 8, np.nan]})
@@ -96,11 +93,7 @@ def test_corrwith(self):
# and https://github.com/pandas-dev/pandas/pull/46174 for the initial PR that causes.
df_bool = ps.DataFrame({"A": [True, True, False, False], "B": [True, False, False, True]})
ser_bool = ps.Series([True, True, False, True])
- if LooseVersion(pd.__version__) == LooseVersion("1.5.0"):
- expected = ps.Series([0.5773502691896257, 0.5773502691896257], index=["B", "A"])
- self.assert_eq(df_bool.corrwith(ser_bool), expected, almost=True)
- else:
- self._test_corrwith(df_bool, ser_bool)
+ self._test_corrwith(df_bool, ser_bool)
self._test_corrwith(self.psdf1, self.psdf1)
self._test_corrwith(self.psdf1, self.psdf2)
@@ -108,15 +101,7 @@ def test_corrwith(self):
self._test_corrwith(self.psdf3, self.psdf4)
self._test_corrwith(self.psdf1, self.psdf1.a)
- # There was a regression in pandas 1.5.0, and fixed in pandas 1.5.1.
- # Therefore, we only test the pandas 1.5.0 in different way.
- # See https://github.com/pandas-dev/pandas/issues/49141 for the reported issue,
- # and https://github.com/pandas-dev/pandas/pull/46174 for the initial PR that causes.
- if LooseVersion(pd.__version__) == LooseVersion("1.5.0"):
- expected = ps.Series([-0.08827348295047496, 0.4413674147523748], index=["b", "a"])
- self.assert_eq(self.psdf1.corrwith(self.psdf2.b), expected, almost=True)
- else:
- self._test_corrwith(self.psdf1, self.psdf2.b)
+ self._test_corrwith(self.psdf1, self.psdf2.b)
self._test_corrwith(self.psdf2, self.psdf3.c)
self._test_corrwith(self.psdf3, self.psdf4.f)
diff --git a/python/pyspark/pandas/tests/diff_frames_ops/test_dot_frame.py b/python/pyspark/pandas/tests/diff_frames_ops/test_dot_frame.py
index 628136d3519db..7a94e1858f09e 100644
--- a/python/pyspark/pandas/tests/diff_frames_ops/test_dot_frame.py
+++ b/python/pyspark/pandas/tests/diff_frames_ops/test_dot_frame.py
@@ -21,7 +21,6 @@
from pyspark.pandas.config import set_option, reset_option
from pyspark.testing.pandasutils import PandasOnSparkTestCase
from pyspark.testing.sqlutils import SQLTestUtils
-from pyspark.testing.utils import is_ansi_mode_test, ansi_mode_not_supported_message
class DiffFramesDotFrameMixin:
@@ -35,7 +34,6 @@ def tearDownClass(cls):
reset_option("compute.ops_on_diff_frames")
super().tearDownClass()
- @unittest.skipIf(is_ansi_mode_test, ansi_mode_not_supported_message)
def test_frame_dot(self):
pdf = pd.DataFrame([[0, 1, -2, -1], [1, 1, 1, 1]])
psdf = ps.from_pandas(pdf)
diff --git a/python/pyspark/pandas/tests/frame/test_constructor.py b/python/pyspark/pandas/tests/frame/test_constructor.py
index 2d4f0f541f107..d0d842d8264c7 100644
--- a/python/pyspark/pandas/tests/frame/test_constructor.py
+++ b/python/pyspark/pandas/tests/frame/test_constructor.py
@@ -30,7 +30,6 @@
from pyspark.testing.pandasutils import PandasOnSparkTestCase
from pyspark.testing.sqlutils import SQLTestUtils
-from pyspark.testing.utils import is_ansi_mode_test, ansi_mode_not_supported_message
# This file contains test cases for 'Constructor'
@@ -549,7 +548,6 @@ def test_astype_extension_object_dtypes(self):
@unittest.skipIf(
not extension_float_dtypes_available, "pandas extension float dtypes are not available"
)
- @unittest.skipIf(is_ansi_mode_test, ansi_mode_not_supported_message)
def test_extension_float_dtypes(self):
pdf = pd.DataFrame(
{
diff --git a/python/pyspark/pandas/tests/frame/test_reindexing.py b/python/pyspark/pandas/tests/frame/test_reindexing.py
index ebfaa8910cb7c..0d231a4a6cc71 100644
--- a/python/pyspark/pandas/tests/frame/test_reindexing.py
+++ b/python/pyspark/pandas/tests/frame/test_reindexing.py
@@ -25,7 +25,6 @@
from pyspark.pandas.config import option_context
from pyspark.testing.pandasutils import PandasOnSparkTestCase
from pyspark.testing.sqlutils import SQLTestUtils
-from pyspark.testing.utils import is_ansi_mode_test, ansi_mode_not_supported_message
# This file contains test cases for 'Reindexing / Selection / Label manipulation'
@@ -765,7 +764,6 @@ def test_swapaxes(self):
self.assertRaises(AssertionError, lambda: psdf.swapaxes(0, 1, copy=False))
self.assertRaises(ValueError, lambda: psdf.swapaxes(0, -1))
- @unittest.skipIf(is_ansi_mode_test, ansi_mode_not_supported_message)
def test_isin(self):
pdf = pd.DataFrame(
{
diff --git a/python/pyspark/pandas/tests/groupby/test_corr.py b/python/pyspark/pandas/tests/groupby/test_corr.py
index b44087c828090..39d6d91de4b0a 100644
--- a/python/pyspark/pandas/tests/groupby/test_corr.py
+++ b/python/pyspark/pandas/tests/groupby/test_corr.py
@@ -21,7 +21,6 @@
from pyspark import pandas as ps
from pyspark.testing.pandasutils import PandasOnSparkTestCase
from pyspark.testing.sqlutils import SQLTestUtils
-from pyspark.testing.utils import is_ansi_mode_test, ansi_mode_not_supported_message
class CorrMixin:
@@ -48,7 +47,6 @@ def test_corr(self):
almost=True,
)
- @unittest.skipIf(is_ansi_mode_test, ansi_mode_not_supported_message)
def test_method(self):
for m in ["pearson", "spearman", "kendall"]:
self.assert_eq(
diff --git a/python/pyspark/pandas/tests/indexes/test_category.py b/python/pyspark/pandas/tests/indexes/test_category.py
index 761e1100d8afa..acd80378333e8 100644
--- a/python/pyspark/pandas/tests/indexes/test_category.py
+++ b/python/pyspark/pandas/tests/indexes/test_category.py
@@ -21,7 +21,6 @@
from pandas.api.types import CategoricalDtype
import pyspark.pandas as ps
-from pyspark.loose_version import LooseVersion
from pyspark.testing.pandasutils import PandasOnSparkTestCase, TestUtils
@@ -202,18 +201,9 @@ def test_append(self):
psidx3 = ps.from_pandas(pidx3)
self.assert_eq(psidx1.append(psidx2), pidx1.append(pidx2))
- if LooseVersion(pd.__version__) >= LooseVersion("1.5.0"):
- self.assert_eq(
- psidx1.append(psidx3.astype("category")), pidx1.append(pidx3.astype("category"))
- )
- else:
- expected_result = ps.CategoricalIndex(
- ["x", "y", "z", "y", "x", "w", "z"],
- categories=["z", "y", "x", "w"],
- ordered=False,
- dtype="category",
- )
- self.assert_eq(psidx1.append(psidx3.astype("category")), expected_result)
+ self.assert_eq(
+ psidx1.append(psidx3.astype("category")), pidx1.append(pidx3.astype("category"))
+ )
# TODO: append non-categorical or categorical with a different category
self.assertRaises(NotImplementedError, lambda: psidx1.append(psidx3))
diff --git a/python/pyspark/pandas/tests/indexes/test_conversion.py b/python/pyspark/pandas/tests/indexes/test_conversion.py
index 9759a3d06a759..3ccc9697070fc 100644
--- a/python/pyspark/pandas/tests/indexes/test_conversion.py
+++ b/python/pyspark/pandas/tests/indexes/test_conversion.py
@@ -20,9 +20,13 @@
import pandas as pd
from pyspark import pandas as ps
-from pyspark.loose_version import LooseVersion
-from pyspark.testing.pandasutils import PandasOnSparkTestCase, SPARK_CONF_ARROW_ENABLED
+from pyspark.testing.pandasutils import PandasOnSparkTestCase
+from pyspark.pandas.utils import (
+ SPARK_CONF_ARROW_ENABLED,
+ SPARK_CONF_PANDAS_STRUCT_MODE,
+)
from pyspark.testing.sqlutils import SQLTestUtils
+from pyspark.testing.utils import is_ansi_mode_test
class ConversionMixin:
@@ -102,15 +106,10 @@ def test_multi_index_from_index(self):
self.assert_eq(pmidx, psmidx)
# Specify the `names`
- # Specify the `names` while Index creating is no longer supported from pandas 2.0.0.
- if LooseVersion(pd.__version__) >= LooseVersion("2.0.0"):
- pmidx = pd.Index(tuples)
- pmidx.names = ["Hello", "Koalas"]
- psmidx = ps.Index(tuples)
- psmidx.names = ["Hello", "Koalas"]
- else:
- pmidx = pd.Index(tuples, names=["Hello", "Koalas"])
- psmidx = ps.Index(tuples, names=["Hello", "Koalas"])
+ pmidx = pd.Index(tuples)
+ pmidx.names = ["Hello", "Koalas"]
+ psmidx = ps.Index(tuples)
+ psmidx.names = ["Hello", "Koalas"]
self.assertTrue(isinstance(psmidx, ps.MultiIndex))
self.assert_eq(pmidx, psmidx)
@@ -191,10 +190,28 @@ def test_to_series(self):
self.assert_eq((psidx + 1).to_series(), (pidx + 1).to_series())
+ # Multiindex
+ arrays = [[1, 2], ["red", "blue"]]
+ pidx = pd.MultiIndex.from_arrays(arrays, names=("number", "color"))
+ psidx = ps.from_pandas(pidx)
+
+ if is_ansi_mode_test:
+ with self.sql_conf(
+ {
+ SPARK_CONF_PANDAS_STRUCT_MODE: "row",
+ }
+ ):
+ self.assert_eq(
+ list(psidx.to_series().values),
+ list(pidx.to_series().values),
+ )
+ else:
+ self.assert_eq(list(psidx.to_series().values), [["1", "red"], ["2", "blue"]])
+
pidx = self.pdf.set_index("b", append=True).index
psidx = self.psdf.set_index("b", append=True).index
- with self.sql_conf({SPARK_CONF_ARROW_ENABLED: False}):
+ with self.sql_conf({SPARK_CONF_ARROW_ENABLED: False, SPARK_CONF_PANDAS_STRUCT_MODE: "row"}):
self.assert_eq(psidx.to_series(), pidx.to_series(), check_exact=False)
self.assert_eq(psidx.to_series(name="a"), pidx.to_series(name="a"), check_exact=False)
@@ -243,36 +260,30 @@ def test_to_frame(self):
# non-string names
self.assert_eq(psidx.to_frame(name=[10, 20]), pidx.to_frame(name=[10, 20]))
self.assert_eq(psidx.to_frame(name=("x", 10)), pidx.to_frame(name=("x", 10)))
- if LooseVersion(pd.__version__) < LooseVersion("1.5.0"):
- self.assert_eq(
- psidx.to_frame(name=[("x", 10), ("y", 20)]),
- pidx.to_frame(name=[("x", 10), ("y", 20)]),
- )
- else:
- # Since pandas 1.5.0, the result is changed as below:
- # (x, 10) (y, 20)
- # b
- # 0 4 0 4
- # 1 5 1 5
- # 3 6 3 6
- # 5 3 5 3
- # 6 2 6 2
- # 8 1 8 1
- # 9 0 9 0
- # 0 9 0
- # 0 9 0
- #
- # The columns should be `Index([('x', 20), ('y', 20)], dtype='object')`,
- # but pandas API on Spark doesn't support such a way for creating Index.
- # So, we currently cannot follow the behavior of pandas.
- expected_result = ps.DataFrame(
- {("x", 10): [0, 1, 3, 5, 6, 8, 9, 9, 9], ("y", 20): [4, 5, 6, 3, 2, 1, 0, 0, 0]},
- index=ps.MultiIndex.from_tuples(
- [(0, 4), (1, 5), (3, 6), (5, 3), (6, 2), (8, 1), (9, 0), (9, 0), (9, 0)],
- names=[None, "b"],
- ),
- )
- self.assert_eq(psidx.to_frame(name=[("x", 10), ("y", 20)]), expected_result)
+ # Since pandas 1.5.0, the result is changed as below:
+ # (x, 10) (y, 20)
+ # b
+ # 0 4 0 4
+ # 1 5 1 5
+ # 3 6 3 6
+ # 5 3 5 3
+ # 6 2 6 2
+ # 8 1 8 1
+ # 9 0 9 0
+ # 0 9 0
+ # 0 9 0
+ #
+ # The columns should be `Index([('x', 20), ('y', 20)], dtype='object')`,
+ # but pandas API on Spark doesn't support such a way for creating Index.
+ # So, we currently cannot follow the behavior of pandas.
+ expected_result = ps.DataFrame(
+ {("x", 10): [0, 1, 3, 5, 6, 8, 9, 9, 9], ("y", 20): [4, 5, 6, 3, 2, 1, 0, 0, 0]},
+ index=ps.MultiIndex.from_tuples(
+ [(0, 4), (1, 5), (3, 6), (5, 3), (6, 2), (8, 1), (9, 0), (9, 0), (9, 0)],
+ names=[None, "b"],
+ ),
+ )
+ self.assert_eq(psidx.to_frame(name=[("x", 10), ("y", 20)]), expected_result)
def test_to_list(self):
# Index
diff --git a/python/pyspark/pandas/tests/indexes/test_insert.py b/python/pyspark/pandas/tests/indexes/test_insert.py
index a95c6ee194f54..71fd2f6ad04f6 100644
--- a/python/pyspark/pandas/tests/indexes/test_insert.py
+++ b/python/pyspark/pandas/tests/indexes/test_insert.py
@@ -27,14 +27,8 @@ def test_insert(self):
# Integer
pidx = pd.Index([1, 2, 3], name="Koalas")
psidx = ps.from_pandas(pidx)
- self.assert_eq(
- pidx.insert(1, 100).sort_values(),
- psidx.insert(1, 100).sort_values(),
- )
- self.assert_eq(
- pidx.insert(-1, 100).sort_values(),
- psidx.insert(-1, 100).sort_values(),
- )
+ self.assert_eq(pidx.insert(1, 100), psidx.insert(1, 100))
+ self.assert_eq(pidx.insert(-1, 100), psidx.insert(-1, 100))
err_msg = "index 100 is out of bounds for axis 0 with size 3"
with self.assertRaisesRegex(IndexError, err_msg):
psidx.insert(100, 100)
@@ -45,14 +39,8 @@ def test_insert(self):
# Floating
pidx = pd.Index([1.0, 2.0, 3.0], name="Koalas")
psidx = ps.from_pandas(pidx)
- self.assert_eq(
- pidx.insert(1, 100.0).sort_values(),
- psidx.insert(1, 100.0).sort_values(),
- )
- self.assert_eq(
- pidx.insert(-1, 100.0).sort_values(),
- psidx.insert(-1, 100.0).sort_values(),
- )
+ self.assert_eq(pidx.insert(1, 100.0), psidx.insert(1, 100.0))
+ self.assert_eq(pidx.insert(-1, 100.0), psidx.insert(-1, 100.0))
err_msg = "index 100 is out of bounds for axis 0 with size 3"
with self.assertRaisesRegex(IndexError, err_msg):
psidx.insert(100, 100)
@@ -63,14 +51,8 @@ def test_insert(self):
# String
pidx = pd.Index(["a", "b", "c"], name="Koalas")
psidx = ps.from_pandas(pidx)
- self.assert_eq(
- pidx.insert(1, "x").sort_values(),
- psidx.insert(1, "x").sort_values(),
- )
- self.assert_eq(
- pidx.insert(-1, "x").sort_values(),
- psidx.insert(-1, "x").sort_values(),
- )
+ self.assert_eq(pidx.insert(1, "x"), psidx.insert(1, "x"))
+ self.assert_eq(pidx.insert(-1, "x"), psidx.insert(-1, "x"))
err_msg = "index 100 is out of bounds for axis 0 with size 3"
with self.assertRaisesRegex(IndexError, err_msg):
psidx.insert(100, "x")
@@ -81,14 +63,8 @@ def test_insert(self):
# Boolean
pidx = pd.Index([True, False, True, False], name="Koalas")
psidx = ps.from_pandas(pidx)
- self.assert_eq(
- pidx.insert(1, True).sort_values(),
- psidx.insert(1, True).sort_values(),
- )
- self.assert_eq(
- pidx.insert(-1, True).sort_values(),
- psidx.insert(-1, True).sort_values(),
- )
+ self.assert_eq(pidx.insert(1, True), psidx.insert(1, True))
+ self.assert_eq(pidx.insert(-1, True), psidx.insert(-1, True))
err_msg = "index 100 is out of bounds for axis 0 with size 4"
with self.assertRaisesRegex(IndexError, err_msg):
psidx.insert(100, True)
@@ -101,14 +77,8 @@ def test_insert(self):
[("a", "x"), ("b", "y"), ("c", "z")], names=["Hello", "Koalas"]
)
psmidx = ps.from_pandas(pmidx)
- self.assert_eq(
- pmidx.insert(2, ("h", "j")).sort_values(),
- psmidx.insert(2, ("h", "j")).sort_values(),
- )
- self.assert_eq(
- pmidx.insert(-1, ("h", "j")).sort_values(),
- psmidx.insert(-1, ("h", "j")).sort_values(),
- )
+ self.assert_eq(pmidx.insert(2, ("h", "j")), psmidx.insert(2, ("h", "j")))
+ self.assert_eq(pmidx.insert(-1, ("h", "j")), psmidx.insert(-1, ("h", "j")))
err_msg = "index 4 is out of bounds for axis 0 with size 3"
with self.assertRaisesRegex(IndexError, err_msg):
diff --git a/python/pyspark/pandas/tests/indexes/test_name.py b/python/pyspark/pandas/tests/indexes/test_name.py
index cacf3efcb38b1..d7e3bf1786e8a 100644
--- a/python/pyspark/pandas/tests/indexes/test_name.py
+++ b/python/pyspark/pandas/tests/indexes/test_name.py
@@ -20,7 +20,6 @@
import pandas as pd
from pyspark import pandas as ps
-from pyspark.loose_version import LooseVersion
from pyspark.pandas.exceptions import PandasNotImplementedError
from pyspark.testing.pandasutils import PandasOnSparkTestCase
from pyspark.testing.sqlutils import SQLTestUtils
@@ -88,12 +87,6 @@ def test_index_names(self):
psidx.name = ["renamed"]
with self.assertRaisesRegex(TypeError, expected_error_message):
psidx.name = ["0", "1"]
- # Specifying `names` when creating Index is no longer supported from pandas 2.0.0.
- if LooseVersion(pd.__version__) >= LooseVersion("2.0.0"):
- pass
- else:
- with self.assertRaisesRegex(TypeError, expected_error_message):
- ps.Index([(1, 2), (3, 4)], names=["a", ["b"]])
def test_multi_index_names(self):
arrays = [[1, 1, 2, 2], ["red", "blue", "red", "blue"]]
diff --git a/python/pyspark/pandas/tests/indexes/test_symmetric_diff.py b/python/pyspark/pandas/tests/indexes/test_symmetric_diff.py
index 2eca8cf4a4312..310fea2035926 100644
--- a/python/pyspark/pandas/tests/indexes/test_symmetric_diff.py
+++ b/python/pyspark/pandas/tests/indexes/test_symmetric_diff.py
@@ -20,7 +20,6 @@
import pandas as pd
import pyspark.pandas as ps
-from pyspark.loose_version import LooseVersion
from pyspark.testing.pandasutils import PandasOnSparkTestCase, TestUtils
@@ -39,17 +38,10 @@ def test_index_symmetric_difference(self):
(psidx1 + 1).symmetric_difference(psidx2).sort_values(),
(pidx1 + 1).symmetric_difference(pidx2).sort_values(),
)
- # No longer supported from pandas 2.0.0.
- if LooseVersion(pd.__version__) >= LooseVersion("2.0.0"):
- self.assert_eq(
- (psidx1 ^ psidx2).sort_values(),
- ps.Index([1, 5], dtype="int64"),
- )
- else:
- self.assert_eq(
- (psidx1 ^ psidx2).sort_values(),
- (pidx1 ^ pidx2).sort_values(),
- )
+ self.assert_eq(
+ (psidx1 ^ psidx2).sort_values(),
+ ps.Index([1, 5], dtype="int64"),
+ )
self.assert_eq(
psidx1.symmetric_difference(psidx2, result_name="result").sort_values(),
pidx1.symmetric_difference(pidx2, result_name="result").sort_values(),
diff --git a/python/pyspark/pandas/tests/io/test_dataframe_spark_io.py b/python/pyspark/pandas/tests/io/test_dataframe_spark_io.py
index af77ea8aa64ff..065a0e8d6ecd5 100644
--- a/python/pyspark/pandas/tests/io/test_dataframe_spark_io.py
+++ b/python/pyspark/pandas/tests/io/test_dataframe_spark_io.py
@@ -21,7 +21,6 @@
import pandas as pd
from pyspark import pandas as ps
-from pyspark.loose_version import LooseVersion
from pyspark.testing.pandasutils import PandasOnSparkTestCase, TestUtils
from pyspark.testing.utils import have_openpyxl, openpyxl_requirement_message
@@ -96,17 +95,9 @@ def test_parquet_read_with_pandas_metadata(self):
self.assert_eq(ps.read_parquet(path2, pandas_metadata=True), expected2)
expected3 = expected2.set_index("index", append=True)
- # There is a bug in `to_parquet` from pandas 1.5.0 when writing MultiIndex.
- # See https://github.com/pandas-dev/pandas/issues/48848 for the reported issue.
- if LooseVersion(pd.__version__) > LooseVersion("1.5.0"):
- expected_psdf = ps.read_parquet(path2, pandas_metadata=True).set_index(
- "index", append=True
- )
- else:
- path3 = "{}/file3.parquet".format(tmp)
- expected3.to_parquet(path3)
- expected_psdf = ps.read_parquet(path3, pandas_metadata=True)
-
+ expected_psdf = ps.read_parquet(path2, pandas_metadata=True).set_index(
+ "index", append=True
+ )
self.assert_eq(expected_psdf, expected3)
def test_parquet_write(self):
diff --git a/python/pyspark/pandas/tests/series/test_as_type.py b/python/pyspark/pandas/tests/series/test_as_type.py
index dfd66ee05d382..da9a399abf7be 100644
--- a/python/pyspark/pandas/tests/series/test_as_type.py
+++ b/python/pyspark/pandas/tests/series/test_as_type.py
@@ -22,6 +22,7 @@
from pyspark import pandas as ps
from pyspark.testing.pandasutils import PandasOnSparkTestCase
from pyspark.testing.sqlutils import SQLTestUtils
+from pyspark.testing.utils import is_ansi_mode_test
from pyspark.pandas.typedef.typehints import (
extension_dtypes_available,
extension_float_dtypes_available,
@@ -31,6 +32,7 @@
class SeriesAsTypeMixin:
def test_astype(self):
+ # numeric
psers = [pd.Series([10, 20, 15, 30, 45], name="x")]
if extension_dtypes_available:
@@ -41,12 +43,14 @@ def test_astype(self):
for pser in psers:
self._test_numeric_astype(pser)
+ # numeric with nulls
pser = pd.Series([10, 20, 15, 30, 45, None, np.nan], name="x")
psser = ps.Series(pser)
self.assert_eq(psser.astype(bool), pser.astype(bool))
self.assert_eq(psser.astype(str), pser.astype(str))
+ # strings
pser = pd.Series(["hi", "hi ", " ", " \t", "", None], name="x")
psser = ps.Series(pser)
@@ -60,12 +64,16 @@ def test_astype(self):
self._check_extension(psser.astype("string"), pser.astype("string"))
self._check_extension(psser.astype(StringDtype()), pser.astype(StringDtype()))
+ # bools
pser = pd.Series([True, False, None], name="x")
psser = ps.Series(pser)
-
self.assert_eq(psser.astype(bool), pser.astype(bool))
self.assert_eq(psser.astype(str), pser.astype(str))
+ if is_ansi_mode_test:
+ with self.assertRaisesRegex(ValueError, "with missing values to integer"):
+ self.assert_eq(psser.astype(int))
+
if extension_object_dtypes_available:
from pandas import BooleanDtype, StringDtype
@@ -74,6 +82,7 @@ def test_astype(self):
self._check_extension(psser.astype("string"), pser.astype("string"))
self._check_extension(psser.astype(StringDtype()), pser.astype(StringDtype()))
+ # datetimes
pser = pd.Series(["2020-10-27 00:00:01", None], name="x")
psser = ps.Series(pser)
diff --git a/python/pyspark/pandas/tests/series/test_series.py b/python/pyspark/pandas/tests/series/test_series.py
index 7409bcc26c29d..26045b5beb947 100644
--- a/python/pyspark/pandas/tests/series/test_series.py
+++ b/python/pyspark/pandas/tests/series/test_series.py
@@ -30,7 +30,6 @@
SPARK_CONF_ARROW_ENABLED,
)
from pyspark.testing.sqlutils import SQLTestUtils
-from pyspark.testing.utils import is_ansi_mode_test, ansi_mode_not_supported_message
from pyspark.pandas.exceptions import PandasNotImplementedError
from pyspark.pandas.missing.series import MissingPandasLikeSeries
from pyspark.pandas.typedef.typehints import extension_object_dtypes_available
@@ -103,6 +102,29 @@ def test_empty_series(self):
self.assertTrue(pser_a.empty)
+ def test_series_from_series(self):
+ psser = ps.Series([1, 2, 3, 4, 5, 6, 7], name="x")
+
+ psser_from_psser = ps.Series(psser)
+ self.assert_eq(psser_from_psser, psser)
+
+ psser = ps.Series([1, 2, 3])
+
+ # Specify new index
+ psser_from_psser = ps.Series(psser, index=[1])
+ self.assert_eq(psser_from_psser, ps.Series([2], index=[1]))
+
+ psser_from_psser = ps.Series(psser, index=[1, 2])
+ self.assert_eq(psser_from_psser, ps.Series([2, 3], index=[1, 2]))
+
+ # Specify new out-of-order index
+ psser_from_psser = ps.Series(psser, index=[1, 2, 0])
+ self.assert_eq(psser_from_psser, ps.Series([2, 3, 1], index=[1, 2, 0]))
+
+ # Specify new dtype and name
+ psser_from_psser = ps.Series(psser, name="y", dtype=float)
+ self.assert_eq(psser_from_psser, ps.Series([1, 2, 3], name="y", dtype=float))
+
def test_all_null_series(self):
pser_a = pd.Series([None, None, None], dtype="float64")
pser_b = pd.Series([None, None, None], dtype="str")
@@ -662,7 +684,6 @@ def test_items(self):
self.assert_eq(p_name, k_name)
self.assert_eq(p_items, k_items)
- @unittest.skipIf(is_ansi_mode_test, ansi_mode_not_supported_message)
def test_dot(self):
pdf = pd.DataFrame({"a": [1, 2, 3], "b": [4, 5, 6]})
psdf = ps.from_pandas(pdf)
diff --git a/python/pyspark/pandas/tests/series/test_stat.py b/python/pyspark/pandas/tests/series/test_stat.py
index a779ca558da65..077a8e31d20ad 100644
--- a/python/pyspark/pandas/tests/series/test_stat.py
+++ b/python/pyspark/pandas/tests/series/test_stat.py
@@ -23,7 +23,6 @@
from pyspark import pandas as ps
from pyspark.testing.pandasutils import PandasOnSparkTestCase
from pyspark.testing.sqlutils import SQLTestUtils
-from pyspark.testing.utils import is_ansi_mode_test, ansi_mode_not_supported_message
class SeriesStatMixin:
@@ -443,7 +442,6 @@ def test_rdivmod(self):
self.assert_eq(krdiv, prdiv)
self.assert_eq(krmod, prmod)
- @unittest.skipIf(is_ansi_mode_test, ansi_mode_not_supported_message)
def test_mod(self):
pser = pd.Series([100, None, -300, None, 500, -700], name="Koalas")
psser = ps.from_pandas(pser)
@@ -608,6 +606,9 @@ def test_autocorr(self):
with self.assertRaisesRegex(TypeError, r"lag should be an int; however, got"):
psser.autocorr(1.0)
+ psser = ps.Series([1, 0, 0, 0])
+ self.assertTrue(bool(np.isnan(psser.autocorr())))
+
def _test_autocorr(self, pdf):
psdf = ps.from_pandas(pdf)
for lag in range(-10, 10):
diff --git a/python/pyspark/pandas/tests/test_categorical.py b/python/pyspark/pandas/tests/test_categorical.py
index 2730723b27e35..49cbcee47b80c 100644
--- a/python/pyspark/pandas/tests/test_categorical.py
+++ b/python/pyspark/pandas/tests/test_categorical.py
@@ -91,7 +91,6 @@ def test_add_categories(self):
self.assert_eq(pser.cat.add_categories([4, 5]), psser.cat.add_categories([4, 5]))
self.assert_eq(pser.cat.add_categories([]), psser.cat.add_categories([]))
- pser = pser.cat.add_categories(4)
psser = psser.cat.add_categories(4)
self.assertRaises(ValueError, lambda: psser.cat.add_categories(4))
diff --git a/python/pyspark/pandas/tests/test_namespace.py b/python/pyspark/pandas/tests/test_namespace.py
index c86cb17c0e5d0..141c6873d7f59 100644
--- a/python/pyspark/pandas/tests/test_namespace.py
+++ b/python/pyspark/pandas/tests/test_namespace.py
@@ -29,7 +29,6 @@
from pyspark.pandas.missing.general_functions import MissingPandasLikeGeneralFunctions
from pyspark.testing.pandasutils import PandasOnSparkTestCase
from pyspark.testing.sqlutils import SQLTestUtils
-from pyspark.testing.utils import is_ansi_mode_test, ansi_mode_not_supported_message
from pyspark.pandas.testing import assert_frame_equal
@@ -561,7 +560,6 @@ def test_read_delta_with_wrong_input(self):
lambda: read_delta("fake_path", version="0", timestamp="2021-06-22"),
)
- @unittest.skipIf(is_ansi_mode_test, ansi_mode_not_supported_message)
def test_to_numeric(self):
pser = pd.Series(["1", "2", None, "4", "hello"])
psser = ps.from_pandas(pser)
diff --git a/python/pyspark/pandas/tests/test_numpy_compat.py b/python/pyspark/pandas/tests/test_numpy_compat.py
index 39786c6ac2286..f754ee08a7835 100644
--- a/python/pyspark/pandas/tests/test_numpy_compat.py
+++ b/python/pyspark/pandas/tests/test_numpy_compat.py
@@ -23,10 +23,15 @@
from pyspark.pandas import set_option, reset_option
from pyspark.testing.pandasutils import PandasOnSparkTestCase
from pyspark.testing.sqlutils import SQLTestUtils
-from pyspark.testing.utils import is_ansi_mode_test, ansi_mode_not_supported_message
class NumPyCompatTestsMixin:
+ @classmethod
+ def setUpClass(cls):
+ super(NumPyCompatTestsMixin, cls).setUpClass()
+ # Some nanosecond->microsecond conversions throw loss of precision errors
+ cls.spark.conf.set("spark.sql.execution.pandas.convertToArrowArraySafely", "false")
+
blacklist = [
# Pandas-on-Spark does not currently support
"conj",
@@ -132,7 +137,6 @@ def test_np_spark_compat_series(self):
finally:
reset_option("compute.ops_on_diff_frames")
- @unittest.skipIf(is_ansi_mode_test, ansi_mode_not_supported_message)
def test_np_spark_compat_frame(self):
from pyspark.pandas.numpy_compat import unary_np_spark_mappings, binary_np_spark_mappings
diff --git a/python/pyspark/pandas/tests/test_utils.py b/python/pyspark/pandas/tests/test_utils.py
index 26f571233a8bd..6286df8e54690 100644
--- a/python/pyspark/pandas/tests/test_utils.py
+++ b/python/pyspark/pandas/tests/test_utils.py
@@ -163,6 +163,60 @@ def test_index_error_assert_pandas_equal(self):
},
)
+ def test_dataframe_error_assert_pandas_almost_equal(self):
+ pdf1 = pd.DataFrame({"a": [1, 2, 3], "b": [4, 5, 6]})
+ pdf2 = pd.DataFrame({"a": [1, 3, 3], "b": [4, 5, 6]})
+
+ with self.assertRaises(PySparkAssertionError) as pe:
+ _assert_pandas_almost_equal(pdf1, pdf2, True)
+
+ self.check_error(
+ exception=pe.exception,
+ errorClass="DIFFERENT_PANDAS_DATAFRAME",
+ messageParameters={
+ "left": pdf1.to_string(),
+ "left_dtype": str(pdf1.dtypes),
+ "right": pdf2.to_string(),
+ "right_dtype": str(pdf2.dtypes),
+ },
+ )
+
+ def test_series_error_assert_pandas_equal(self):
+ series1 = pd.Series([1, 2, 3])
+ series2 = pd.Series([4, 5, 6])
+
+ with self.assertRaises(PySparkAssertionError) as pe:
+ _assert_pandas_almost_equal(series1, series2, True)
+
+ self.check_error(
+ exception=pe.exception,
+ errorClass="DIFFERENT_PANDAS_SERIES",
+ messageParameters={
+ "left": series1.to_string(),
+ "left_dtype": str(series1.dtype),
+ "right": series2.to_string(),
+ "right_dtype": str(series2.dtype),
+ },
+ )
+
+ def test_index_error_assert_pandas_almost_equal(self):
+ index1 = pd.Index([1, 2, 3])
+ index2 = pd.Index([4, 5, 6])
+
+ with self.assertRaises(PySparkAssertionError) as pe:
+ _assert_pandas_almost_equal(index1, index2, True)
+
+ self.check_error(
+ exception=pe.exception,
+ errorClass="DIFFERENT_PANDAS_INDEX",
+ messageParameters={
+ "left": index1,
+ "left_dtype": str(index1.dtype),
+ "right": index2,
+ "right_dtype": str(index2.dtype),
+ },
+ )
+
def test_multiindex_error_assert_pandas_almost_equal(self):
pdf1 = pd.DataFrame({"a": [1, 2], "b": [4, 10]}, index=[0, 1])
pdf2 = pd.DataFrame({"a": [1, 5, 3], "b": [1, 5, 6]}, index=[0, 1, 3])
diff --git a/python/pyspark/pandas/typedef/typehints.py b/python/pyspark/pandas/typedef/typehints.py
index 4244f5831aa50..48545d124b2d8 100644
--- a/python/pyspark/pandas/typedef/typehints.py
+++ b/python/pyspark/pandas/typedef/typehints.py
@@ -362,8 +362,9 @@ def infer_pd_series_spark_type(
if dtype == np.dtype("object"):
if len(pser) == 0 or pser.isnull().all():
return types.NullType()
- elif hasattr(pser.iloc[0], "__UDT__"):
- return pser.iloc[0].__UDT__
+ notnull = pser[pser.notnull()]
+ if hasattr(notnull.iloc[0], "__UDT__"):
+ return notnull.iloc[0].__UDT__
else:
return from_arrow_type(pa.Array.from_pandas(pser).type, prefer_timestamp_ntz)
elif isinstance(dtype, CategoricalDtype):
diff --git a/python/pyspark/pandas/utils.py b/python/pyspark/pandas/utils.py
index 23350c06a147e..6a0561fad9c5a 100644
--- a/python/pyspark/pandas/utils.py
+++ b/python/pyspark/pandas/utils.py
@@ -20,7 +20,9 @@
import functools
from contextlib import contextmanager
+import json
import os
+import threading
from typing import (
Any,
Callable,
@@ -69,6 +71,7 @@
SPARK_CONF_ARROW_ENABLED = "spark.sql.execution.arrow.pyspark.enabled"
+SPARK_CONF_PANDAS_STRUCT_MODE = "spark.sql.execution.pandas.structHandlingMode"
class PandasAPIOnSparkAdviceWarning(Warning):
@@ -1070,11 +1073,101 @@ def xor(df1: PySparkDataFrame, df2: PySparkDataFrame) -> PySparkDataFrame:
)
+_ansi_mode_enabled = threading.local()
+
+
+def _is_in_ansi_mode_context(spark: SparkSession) -> bool:
+ if is_remote():
+ from pyspark.sql.connect.session import SparkSession as ConnectSession
+
+ session_id = cast(ConnectSession, spark).session_id
+ return hasattr(_ansi_mode_enabled, session_id)
+ else:
+ return hasattr(_ansi_mode_enabled, "enabled")
+
+
+def _set_ansi_mode_enabled_in_context(spark: SparkSession, enabled: Optional[bool] = None) -> None:
+ if enabled is not None:
+ assert _is_in_ansi_mode_context(spark)
+
+ if is_remote():
+ from pyspark.sql.connect.session import SparkSession as ConnectSession
+
+ session_id = cast(ConnectSession, spark).session_id
+ setattr(_ansi_mode_enabled, session_id, enabled)
+ else:
+ _ansi_mode_enabled.enabled = enabled
+
+
+def _get_ansi_mode_enabled_in_context(spark: SparkSession) -> Optional[bool]:
+ assert _is_in_ansi_mode_context(spark)
+
+ if is_remote():
+ from pyspark.sql.connect.session import SparkSession as ConnectSession
+
+ session_id = cast(ConnectSession, spark).session_id
+ return getattr(_ansi_mode_enabled, session_id)
+ else:
+ return _ansi_mode_enabled.enabled
+
+
+def _unset_ansi_mode_enabled_in_context(spark: SparkSession) -> None:
+ assert _is_in_ansi_mode_context(spark)
+
+ if is_remote():
+ from pyspark.sql.connect.session import SparkSession as ConnectSession
+
+ session_id = cast(ConnectSession, spark).session_id
+ delattr(_ansi_mode_enabled, session_id)
+ else:
+ del _ansi_mode_enabled.enabled
+
+
+@contextmanager
+def ansi_mode_context(spark: SparkSession) -> Iterator[None]:
+ if _is_in_ansi_mode_context(spark):
+ yield
+ else:
+ _set_ansi_mode_enabled_in_context(spark)
+ try:
+ yield
+ finally:
+ _unset_ansi_mode_enabled_in_context(spark)
+
+
def is_ansi_mode_enabled(spark: SparkSession) -> bool:
- return (
- ps.get_option("compute.ansi_mode_support", spark_session=spark)
- and spark.conf.get("spark.sql.ansi.enabled") == "true"
- )
+ def _is_ansi_mode_enabled() -> bool:
+ if is_remote():
+ from pyspark.sql.connect.session import SparkSession as ConnectSession
+ from pyspark.pandas.config import _key_format, _options_dict
+
+ client = cast(ConnectSession, spark).client
+ (ansi_mode_support, ansi_enabled) = client.get_config_with_defaults(
+ (
+ _key_format("compute.ansi_mode_support"),
+ json.dumps(_options_dict["compute.ansi_mode_support"].default),
+ ),
+ ("spark.sql.ansi.enabled", None),
+ )
+ if ansi_enabled is None:
+ ansi_enabled = spark.conf.get("spark.sql.ansi.enabled")
+ # Explicitly set the default value to reduce the roundtrip for the next time.
+ spark.conf.set("spark.sql.ansi.enabled", ansi_enabled)
+ return json.loads(ansi_mode_support) and ansi_enabled.lower() == "true"
+ else:
+ return (
+ ps.get_option("compute.ansi_mode_support", spark_session=spark)
+ and spark.conf.get("spark.sql.ansi.enabled").lower() == "true"
+ )
+
+ if _is_in_ansi_mode_context(spark):
+ enabled = _get_ansi_mode_enabled_in_context(spark)
+ if enabled is None:
+ enabled = _is_ansi_mode_enabled()
+ _set_ansi_mode_enabled_in_context(spark, enabled)
+ return enabled
+ else:
+ return _is_ansi_mode_enabled()
def _test() -> None:
diff --git a/python/pyspark/pipelines/api.py b/python/pyspark/pipelines/api.py
index 57de64e970351..35beff19c981d 100644
--- a/python/pyspark/pipelines/api.py
+++ b/python/pyspark/pipelines/api.py
@@ -35,21 +35,16 @@ def append_flow(
*,
target: str,
name: Optional[str] = None,
- comment: Optional[str] = None,
spark_conf: Optional[Dict[str, str]] = None,
- once: bool = False,
) -> Callable[[QueryFunction], None]:
"""
Return a decorator on a query function to define a flow in a pipeline.
:param name: The name of the flow. If unspecified, the query function's name will be used.
:param target: The name of the dataset this flow writes to. Must be specified.
- :param comment: Description of the flow. If unspecified, the dataset's comment will be used.
:param spark_conf: A dict whose keys are the conf names and values are the conf values. \
These confs will be set when the flow is executed; they can override confs set for the \
destination, for the pipeline, or on the cluster.
- :param once: If True, indicates this flow should run only once. (It will be rerun upon a full \
- refresh operation.)
"""
if name is not None and type(name) is not str:
raise PySparkTypeError(
@@ -69,7 +64,6 @@ def outer(func: QueryFunction) -> None:
target=target,
spark_conf=spark_conf,
source_code_location=source_code_location,
- once=once,
func=func,
)
get_active_graph_element_registry().register_flow(flow)
@@ -179,7 +173,6 @@ def outer(
target=resolved_name,
spark_conf=spark_conf or {},
source_code_location=source_code_location,
- once=False,
func=decorated,
)
)
@@ -282,7 +275,6 @@ def outer(
target=resolved_name,
spark_conf=spark_conf or {},
source_code_location=source_code_location,
- once=False,
func=decorated,
)
)
@@ -373,7 +365,6 @@ def outer(decorated: QueryFunction) -> None:
spark_conf=spark_conf or {},
name=resolved_name,
source_code_location=source_code_location,
- once=False,
)
)
@@ -409,7 +400,6 @@ def create_streaming_table(
name: str,
*,
comment: Optional[str] = None,
- spark_conf: Optional[Dict[str, str]] = None,
table_properties: Optional[Dict[str, str]] = None,
partition_cols: Optional[List[str]] = None,
schema: Optional[Union[StructType, str]] = None,
@@ -423,9 +413,6 @@ def create_streaming_table(
:param name: The name of the table.
:param comment: Description of the table.
- :param spark_conf: A dict whose keys are the conf names and values are the conf values. \
- These confs will be set when the query for the dataset is executed and they can override \
- confs set for the pipeline or on the cluster.
:param table_properties: A dict where the keys are the property names and the values are the \
property values. These properties will be set on the table.
:param partition_cols: A list containing the column names of the partition columns.
diff --git a/python/pyspark/pipelines/block_session_mutations.py b/python/pyspark/pipelines/block_session_mutations.py
new file mode 100644
index 0000000000000..df63d2023a4ba
--- /dev/null
+++ b/python/pyspark/pipelines/block_session_mutations.py
@@ -0,0 +1,135 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+from contextlib import contextmanager
+from typing import Generator, NoReturn, List, Callable
+
+from pyspark.errors import PySparkException
+from pyspark.sql.connect.catalog import Catalog
+from pyspark.sql.connect.conf import RuntimeConf
+from pyspark.sql.connect.dataframe import DataFrame
+from pyspark.sql.connect.udf import UDFRegistration
+
+# pyspark methods that should be blocked from executing in python pipeline definition files
+ERROR_CLASS = "SESSION_MUTATION_IN_DECLARATIVE_PIPELINE"
+BLOCKED_METHODS: List = [
+ {
+ "class": RuntimeConf,
+ "method": "set",
+ "error_sub_class": "SET_RUNTIME_CONF",
+ },
+ {
+ "class": Catalog,
+ "method": "setCurrentCatalog",
+ "error_sub_class": "SET_CURRENT_CATALOG",
+ },
+ {
+ "class": Catalog,
+ "method": "setCurrentDatabase",
+ "error_sub_class": "SET_CURRENT_DATABASE",
+ },
+ {
+ "class": Catalog,
+ "method": "dropTempView",
+ "error_sub_class": "DROP_TEMP_VIEW",
+ },
+ {
+ "class": Catalog,
+ "method": "dropGlobalTempView",
+ "error_sub_class": "DROP_GLOBAL_TEMP_VIEW",
+ },
+ {
+ "class": DataFrame,
+ "method": "createTempView",
+ "error_sub_class": "CREATE_TEMP_VIEW",
+ },
+ {
+ "class": DataFrame,
+ "method": "createOrReplaceTempView",
+ "error_sub_class": "CREATE_OR_REPLACE_TEMP_VIEW",
+ },
+ {
+ "class": DataFrame,
+ "method": "createGlobalTempView",
+ "error_sub_class": "CREATE_GLOBAL_TEMP_VIEW",
+ },
+ {
+ "class": DataFrame,
+ "method": "createOrReplaceGlobalTempView",
+ "error_sub_class": "CREATE_OR_REPLACE_GLOBAL_TEMP_VIEW",
+ },
+ {
+ "class": UDFRegistration,
+ "method": "register",
+ "error_sub_class": "REGISTER_UDF",
+ },
+ {
+ "class": UDFRegistration,
+ "method": "registerJavaFunction",
+ "error_sub_class": "REGISTER_JAVA_UDF",
+ },
+ {
+ "class": UDFRegistration,
+ "method": "registerJavaUDAF",
+ "error_sub_class": "REGISTER_JAVA_UDAF",
+ },
+]
+
+
+def _create_blocked_method(error_method_name: str, error_sub_class: str) -> Callable:
+ def blocked_method(*args: object, **kwargs: object) -> NoReturn:
+ raise PySparkException(
+ errorClass=f"{ERROR_CLASS}.{error_sub_class}",
+ messageParameters={
+ "method": error_method_name,
+ },
+ )
+
+ return blocked_method
+
+
+@contextmanager
+def block_session_mutations() -> Generator[None, None, None]:
+ """
+ Context manager that blocks imperative constructs found in a pipeline python definition file
+ See BLOCKED_METHODS above for a list
+ """
+ # Store original methods
+ original_methods = {}
+ for method_info in BLOCKED_METHODS:
+ cls = method_info["class"]
+ method_name = method_info["method"]
+ original_methods[(cls, method_name)] = getattr(cls, method_name)
+
+ try:
+ # Replace methods with blocked versions
+ for method_info in BLOCKED_METHODS:
+ cls = method_info["class"]
+ method_name = method_info["method"]
+ error_method_name = f"'{cls.__name__}.{method_name}'"
+ blocked_method = _create_blocked_method(
+ error_method_name, method_info["error_sub_class"]
+ )
+ setattr(cls, method_name, blocked_method)
+
+ yield
+ finally:
+ # Restore original methods
+ for method_info in BLOCKED_METHODS:
+ cls = method_info["class"]
+ method_name = method_info["method"]
+ original_method = original_methods[(cls, method_name)]
+ setattr(cls, method_name, original_method)
diff --git a/python/pyspark/pipelines/cli.py b/python/pyspark/pipelines/cli.py
index 824c2a5fff166..43f9ae150f3f3 100644
--- a/python/pyspark/pipelines/cli.py
+++ b/python/pyspark/pipelines/cli.py
@@ -28,10 +28,11 @@
import yaml
from dataclasses import dataclass
from pathlib import Path
-from typing import Any, Generator, Mapping, Optional, Sequence
+from typing import Any, Generator, List, Mapping, Optional, Sequence
from pyspark.errors import PySparkException, PySparkTypeError
from pyspark.sql import SparkSession
+from pyspark.pipelines.block_session_mutations import block_session_mutations
from pyspark.pipelines.graph_element_registry import (
graph_element_registration_context,
GraphElementRegistry,
@@ -61,12 +62,14 @@ class DefinitionsGlob:
class PipelineSpec:
"""Spec for a pipeline.
+ :param name: The name of the pipeline.
:param catalog: The default catalog to use for the pipeline.
:param database: The default database to use for the pipeline.
:param configuration: A dictionary of Spark configuration properties to set for the pipeline.
:param definitions: A list of glob patterns for finding pipeline definitions files.
"""
+ name: str
catalog: Optional[str]
database: Optional[str]
configuration: Mapping[str, str]
@@ -110,13 +113,23 @@ def load_pipeline_spec(spec_path: Path) -> PipelineSpec:
def unpack_pipeline_spec(spec_data: Mapping[str, Any]) -> PipelineSpec:
+ ALLOWED_FIELDS = {"name", "catalog", "database", "schema", "configuration", "definitions"}
+ REQUIRED_FIELDS = ["name"]
for key in spec_data.keys():
- if key not in ["catalog", "database", "schema", "configuration", "definitions"]:
+ if key not in ALLOWED_FIELDS:
raise PySparkException(
errorClass="PIPELINE_SPEC_UNEXPECTED_FIELD", messageParameters={"field_name": key}
)
+ for key in REQUIRED_FIELDS:
+ if key not in spec_data:
+ raise PySparkException(
+ errorClass="PIPELINE_SPEC_MISSING_REQUIRED_FIELD",
+ messageParameters={"field_name": key},
+ )
+
return PipelineSpec(
+ name=spec_data["name"],
catalog=spec_data.get("catalog"),
database=spec_data.get("database", spec_data.get("schema")),
configuration=validate_str_dict(spec_data.get("configuration", {}), "configuration"),
@@ -180,7 +193,8 @@ def register_definitions(
assert (
module_spec.loader is not None
), f"Module spec has no loader for {file}"
- module_spec.loader.exec_module(module)
+ with block_session_mutations():
+ module_spec.loader.exec_module(module)
elif file.suffix == ".sql":
log_with_curr_timestamp(f"Registering SQL file {file}...")
with file.open("r") as f:
@@ -205,17 +219,46 @@ def change_dir(path: Path) -> Generator[None, None, None]:
os.chdir(prev)
-def run(spec_path: Path, remote: str) -> None:
- """Run the pipeline defined with the given spec."""
+def run(
+ spec_path: Path,
+ full_refresh: Sequence[str],
+ full_refresh_all: bool,
+ refresh: Sequence[str],
+ dry: bool,
+) -> None:
+ """Run the pipeline defined with the given spec.
+
+ :param spec_path: Path to the pipeline specification file.
+ :param full_refresh: List of datasets to reset and recompute.
+ :param full_refresh_all: Perform a full graph reset and recompute.
+ :param refresh: List of datasets to update.
+ """
+ # Validate conflicting arguments
+ if full_refresh_all:
+ if full_refresh:
+ raise PySparkException(
+ errorClass="CONFLICTING_PIPELINE_REFRESH_OPTIONS",
+ messageParameters={
+ "conflicting_option": "--full_refresh",
+ },
+ )
+ if refresh:
+ raise PySparkException(
+ errorClass="CONFLICTING_PIPELINE_REFRESH_OPTIONS",
+ messageParameters={
+ "conflicting_option": "--refresh",
+ },
+ )
+
log_with_curr_timestamp(f"Loading pipeline spec from {spec_path}...")
spec = load_pipeline_spec(spec_path)
log_with_curr_timestamp("Creating Spark session...")
- spark_builder = SparkSession.builder.remote(remote)
+ spark_builder = SparkSession.builder
for key, value in spec.configuration.items():
spark_builder = spark_builder.config(key, value)
- spark = spark_builder.create()
+ spark = spark_builder.getOrCreate()
log_with_curr_timestamp("Creating dataflow graph...")
dataflow_graph_id = create_dataflow_graph(
@@ -230,23 +273,60 @@ def run(spec_path: Path, remote: str) -> None:
register_definitions(spec_path, registry, spec)
log_with_curr_timestamp("Starting run...")
- result_iter = start_run(spark, dataflow_graph_id)
+ result_iter = start_run(
+ spark,
+ dataflow_graph_id,
+ full_refresh=full_refresh,
+ full_refresh_all=full_refresh_all,
+ refresh=refresh,
+ dry=dry,
+ )
try:
handle_pipeline_events(result_iter)
finally:
spark.stop()
+def parse_table_list(value: str) -> List[str]:
+ """Parse a comma-separated list of table names, handling whitespace."""
+ return [table.strip() for table in value.split(",") if table.strip()]
+
+
if __name__ == "__main__":
parser = argparse.ArgumentParser(description="Pipeline CLI")
subparsers = parser.add_subparsers(dest="command", required=True)
# "run" subcommand
- run_parser = subparsers.add_parser("run", help="Run a pipeline.")
+ run_parser = subparsers.add_parser(
+ "run",
+ help="Run a pipeline. If no refresh options specified, "
+ "a default incremental update is performed.",
+ )
run_parser.add_argument("--spec", help="Path to the pipeline spec.")
run_parser.add_argument(
- "--remote", help="The Spark Connect remote to connect to.", required=True
+ "--full-refresh",
+ type=parse_table_list,
+ action="extend",
+ help="List of datasets to reset and recompute (comma-separated).",
+ default=[],
+ )
+ run_parser.add_argument(
+ "--full-refresh-all", action="store_true", help="Perform a full graph reset and recompute."
)
+ run_parser.add_argument(
+ "--refresh",
+ type=parse_table_list,
+ action="extend",
+ help="List of datasets to update (comma-separated).",
+ default=[],
+ )
+
+ # "dry-run" subcommand
+ dry_run_parser = subparsers.add_parser(
+ "dry-run",
+ help="Launch a run that just validates the graph and checks for errors.",
+ )
+ dry_run_parser.add_argument("--spec", help="Path to the pipeline spec.")
# "init" subcommand
init_parser = subparsers.add_parser(
@@ -261,9 +341,9 @@ def run(spec_path: Path, remote: str) -> None:
)
args = parser.parse_args()
- assert args.command in ["run", "init"]
+ assert args.command in ["run", "dry-run", "init"]
- if args.command == "run":
+ if args.command in ["run", "dry-run"]:
if args.spec is not None:
spec_path = Path(args.spec)
if not spec_path.is_file():
@@ -274,6 +354,22 @@ def run(spec_path: Path, remote: str) -> None:
else:
spec_path = find_pipeline_spec(Path.cwd())
- run(spec_path=spec_path, remote=args.remote)
+ if args.command == "run":
+ run(
+ spec_path=spec_path,
+ full_refresh=args.full_refresh,
+ full_refresh_all=args.full_refresh_all,
+ refresh=args.refresh,
+ dry=args.command == "dry-run",
+ )
+ else:
+ assert args.command == "dry-run"
+ run(
+ spec_path=spec_path,
+ full_refresh=[],
+ full_refresh_all=False,
+ refresh=[],
+ dry=True,
+ )
elif args.command == "init":
init(args.name)
diff --git a/python/pyspark/pipelines/dataset.py b/python/pyspark/pipelines/dataset.py
index 1355794f25be7..b46bc9f6e2b6b 100644
--- a/python/pyspark/pipelines/dataset.py
+++ b/python/pyspark/pipelines/dataset.py
@@ -64,7 +64,7 @@ class MaterializedView(Table):
@dataclass(frozen=True)
class StreamingTable(Table):
- """Definition of a materialized view in a pipeline dataflow graph. A streaming table is a
+ """Definition of a streaming table in a pipeline dataflow graph. A streaming table is a
table whose contents are produced by one or more streaming flows."""
diff --git a/python/pyspark/pipelines/flow.py b/python/pyspark/pipelines/flow.py
index c2f8599ebf9f5..7c499c0b36221 100644
--- a/python/pyspark/pipelines/flow.py
+++ b/python/pyspark/pipelines/flow.py
@@ -33,7 +33,6 @@ class Flow:
:param spark_conf: A dict where the keys are the Spark configuration property names and the
values are the property values. These properties will be set on the flow.
:param source_code_location: The location of the source code that created this flow.
- :param once: If True, the flow will be executed once per run.
:param func: The function that defines the flow. This function should return a DataFrame.
"""
@@ -41,5 +40,4 @@ class Flow:
target: str
spark_conf: Dict[str, str]
source_code_location: SourceCodeLocation
- once: bool
func: QueryFunction
diff --git a/python/pyspark/pipelines/init_cli.py b/python/pyspark/pipelines/init_cli.py
index 1be4f288fad03..227e5aa5deca6 100644
--- a/python/pyspark/pipelines/init_cli.py
+++ b/python/pyspark/pipelines/init_cli.py
@@ -18,6 +18,7 @@
from pathlib import Path
SPEC = """
+name: {{ name }}
definitions:
- glob:
include: transformations/**/*.py
@@ -25,12 +26,12 @@
include: transformations/**/*.sql
"""
-PYTHON_EXAMPLE = """from pyspark import pipelines as sdp
+PYTHON_EXAMPLE = """from pyspark import pipelines as dp
from pyspark.sql import DataFrame, SparkSession
spark = SparkSession.active()
-@sdp.materialized_view
+@dp.materialized_view
def example_python_materialized_view() -> DataFrame:
return spark.range(10)
"""
@@ -49,7 +50,7 @@ def init(name: str) -> None:
# Write the spec file to the project directory
spec_file = project_dir / "pipeline.yml"
with open(spec_file, "w") as f:
- f.write(SPEC)
+ f.write(SPEC.replace("{{ name }}", name))
# Create the transformations directory
transformations_dir = project_dir / "transformations"
diff --git a/python/pyspark/pipelines/spark_connect_graph_element_registry.py b/python/pyspark/pipelines/spark_connect_graph_element_registry.py
index 8bc4aeefd2264..020c7989138d4 100644
--- a/python/pyspark/pipelines/spark_connect_graph_element_registry.py
+++ b/python/pyspark/pipelines/spark_connect_graph_element_registry.py
@@ -93,9 +93,8 @@ def register_flow(self, flow: Flow) -> None:
dataflow_graph_id=self._dataflow_graph_id,
flow_name=flow.name,
target_dataset_name=flow.target,
- plan=relation,
+ relation=relation,
sql_conf=flow.spark_conf,
- once=flow.once,
)
command = pb2.Command()
command.pipeline_command.define_flow.CopyFrom(inner_command)
diff --git a/python/pyspark/pipelines/spark_connect_pipeline.py b/python/pyspark/pipelines/spark_connect_pipeline.py
index 12f43a236c289..61b72956e5ccc 100644
--- a/python/pyspark/pipelines/spark_connect_pipeline.py
+++ b/python/pyspark/pipelines/spark_connect_pipeline.py
@@ -15,7 +15,7 @@
# limitations under the License.
#
from datetime import timezone
-from typing import Any, Dict, Mapping, Iterator, Optional, cast
+from typing import Any, Dict, Mapping, Iterator, Optional, cast, Sequence
import pyspark.sql.connect.proto as pb2
from pyspark.sql import SparkSession
@@ -65,12 +65,28 @@ def handle_pipeline_events(iter: Iterator[Dict[str, Any]]) -> None:
log_with_provided_timestamp(event.message, dt)
-def start_run(spark: SparkSession, dataflow_graph_id: str) -> Iterator[Dict[str, Any]]:
+def start_run(
+ spark: SparkSession,
+ dataflow_graph_id: str,
+ full_refresh: Optional[Sequence[str]],
+ full_refresh_all: bool,
+ refresh: Optional[Sequence[str]],
+ dry: bool,
+) -> Iterator[Dict[str, Any]]:
"""Start a run of the dataflow graph in the Spark Connect server.
:param dataflow_graph_id: The ID of the dataflow graph to start.
+ :param full_refresh: List of datasets to reset and recompute.
+ :param full_refresh_all: Perform a full graph reset and recompute.
+ :param refresh: List of datasets to update.
"""
- inner_command = pb2.PipelineCommand.StartRun(dataflow_graph_id=dataflow_graph_id)
+ inner_command = pb2.PipelineCommand.StartRun(
+ dataflow_graph_id=dataflow_graph_id,
+ full_refresh_selection=full_refresh or [],
+ full_refresh_all=full_refresh_all,
+ refresh_selection=refresh or [],
+ dry=dry,
+ )
command = pb2.Command()
command.pipeline_command.start_run.CopyFrom(inner_command)
# Cast because mypy seems to think `spark`` is a function, not an object. Likely related to
diff --git a/python/pyspark/pipelines/tests/test_block_session_mutations.py b/python/pyspark/pipelines/tests/test_block_session_mutations.py
new file mode 100644
index 0000000000000..3384a5fcbfb35
--- /dev/null
+++ b/python/pyspark/pipelines/tests/test_block_session_mutations.py
@@ -0,0 +1,260 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+import unittest
+
+from pyspark.errors import PySparkException
+from pyspark.sql.types import StringType
+from pyspark.testing.connectutils import (
+ ReusedConnectTestCase,
+ should_test_connect,
+ connect_requirement_message,
+)
+
+if should_test_connect:
+ from pyspark.pipelines.block_session_mutations import (
+ block_session_mutations,
+ BLOCKED_METHODS,
+ ERROR_CLASS,
+ )
+
+
+@unittest.skipIf(not should_test_connect, connect_requirement_message or "Connect not available")
+class BlockImperativeConfSetConnectTests(ReusedConnectTestCase):
+ def test_blocks_runtime_conf_set(self):
+ """Test that spark.conf.set() is blocked."""
+ config = self.spark.conf
+
+ test_cases = [
+ ("spark.test.string", "string_value"),
+ ("spark.test.int", 42),
+ ("spark.test.bool", True),
+ ]
+
+ for key, value in test_cases:
+ with self.subTest(key=key, value=value):
+ with block_session_mutations():
+ with self.assertRaises(PySparkException) as context:
+ config.set(key, value)
+
+ self.assertEqual(
+ context.exception.getCondition(),
+ f"{ERROR_CLASS}.SET_RUNTIME_CONF",
+ )
+ self.assertIn("'RuntimeConf.set'", str(context.exception))
+
+ def test_blocks_catalog_set_current_catalog(self):
+ """Test that spark.catalog.setCurrentCatalog() is blocked."""
+ catalog = self.spark.catalog
+
+ with block_session_mutations():
+ with self.assertRaises(PySparkException) as context:
+ catalog.setCurrentCatalog("test_catalog")
+
+ self.assertEqual(
+ context.exception.getCondition(),
+ f"{ERROR_CLASS}.SET_CURRENT_CATALOG",
+ )
+ self.assertIn("'Catalog.setCurrentCatalog'", str(context.exception))
+
+ def test_blocks_catalog_set_current_database(self):
+ """Test that spark.catalog.setCurrentDatabase() is blocked."""
+ catalog = self.spark.catalog
+
+ with block_session_mutations():
+ with self.assertRaises(PySparkException) as context:
+ catalog.setCurrentDatabase("test_db")
+
+ self.assertEqual(
+ context.exception.getCondition(),
+ f"{ERROR_CLASS}.SET_CURRENT_DATABASE",
+ )
+ self.assertIn("'Catalog.setCurrentDatabase'", str(context.exception))
+
+ def test_blocks_catalog_drop_temp_view(self):
+ """Test that spark.catalog.dropTempView() is blocked."""
+ catalog = self.spark.catalog
+
+ with block_session_mutations():
+ with self.assertRaises(PySparkException) as context:
+ catalog.dropTempView("test_view")
+
+ self.assertEqual(
+ context.exception.getCondition(),
+ f"{ERROR_CLASS}.DROP_TEMP_VIEW",
+ )
+ self.assertIn("'Catalog.dropTempView'", str(context.exception))
+
+ def test_blocks_catalog_drop_global_temp_view(self):
+ """Test that spark.catalog.dropGlobalTempView() is blocked."""
+ catalog = self.spark.catalog
+
+ with block_session_mutations():
+ with self.assertRaises(PySparkException) as context:
+ catalog.dropGlobalTempView("test_view")
+
+ self.assertEqual(
+ context.exception.getCondition(),
+ f"{ERROR_CLASS}.DROP_GLOBAL_TEMP_VIEW",
+ )
+ self.assertIn("'Catalog.dropGlobalTempView'", str(context.exception))
+
+ def test_blocks_dataframe_create_temp_view(self):
+ """Test that DataFrame.createTempView() is blocked."""
+ df = self.spark.range(1)
+
+ with block_session_mutations():
+ with self.assertRaises(PySparkException) as context:
+ df.createTempView("test_view")
+
+ self.assertEqual(
+ context.exception.getCondition(),
+ f"{ERROR_CLASS}.CREATE_TEMP_VIEW",
+ )
+ self.assertIn("'DataFrame.createTempView'", str(context.exception))
+
+ def test_blocks_dataframe_create_or_replace_temp_view(self):
+ """Test that DataFrame.createOrReplaceTempView() is blocked."""
+ df = self.spark.range(1)
+
+ with block_session_mutations():
+ with self.assertRaises(PySparkException) as context:
+ df.createOrReplaceTempView("test_view")
+
+ self.assertEqual(
+ context.exception.getCondition(),
+ f"{ERROR_CLASS}.CREATE_OR_REPLACE_TEMP_VIEW",
+ )
+ self.assertIn("'DataFrame.createOrReplaceTempView'", str(context.exception))
+
+ def test_blocks_dataframe_create_global_temp_view(self):
+ """Test that DataFrame.createGlobalTempView() is blocked."""
+ df = self.spark.range(1)
+
+ with block_session_mutations():
+ with self.assertRaises(PySparkException) as context:
+ df.createGlobalTempView("test_view")
+
+ self.assertEqual(
+ context.exception.getCondition(),
+ f"{ERROR_CLASS}.CREATE_GLOBAL_TEMP_VIEW",
+ )
+ self.assertIn("'DataFrame.createGlobalTempView'", str(context.exception))
+
+ def test_blocks_dataframe_create_or_replace_global_temp_view(self):
+ """Test that DataFrame.createOrReplaceGlobalTempView() is blocked."""
+ df = self.spark.range(1)
+
+ with block_session_mutations():
+ with self.assertRaises(PySparkException) as context:
+ df.createOrReplaceGlobalTempView("test_view")
+
+ self.assertEqual(
+ context.exception.getCondition(),
+ f"{ERROR_CLASS}.CREATE_OR_REPLACE_GLOBAL_TEMP_VIEW",
+ )
+ self.assertIn("'DataFrame.createOrReplaceGlobalTempView'", str(context.exception))
+
+ def test_blocks_udf_register(self):
+ """Test that spark.udf.register() is blocked."""
+ udf_registry = self.spark.udf
+
+ def test_func(x):
+ return x + 1
+
+ with block_session_mutations():
+ with self.assertRaises(PySparkException) as context:
+ udf_registry.register("test_udf", test_func, StringType())
+
+ self.assertEqual(
+ context.exception.getCondition(),
+ f"{ERROR_CLASS}.REGISTER_UDF",
+ )
+ self.assertIn("'UDFRegistration.register'", str(context.exception))
+
+ def test_blocks_udf_register_java_function(self):
+ """Test that spark.udf.registerJavaFunction() is blocked."""
+ udf_registry = self.spark.udf
+
+ with block_session_mutations():
+ with self.assertRaises(PySparkException) as context:
+ udf_registry.registerJavaFunction(
+ "test_java_udf", "com.example.TestUDF", StringType()
+ )
+
+ self.assertEqual(
+ context.exception.getCondition(),
+ f"{ERROR_CLASS}.REGISTER_JAVA_UDF",
+ )
+ self.assertIn("'UDFRegistration.registerJavaFunction'", str(context.exception))
+
+ def test_blocks_udf_register_java_udaf(self):
+ """Test that spark.udf.registerJavaUDAF() is blocked."""
+ udf_registry = self.spark.udf
+
+ with block_session_mutations():
+ with self.assertRaises(PySparkException) as context:
+ udf_registry.registerJavaUDAF("test_java_udaf", "com.example.TestUDAF")
+
+ self.assertEqual(
+ context.exception.getCondition(),
+ f"{ERROR_CLASS}.REGISTER_JAVA_UDAF",
+ )
+ self.assertIn("'UDFRegistration.registerJavaUDAF'", str(context.exception))
+
+ def test_restores_original_methods_after_context(self):
+ """Test that all methods are properly restored after context manager exits."""
+ # Store original methods
+ original_methods = {}
+ for method_info in BLOCKED_METHODS:
+ cls = method_info["class"]
+ method_name = method_info["method"]
+ original_methods[(cls, method_name)] = getattr(cls, method_name)
+
+ # Verify methods are originally set correctly
+ for method_info in BLOCKED_METHODS:
+ cls = method_info["class"]
+ method_name = method_info["method"]
+ with self.subTest(class_method=f"{cls.__name__}.{method_name}"):
+ self.assertIs(getattr(cls, method_name), original_methods[(cls, method_name)])
+
+ # Verify methods are replaced during context
+ with block_session_mutations():
+ for method_info in BLOCKED_METHODS:
+ cls = method_info["class"]
+ method_name = method_info["method"]
+ with self.subTest(class_method=f"{cls.__name__}.{method_name}"):
+ self.assertIsNot(
+ getattr(cls, method_name), original_methods[(cls, method_name)]
+ )
+
+ # Verify methods are restored after context
+ for method_info in BLOCKED_METHODS:
+ cls = method_info["class"]
+ method_name = method_info["method"]
+ with self.subTest(class_method=f"{cls.__name__}.{method_name}"):
+ self.assertIs(getattr(cls, method_name), original_methods[(cls, method_name)])
+
+
+if __name__ == "__main__":
+ try:
+ import xmlrunner # type: ignore
+
+ testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2)
+ except ImportError:
+ testRunner = None
+ unittest.main(testRunner=testRunner, verbosity=2)
diff --git a/python/pyspark/pipelines/tests/test_cli.py b/python/pyspark/pipelines/tests/test_cli.py
index 92de53029f78b..8055723ddc5ae 100644
--- a/python/pyspark/pipelines/tests/test_cli.py
+++ b/python/pyspark/pipelines/tests/test_cli.py
@@ -36,6 +36,7 @@
unpack_pipeline_spec,
DefinitionsGlob,
PipelineSpec,
+ run,
)
from pyspark.pipelines.tests.local_graph_element_registry import LocalGraphElementRegistry
@@ -50,6 +51,7 @@ def test_load_pipeline_spec(self):
tmpfile.write(
"""
{
+ "name": "test_pipeline",
"catalog": "test_catalog",
"database": "test_database",
"configuration": {
@@ -64,17 +66,44 @@ def test_load_pipeline_spec(self):
)
tmpfile.flush()
spec = load_pipeline_spec(Path(tmpfile.name))
+ assert spec.name == "test_pipeline"
assert spec.catalog == "test_catalog"
assert spec.database == "test_database"
assert spec.configuration == {"key1": "value1", "key2": "value2"}
assert len(spec.definitions) == 1
assert spec.definitions[0].include == "test_include"
+ def test_load_pipeline_spec_name_is_required(self):
+ with tempfile.NamedTemporaryFile(mode="w") as tmpfile:
+ tmpfile.write(
+ """
+ {
+ "catalog": "test_catalog",
+ "database": "test_database",
+ "configuration": {
+ "key1": "value1",
+ "key2": "value2"
+ },
+ "definitions": [
+ {"glob": {"include": "test_include"}}
+ ]
+ }
+ """
+ )
+ tmpfile.flush()
+ with self.assertRaises(PySparkException) as context:
+ load_pipeline_spec(Path(tmpfile.name))
+ self.assertEqual(
+ context.exception.getCondition(), "PIPELINE_SPEC_MISSING_REQUIRED_FIELD"
+ )
+ self.assertEqual(context.exception.getMessageParameters(), {"field_name": "name"})
+
def test_load_pipeline_spec_schema_fallback(self):
with tempfile.NamedTemporaryFile(mode="w") as tmpfile:
tmpfile.write(
"""
{
+ "name": "test_pipeline",
"catalog": "test_catalog",
"schema": "test_database",
"configuration": {
@@ -120,20 +149,22 @@ def test_load_pipeline_spec_invalid(self):
)
def test_unpack_empty_pipeline_spec(self):
- empty_spec = PipelineSpec(catalog=None, database=None, configuration={}, definitions=[])
- self.assertEqual(unpack_pipeline_spec({}), empty_spec)
+ empty_spec = PipelineSpec(
+ name="test_pipeline", catalog=None, database=None, configuration={}, definitions=[]
+ )
+ self.assertEqual(unpack_pipeline_spec({"name": "test_pipeline"}), empty_spec)
def test_unpack_pipeline_spec_bad_configuration(self):
with self.assertRaises(TypeError) as context:
- unpack_pipeline_spec({"configuration": "not_a_dict"})
+ unpack_pipeline_spec({"name": "test_pipeline", "configuration": "not_a_dict"})
self.assertIn("should be a dict", str(context.exception))
with self.assertRaises(TypeError) as context:
- unpack_pipeline_spec({"configuration": {"key": {}}})
+ unpack_pipeline_spec({"name": "test_pipeline", "configuration": {"key": {}}})
self.assertIn("key", str(context.exception))
with self.assertRaises(TypeError) as context:
- unpack_pipeline_spec({"configuration": {1: "something"}})
+ unpack_pipeline_spec({"name": "test_pipeline", "configuration": {1: "something"}})
self.assertIn("int", str(context.exception))
def test_find_pipeline_spec_in_current_directory(self):
@@ -205,6 +236,7 @@ def test_find_pipeline_spec_in_parent_directory(self):
def test_register_definitions(self):
spec = PipelineSpec(
+ name="test_pipeline",
catalog=None,
database=None,
configuration={},
@@ -220,8 +252,8 @@ def test_register_definitions(self):
f.write(
textwrap.dedent(
"""
- from pyspark import pipelines as sdp
- @sdp.materialized_view
+ from pyspark import pipelines as dp
+ @dp.materialized_view
def mv1():
raise NotImplementedError()
"""
@@ -232,7 +264,7 @@ def mv1():
f.write(
textwrap.dedent(
"""
- from pyspark import pipelines as sdp
+ from pyspark import pipelines as dp
def mv2():
raise NotImplementedError()
"""
@@ -247,6 +279,7 @@ def mv2():
def test_register_definitions_file_raises_error(self):
"""Errors raised while executing definitions code should make it to the outer context."""
spec = PipelineSpec(
+ name="test_pipeline",
catalog=None,
database=None,
configuration={},
@@ -264,6 +297,7 @@ def test_register_definitions_file_raises_error(self):
def test_register_definitions_unsupported_file_extension_matches_glob(self):
spec = PipelineSpec(
+ name="test_pipeline",
catalog=None,
database=None,
configuration={},
@@ -317,6 +351,7 @@ def test_python_import_current_directory(self):
inner_dir1 / "pipeline.yaml",
registry,
PipelineSpec(
+ name="test_pipeline",
catalog=None,
database=None,
configuration={},
@@ -324,6 +359,98 @@ def test_python_import_current_directory(self):
),
)
+ def test_full_refresh_all_conflicts_with_full_refresh(self):
+ with tempfile.TemporaryDirectory() as temp_dir:
+ # Create a minimal pipeline spec
+ spec_path = Path(temp_dir) / "pipeline.yaml"
+ with spec_path.open("w") as f:
+ f.write('{"name": "test_pipeline"}')
+
+ # Test that providing both --full-refresh-all and --full-refresh raises an exception
+ with self.assertRaises(PySparkException) as context:
+ run(
+ spec_path=spec_path,
+ full_refresh=["table1", "table2"],
+ full_refresh_all=True,
+ refresh=[],
+ dry=False,
+ )
+
+ self.assertEqual(
+ context.exception.getCondition(), "CONFLICTING_PIPELINE_REFRESH_OPTIONS"
+ )
+ self.assertEqual(
+ context.exception.getMessageParameters(), {"conflicting_option": "--full_refresh"}
+ )
+
+ def test_full_refresh_all_conflicts_with_refresh(self):
+ with tempfile.TemporaryDirectory() as temp_dir:
+ # Create a minimal pipeline spec
+ spec_path = Path(temp_dir) / "pipeline.yaml"
+ with spec_path.open("w") as f:
+ f.write('{"name": "test_pipeline"}')
+
+ # Test that providing both --full-refresh-all and --refresh raises an exception
+ with self.assertRaises(PySparkException) as context:
+ run(
+ spec_path=spec_path,
+ full_refresh=[],
+ full_refresh_all=True,
+ refresh=["table1", "table2"],
+ dry=False,
+ )
+
+ self.assertEqual(
+ context.exception.getCondition(), "CONFLICTING_PIPELINE_REFRESH_OPTIONS"
+ )
+ self.assertEqual(
+ context.exception.getMessageParameters(),
+ {"conflicting_option": "--refresh"},
+ )
+
+ def test_full_refresh_all_conflicts_with_both(self):
+ with tempfile.TemporaryDirectory() as temp_dir:
+ # Create a minimal pipeline spec
+ spec_path = Path(temp_dir) / "pipeline.yaml"
+ with spec_path.open("w") as f:
+ f.write('{"name": "test_pipeline"}')
+
+ # Test that providing --full-refresh-all with both other options raises an exception
+ # (it should catch the first conflict - full_refresh)
+ with self.assertRaises(PySparkException) as context:
+ run(
+ spec_path=spec_path,
+ full_refresh=["table1"],
+ full_refresh_all=True,
+ refresh=["table2"],
+ dry=False,
+ )
+
+ self.assertEqual(
+ context.exception.getCondition(), "CONFLICTING_PIPELINE_REFRESH_OPTIONS"
+ )
+
+ def test_parse_table_list_single_table(self):
+ """Test parsing a single table name."""
+ from pyspark.pipelines.cli import parse_table_list
+
+ result = parse_table_list("table1")
+ self.assertEqual(result, ["table1"])
+
+ def test_parse_table_list_multiple_tables(self):
+ """Test parsing multiple table names."""
+ from pyspark.pipelines.cli import parse_table_list
+
+ result = parse_table_list("table1,table2,table3")
+ self.assertEqual(result, ["table1", "table2", "table3"])
+
+ def test_parse_table_list_with_spaces(self):
+ """Test parsing table names with spaces."""
+ from pyspark.pipelines.cli import parse_table_list
+
+ result = parse_table_list("table1, table2 , table3")
+ self.assertEqual(result, ["table1", "table2", "table3"])
+
if __name__ == "__main__":
try:
diff --git a/python/pyspark/pipelines/tests/test_decorators.py b/python/pyspark/pipelines/tests/test_decorators.py
index ac575b43d1090..31a966da164d1 100644
--- a/python/pyspark/pipelines/tests/test_decorators.py
+++ b/python/pyspark/pipelines/tests/test_decorators.py
@@ -18,12 +18,12 @@
import unittest
from pyspark.errors import PySparkTypeError
-from pyspark import pipelines as sdp
+from pyspark import pipelines as dp
class DecoratorsTest(unittest.TestCase):
def test_dataset_name_not_string(self):
- for decorator in [sdp.table, sdp.temporary_view, sdp.materialized_view]:
+ for decorator in [dp.table, dp.temporary_view, dp.materialized_view]:
with self.assertRaises(PySparkTypeError) as context:
@decorator(name=5)
@@ -37,7 +37,7 @@ def dataset_with_non_string_name():
}, context.exception.getMessageParameters()
def test_invalid_partition_cols(self):
- for decorator in [sdp.table, sdp.materialized_view]:
+ for decorator in [dp.table, dp.materialized_view]:
with self.assertRaises(PySparkTypeError) as context:
@decorator(partition_cols=["a", 1, 2]) # type: ignore
@@ -51,7 +51,7 @@ def dataset_with_invalid_partition_cols():
}, context.exception.getMessageParameters()
def test_decorator_with_positional_arg(self):
- for decorator in [sdp.table, sdp.temporary_view, sdp.materialized_view]:
+ for decorator in [dp.table, dp.temporary_view, dp.materialized_view]:
with self.assertRaises(PySparkTypeError) as context:
decorator("table1")
diff --git a/python/pyspark/pipelines/tests/test_graph_element_registry.py b/python/pyspark/pipelines/tests/test_graph_element_registry.py
index 9ebf29d1ed47b..2f9d2c69b7d5c 100644
--- a/python/pyspark/pipelines/tests/test_graph_element_registry.py
+++ b/python/pyspark/pipelines/tests/test_graph_element_registry.py
@@ -19,7 +19,7 @@
from pyspark.errors import PySparkException
from pyspark.pipelines.graph_element_registry import graph_element_registration_context
-from pyspark import pipelines as sdp
+from pyspark import pipelines as dp
from pyspark.pipelines.tests.local_graph_element_registry import LocalGraphElementRegistry
@@ -28,21 +28,21 @@ def test_graph_element_registry(self):
registry = LocalGraphElementRegistry()
with graph_element_registration_context(registry):
- @sdp.materialized_view
+ @dp.materialized_view
def mv():
raise NotImplementedError()
- @sdp.table
+ @dp.table
def st():
raise NotImplementedError()
- sdp.create_streaming_table("st2")
+ dp.create_streaming_table("st2")
- @sdp.append_flow(target="st2", once=True)
+ @dp.append_flow(target="st2")
def flow1():
raise NotImplementedError()
- @sdp.append_flow(target="st2", once=False)
+ @dp.append_flow(target="st2")
def flow2():
raise NotImplementedError()
@@ -74,17 +74,15 @@ def flow2():
st2_flow1_obj = registry.flows[2]
self.assertEqual(st2_flow1_obj.name, "flow1")
self.assertEqual(st2_flow1_obj.target, "st2")
- self.assertEqual(st2_flow1_obj.once, True)
assert mv_flow_obj.source_code_location.filename.endswith("test_graph_element_registry.py")
st2_flow1_obj = registry.flows[3]
self.assertEqual(st2_flow1_obj.name, "flow2")
self.assertEqual(st2_flow1_obj.target, "st2")
- self.assertEqual(st2_flow1_obj.once, False)
assert mv_flow_obj.source_code_location.filename.endswith("test_graph_element_registry.py")
def test_definition_without_graph_element_registry(self):
- for decorator in [sdp.table, sdp.temporary_view, sdp.materialized_view]:
+ for decorator in [dp.table, dp.temporary_view, dp.materialized_view]:
with self.assertRaises(PySparkException) as context:
@decorator
@@ -97,7 +95,7 @@ def a():
)
with self.assertRaises(PySparkException) as context:
- sdp.create_streaming_table("st")
+ dp.create_streaming_table("st")
self.assertEqual(
context.exception.getCondition(),
@@ -106,7 +104,7 @@ def a():
with self.assertRaises(PySparkException) as context:
- @sdp.append_flow(target="st")
+ @dp.append_flow(target="st")
def b():
raise NotImplementedError()
diff --git a/python/pyspark/pipelines/tests/test_init_cli.py b/python/pyspark/pipelines/tests/test_init_cli.py
index 656e4b609eaa2..f2390853d0f3e 100644
--- a/python/pyspark/pipelines/tests/test_init_cli.py
+++ b/python/pyspark/pipelines/tests/test_init_cli.py
@@ -50,6 +50,7 @@ def test_init(self):
with change_dir(Path(temp_dir) / project_name):
spec_path = find_pipeline_spec(Path.cwd())
spec = load_pipeline_spec(spec_path)
+ assert spec.name == project_name
registry = LocalGraphElementRegistry()
register_definitions(spec_path, registry, spec)
self.assertEqual(len(registry.datasets), 1)
diff --git a/python/pyspark/pipelines/tests/test_spark_connect.py b/python/pyspark/pipelines/tests/test_spark_connect.py
new file mode 100644
index 0000000000000..6d81a98c8c44d
--- /dev/null
+++ b/python/pyspark/pipelines/tests/test_spark_connect.py
@@ -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.
+#
+
+"""
+Tests that run Pipelines against a Spark Connect server.
+"""
+
+import unittest
+
+from pyspark.errors.exceptions.connect import AnalysisException
+from pyspark.pipelines.graph_element_registry import graph_element_registration_context
+from pyspark.pipelines.spark_connect_graph_element_registry import (
+ SparkConnectGraphElementRegistry,
+)
+from pyspark.pipelines.spark_connect_pipeline import (
+ create_dataflow_graph,
+ start_run,
+ handle_pipeline_events,
+)
+from pyspark import pipelines as dp
+from pyspark.testing.connectutils import (
+ ReusedConnectTestCase,
+ should_test_connect,
+ connect_requirement_message,
+)
+
+
+@unittest.skipIf(not should_test_connect, connect_requirement_message)
+class SparkConnectPipelinesTest(ReusedConnectTestCase):
+ def test_dry_run(self):
+ dataflow_graph_id = create_dataflow_graph(self.spark, None, None, None)
+ registry = SparkConnectGraphElementRegistry(self.spark, dataflow_graph_id)
+
+ with graph_element_registration_context(registry):
+
+ @dp.materialized_view
+ def mv():
+ return self.spark.range(1)
+
+ result_iter = start_run(
+ self.spark,
+ dataflow_graph_id,
+ full_refresh=None,
+ refresh=None,
+ full_refresh_all=False,
+ dry=True,
+ )
+ handle_pipeline_events(result_iter)
+
+ def test_dry_run_failure(self):
+ dataflow_graph_id = create_dataflow_graph(self.spark, None, None, None)
+ registry = SparkConnectGraphElementRegistry(self.spark, dataflow_graph_id)
+
+ with graph_element_registration_context(registry):
+
+ @dp.table
+ def st():
+ # Invalid because a streaming query is expected
+ return self.spark.range(1)
+
+ result_iter = start_run(
+ self.spark,
+ dataflow_graph_id,
+ full_refresh=None,
+ refresh=None,
+ full_refresh_all=False,
+ dry=True,
+ )
+ with self.assertRaises(AnalysisException) as context:
+ handle_pipeline_events(result_iter)
+ self.assertIn(
+ "INVALID_FLOW_QUERY_TYPE.BATCH_RELATION_FOR_STREAMING_TABLE", str(context.exception)
+ )
+
+
+if __name__ == "__main__":
+ try:
+ import xmlrunner # type: ignore
+
+ testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2)
+ except ImportError:
+ testRunner = None
+ unittest.main(testRunner=testRunner, verbosity=2)
diff --git a/python/pyspark/sql/_typing.pyi b/python/pyspark/sql/_typing.pyi
index 27fa0f2a90133..fafc9bdf15fc5 100644
--- a/python/pyspark/sql/_typing.pyi
+++ b/python/pyspark/sql/_typing.pyi
@@ -65,6 +65,7 @@ SQLBatchedUDFType = Literal[100]
SQLArrowBatchedUDFType = Literal[101]
SQLTableUDFType = Literal[300]
SQLArrowTableUDFType = Literal[301]
+SQLArrowUDTFType = Literal[302]
class SupportsOpen(Protocol):
def open(self, partition_id: int, epoch_id: int) -> bool: ...
diff --git a/python/pyspark/sql/classic/dataframe.py b/python/pyspark/sql/classic/dataframe.py
index f72c066a65723..722f0615b370a 100644
--- a/python/pyspark/sql/classic/dataframe.py
+++ b/python/pyspark/sql/classic/dataframe.py
@@ -207,7 +207,7 @@ def schema(self) -> StructType:
except Exception as e:
raise PySparkValueError(
errorClass="CANNOT_PARSE_DATATYPE",
- messageParameters={"error": str(e)},
+ messageParameters={"msg": str(e)},
)
def printSchema(self, level: Optional[int] = None) -> None:
diff --git a/python/pyspark/sql/connect/client/core.py b/python/pyspark/sql/connect/client/core.py
index 34719f2b0ba6e..9d2e18ebb7600 100644
--- a/python/pyspark/sql/connect/client/core.py
+++ b/python/pyspark/sql/connect/client/core.py
@@ -63,7 +63,6 @@
from pyspark.util import is_remote_only
from pyspark.accumulators import SpecialAccumulatorIds
-from pyspark.loose_version import LooseVersion
from pyspark.version import __version__
from pyspark.resource.information import ResourceInformation
from pyspark.sql.metrics import MetricValue, PlanMetrics, ExecutionInfo, ObservedMetrics
@@ -109,6 +108,7 @@
if TYPE_CHECKING:
from google.rpc.error_details_pb2 import ErrorInfo
from pyspark.sql.connect._typing import DataTypeOrString
+ from pyspark.sql.connect.session import SparkSession
from pyspark.sql.datasource import DataSource
@@ -606,6 +606,7 @@ def __init__(
channel_options: Optional[List[Tuple[str, Any]]] = None,
retry_policy: Optional[Dict[str, Any]] = None,
use_reattachable_execute: bool = True,
+ session_hooks: Optional[list["SparkSession.Hook"]] = None,
):
"""
Creates a new SparkSession for the Spark Connect interface.
@@ -636,6 +637,8 @@ def __init__(
a failed request. Default: 60000(ms).
use_reattachable_execute: bool
Enable reattachable execution.
+ session_hooks: list[SparkSession.Hook], optional
+ List of session hooks to call.
"""
self.thread_local = threading.local()
@@ -675,6 +678,7 @@ def __init__(
self._user_id, self._session_id, self._channel, self._builder.metadata()
)
self._use_reattachable_execute = use_reattachable_execute
+ self._session_hooks = session_hooks or []
# Configure logging for the SparkConnect client.
# Capture the server-side session ID and set it to None initially. It will
@@ -963,7 +967,7 @@ def to_pandas(
# Rename columns to avoid duplicated column names.
renamed_table = table.rename_columns([f"col_{i}" for i in range(table.num_columns)])
- pandas_options = {}
+ pandas_options = {"coerce_temporal_nanoseconds": True}
if self_destruct:
# Configure PyArrow to use as little memory as possible:
# self_destruct - free columns as they are converted
@@ -976,15 +980,6 @@ def to_pandas(
"use_threads": False,
}
)
- if LooseVersion(pa.__version__) >= LooseVersion("13.0.0"):
- # A legacy option to coerce date32, date64, duration, and timestamp
- # time units to nanoseconds when converting to pandas.
- # This option can only be added since 13.0.0.
- pandas_options.update(
- {
- "coerce_temporal_nanoseconds": True,
- }
- )
pdf = renamed_table.to_pandas(**pandas_options)
pdf.columns = schema.names
@@ -1365,6 +1360,9 @@ def _execute(self, req: pb2.ExecutePlanRequest) -> None:
"""
logger.debug("Execute")
+ for hook in self._session_hooks:
+ req = hook.on_execute_plan(req)
+
def handle_response(b: pb2.ExecutePlanResponse) -> None:
self._verify_response_integrity(b)
@@ -1406,6 +1404,9 @@ def _execute_and_fetch_as_iterator(
# when not at debug log level.
logger.debug(f"ExecuteAndFetchAsIterator. Request: {self._proto_to_string(req)}")
+ for hook in self._session_hooks:
+ req = hook.on_execute_plan(req)
+
num_records = 0
def handle_response(
@@ -1985,7 +1986,7 @@ def _create_profile(self, profile: pb2.ResourceProfile) -> int:
profile_id = properties["create_resource_profile_command_result"]
return profile_id
- def _delete_ml_cache(self, cache_ids: List[str]) -> List[str]:
+ def _delete_ml_cache(self, cache_ids: List[str], evict_only: bool = False) -> List[str]:
# try best to delete the cache
try:
if len(cache_ids) > 0:
@@ -1993,6 +1994,7 @@ def _delete_ml_cache(self, cache_ids: List[str]) -> List[str]:
command.ml_command.delete.obj_refs.extend(
[pb2.ObjectRef(id=cache_id) for cache_id in cache_ids]
)
+ command.ml_command.delete.evict_only = evict_only
(_, properties, _) = self.execute_command(command)
assert properties is not None
@@ -2025,3 +2027,15 @@ def _get_ml_cache_info(self) -> List[str]:
return [item.string for item in ml_command_result.param.array.elements]
return []
+
+ def _query_model_size(self, model_ref_id: str) -> int:
+ command = pb2.Command()
+ command.ml_command.get_model_size.CopyFrom(
+ pb2.MlCommand.GetModelSize(model_ref=pb2.ObjectRef(id=model_ref_id))
+ )
+ (_, properties, _) = self.execute_command(command)
+
+ assert properties is not None
+
+ ml_command_result = properties["ml_command_result"]
+ return ml_command_result.param.long
diff --git a/python/pyspark/sql/connect/client/reattach.py b/python/pyspark/sql/connect/client/reattach.py
index 78b783371ab56..06a4fe17c29f9 100644
--- a/python/pyspark/sql/connect/client/reattach.py
+++ b/python/pyspark/sql/connect/client/reattach.py
@@ -79,8 +79,9 @@ def shutdown(cls: Type["ExecutePlanResponseReattachableIterator"]) -> None:
"""
with cls._lock:
if cls._release_thread_pool_instance is not None:
- cls._get_or_create_release_thread_pool().shutdown()
+ thread_pool = cls._release_thread_pool_instance
cls._release_thread_pool_instance = None
+ thread_pool.shutdown()
def __init__(
self,
diff --git a/python/pyspark/sql/connect/client/retries.py b/python/pyspark/sql/connect/client/retries.py
index e27100133b5ae..898d976f2628e 100644
--- a/python/pyspark/sql/connect/client/retries.py
+++ b/python/pyspark/sql/connect/client/retries.py
@@ -19,10 +19,13 @@
import random
import time
import typing
-from typing import Optional, Callable, Generator, List, Type
+import warnings
+from google.rpc import error_details_pb2
+from grpc_status import rpc_status
+from typing import Optional, Callable, Generator, List, Type, cast
from types import TracebackType
from pyspark.sql.connect.logging import logger
-from pyspark.errors import PySparkRuntimeError, RetriesExceeded
+from pyspark.errors import PySparkRuntimeError
"""
This module contains retry system. The system is designed to be
@@ -45,6 +48,34 @@ class RetryPolicy:
Describes key aspects of RetryPolicy.
It's advised that different policies are implemented as different subclasses.
+
+ Parameters
+ ----------
+ max_retries: int, optional
+ Maximum number of retries.
+ initial_backoff: int
+ Start value of the exponential backoff.
+ max_backoff: int, optional
+ Maximal value of the exponential backoff.
+ backoff_multiplier: float
+ Multiplicative base of the exponential backoff.
+ jitter: int
+ Sample a random value uniformly from the range [0, jitter] and add it to the backoff.
+ min_jitter_threshold: int
+ Minimal value of the backoff to add random jitter.
+ recognize_server_retry_delay: bool
+ Per gRPC standard, the server can send error messages that contain `RetryInfo` message
+ with `retry_delay` field indicating that the client should wait for at least `retry_delay`
+ amount of time before retrying again, see:
+ https://github.com/googleapis/googleapis/blob/master/google/rpc/error_details.proto#L91
+
+ If this flag is set to true, RetryPolicy will use `RetryInfo.retry_delay` field
+ in the backoff computation. Server's `retry_delay` can override client's `max_backoff`.
+
+ This flag does not change which errors are retried, only how the backoff is computed.
+ `DefaultPolicy` additionally has a rule for retrying any error that contains `RetryInfo`.
+ max_server_retry_delay: int, optional
+ Limit for the server-provided `retry_delay`.
"""
def __init__(
@@ -55,6 +86,8 @@ def __init__(
backoff_multiplier: float = 1.0,
jitter: int = 0,
min_jitter_threshold: int = 0,
+ recognize_server_retry_delay: bool = False,
+ max_server_retry_delay: Optional[int] = None,
):
self.max_retries = max_retries
self.initial_backoff = initial_backoff
@@ -62,6 +95,8 @@ def __init__(
self.backoff_multiplier = backoff_multiplier
self.jitter = jitter
self.min_jitter_threshold = min_jitter_threshold
+ self.recognize_server_retry_delay = recognize_server_retry_delay
+ self.max_server_retry_delay = max_server_retry_delay
self._name = self.__class__.__name__
@property
@@ -98,7 +133,7 @@ def name(self) -> str:
def can_retry(self, exception: BaseException) -> bool:
return self.policy.can_retry(exception)
- def next_attempt(self) -> Optional[int]:
+ def next_attempt(self, exception: Optional[BaseException] = None) -> Optional[int]:
"""
Returns
-------
@@ -119,6 +154,14 @@ def next_attempt(self) -> Optional[int]:
float(self.policy.max_backoff), wait_time * self.policy.backoff_multiplier
)
+ if exception is not None and self.policy.recognize_server_retry_delay:
+ retry_delay = extract_retry_delay(exception)
+ if retry_delay is not None:
+ logger.debug(f"The server has sent a retry delay of {retry_delay} ms.")
+ if self.policy.max_server_retry_delay is not None:
+ retry_delay = min(retry_delay, self.policy.max_server_retry_delay)
+ wait_time = max(wait_time, retry_delay)
+
# Jitter current backoff, after the future backoff was computed
if wait_time >= self.policy.min_jitter_threshold:
wait_time += random.uniform(0, self.policy.jitter)
@@ -160,6 +203,7 @@ class Retrying:
This class is a point of entry into the retry logic.
The class accepts a list of retry policies and applies them in given order.
The first policy accepting an exception will be used.
+ If the error was matched by one policy, the other policies will be skipped.
The usage of the class should be as follows:
for attempt in Retrying(...):
@@ -167,7 +211,7 @@ class Retrying:
Do something that can throw exception
In case error is considered retriable, it would be retried based on policies, and
- RetriesExceeded will be raised if the retries limit would exceed.
+ it will be raised if the retries limit would exceed.
Exceptions not considered retriable will be passed through transparently.
"""
@@ -217,23 +261,25 @@ def _wait(self) -> None:
return
# Attempt to find a policy to wait with
+ matched_policy = None
for policy in self._policies:
- if not policy.can_retry(exception):
- continue
-
- wait_time = policy.next_attempt()
+ if policy.can_retry(exception):
+ matched_policy = policy
+ break
+ if matched_policy is not None:
+ wait_time = matched_policy.next_attempt(exception)
if wait_time is not None:
logger.debug(
f"Got error: {repr(exception)}. "
- + f"Will retry after {wait_time} ms (policy: {policy.name})"
+ + f"Will retry after {wait_time} ms (policy: {matched_policy.name})"
)
-
self._sleep(wait_time / 1000)
return
# Exceeded retries
logger.debug(f"Given up on retrying. error: {repr(exception)}")
- raise RetriesExceeded(errorClass="RETRIES_EXCEEDED", messageParameters={}) from exception
+ warnings.warn("[RETRIES_EXCEEDED] The maximum number of retries has been exceeded.")
+ raise exception
def __iter__(self) -> Generator[AttemptManager, None, None]:
"""
@@ -274,6 +320,8 @@ def __init__(
max_backoff: Optional[int] = 60000,
jitter: int = 500,
min_jitter_threshold: int = 2000,
+ recognize_server_retry_delay: bool = True,
+ max_server_retry_delay: Optional[int] = 10 * 60 * 1000, # 10 minutes
):
super().__init__(
max_retries=max_retries,
@@ -282,6 +330,8 @@ def __init__(
max_backoff=max_backoff,
jitter=jitter,
min_jitter_threshold=min_jitter_threshold,
+ recognize_server_retry_delay=recognize_server_retry_delay,
+ max_server_retry_delay=max_server_retry_delay,
)
def can_retry(self, e: BaseException) -> bool:
@@ -314,4 +364,29 @@ def can_retry(self, e: BaseException) -> bool:
if e.code() == grpc.StatusCode.UNAVAILABLE:
return True
+ if extract_retry_info(e) is not None:
+ # All errors messages containing `RetryInfo` should be retried.
+ return True
+
return False
+
+
+def extract_retry_info(exception: BaseException) -> Optional[error_details_pb2.RetryInfo]:
+ """Extract and return RetryInfo from the grpc.RpcError"""
+ if isinstance(exception, grpc.RpcError):
+ status = rpc_status.from_call(cast(grpc.Call, exception))
+ if status:
+ for d in status.details:
+ if d.Is(error_details_pb2.RetryInfo.DESCRIPTOR):
+ info = error_details_pb2.RetryInfo()
+ d.Unpack(info)
+ return info
+ return None
+
+
+def extract_retry_delay(exception: BaseException) -> Optional[int]:
+ """Extract and return RetryInfo.retry_delay in milliseconds from grpc.RpcError if present."""
+ retry_info = extract_retry_info(exception)
+ if retry_info is not None:
+ return retry_info.retry_delay.ToMilliseconds()
+ return None
diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py
index 101ae06f10edc..7998167976026 100644
--- a/python/pyspark/sql/connect/dataframe.py
+++ b/python/pyspark/sql/connect/dataframe.py
@@ -22,6 +22,7 @@
PySparkAttributeError,
)
from pyspark.resource import ResourceProfile
+from pyspark.sql.connect.logging import logger
from pyspark.sql.connect.utils import check_dependencies
check_dependencies(__name__)
@@ -43,6 +44,7 @@
)
import copy
+import os
import sys
import random
import pyarrow as pa
@@ -69,6 +71,7 @@
PySparkRuntimeError,
)
from pyspark.util import PythonEvalType
+from pyspark.serializers import CPickleSerializer
from pyspark.storagelevel import StorageLevel
import pyspark.sql.connect.plan as plan
from pyspark.sql.conversion import ArrowTableToRowsConversion
@@ -141,6 +144,7 @@ def __init__(
# by __repr__ and _repr_html_ while eager evaluation opens.
self._support_repr_html = False
self._cached_schema: Optional[StructType] = None
+ self._cached_schema_serialized: Optional[bytes] = None
self._execution_info: Optional["ExecutionInfo"] = None
def __reduce__(self) -> Tuple:
@@ -1737,7 +1741,9 @@ def __getitem__(
# }
# validate the column name
- if not hasattr(self._session, "is_mock_session"):
+ if os.environ.get("PYSPARK_VALIDATE_COLUMN_NAME_LEGACY") == "1" and not hasattr(
+ self._session, "is_mock_session"
+ ):
from pyspark.sql.connect.types import verify_col_name
# Try best to verify the column name with cached schema
@@ -1836,11 +1842,24 @@ def _schema(self) -> StructType:
if self._cached_schema is None:
query = self._plan.to_proto(self._session.client)
self._cached_schema = self._session.client.schema(query)
+ try:
+ self._cached_schema_serialized = CPickleSerializer().dumps(self._schema)
+ except Exception as e:
+ logger.warn(f"DataFrame schema pickle dumps failed with exception: {e}.")
+ self._cached_schema_serialized = None
return self._cached_schema
@property
def schema(self) -> StructType:
- return copy.deepcopy(self._schema)
+ # self._schema call will cache the schema and serialize it if it is not cached yet.
+ _schema = self._schema
+ if self._cached_schema_serialized is not None:
+ try:
+ return CPickleSerializer().loads(self._cached_schema_serialized)
+ except Exception as e:
+ logger.warn(f"DataFrame schema pickle loads failed with exception: {e}.")
+ # In case of pickle ser/de failure, fallback to deepcopy approach.
+ return copy.deepcopy(_schema)
@functools.cache
def isLocal(self) -> bool:
diff --git a/python/pyspark/sql/connect/expressions.py b/python/pyspark/sql/connect/expressions.py
index 872770ee22911..4ddf13757db41 100644
--- a/python/pyspark/sql/connect/expressions.py
+++ b/python/pyspark/sql/connect/expressions.py
@@ -54,6 +54,7 @@
DecimalType,
StringType,
DataType,
+ TimeType,
TimestampType,
TimestampNTZType,
DayTimeIntervalType,
@@ -248,6 +249,7 @@ def __init__(self, value: Any, dataType: DataType) -> None:
DecimalType,
StringType,
DateType,
+ TimeType,
TimestampType,
TimestampNTZType,
DayTimeIntervalType,
@@ -298,6 +300,9 @@ def __init__(self, value: Any, dataType: DataType) -> None:
value = DateType().toInternal(value)
else:
value = DateType().toInternal(value.date())
+ elif isinstance(dataType, TimeType):
+ assert isinstance(value, datetime.time)
+ value = TimeType().toInternal(value)
elif isinstance(dataType, TimestampType):
assert isinstance(value, datetime.datetime)
value = TimestampType().toInternal(value)
@@ -352,6 +357,8 @@ def _infer_type(cls, value: Any) -> DataType:
return TimestampNTZType() if is_timestamp_ntz_preferred() else TimestampType()
elif isinstance(value, datetime.date):
return DateType()
+ elif isinstance(value, datetime.time):
+ return TimeType()
elif isinstance(value, datetime.timedelta):
return DayTimeIntervalType()
elif isinstance(value, np.generic):
@@ -416,6 +423,9 @@ def _to_value(
elif literal.HasField("date"):
assert dataType is None or isinstance(dataType, DataType)
return DateType().fromInternal(literal.date)
+ elif literal.HasField("time"):
+ assert dataType is None or isinstance(dataType, TimeType)
+ return TimeType().fromInternal(literal.time.nano)
elif literal.HasField("timestamp"):
assert dataType is None or isinstance(dataType, TimestampType)
return TimestampType().fromInternal(literal.timestamp)
@@ -468,6 +478,9 @@ def to_plan(self, session: "SparkConnectClient") -> "proto.Expression":
expr.literal.string = str(self._value)
elif isinstance(self._dataType, DateType):
expr.literal.date = int(self._value)
+ elif isinstance(self._dataType, TimeType):
+ expr.literal.time.precision = self._dataType.precision
+ expr.literal.time.nano = int(self._value)
elif isinstance(self._dataType, TimestampType):
expr.literal.timestamp = int(self._value)
elif isinstance(self._dataType, TimestampNTZType):
@@ -496,6 +509,10 @@ def __repr__(self) -> str:
dt = DateType().fromInternal(self._value)
if dt is not None and isinstance(dt, datetime.date):
return dt.strftime("%Y-%m-%d")
+ elif isinstance(self._dataType, TimeType):
+ t = TimeType().fromInternal(self._value)
+ if t is not None and isinstance(t, datetime.time):
+ return t.strftime("%H:%M:%S.%f")
elif isinstance(self._dataType, TimestampType):
ts = TimestampType().fromInternal(self._value)
if ts is not None and isinstance(ts, datetime.datetime):
diff --git a/python/pyspark/sql/connect/functions/__init__.py b/python/pyspark/sql/connect/functions/__init__.py
index 087a51e8616b9..4f3657b643a87 100644
--- a/python/pyspark/sql/connect/functions/__init__.py
+++ b/python/pyspark/sql/connect/functions/__init__.py
@@ -16,7 +16,7 @@
#
"""PySpark Functions with Spark Connect"""
-from pyspark.testing import should_test_connect
+from pyspark.testing.utils import should_test_connect
if should_test_connect:
from pyspark.sql.connect.functions.builtin import * # noqa: F401,F403
diff --git a/python/pyspark/sql/connect/functions/builtin.py b/python/pyspark/sql/connect/functions/builtin.py
index 85d78ccac3015..0380b517e6e5e 100644
--- a/python/pyspark/sql/connect/functions/builtin.py
+++ b/python/pyspark/sql/connect/functions/builtin.py
@@ -60,7 +60,7 @@
)
from pyspark.sql.connect.udf import _create_py_udf
from pyspark.sql.connect.udtf import AnalyzeArgument, AnalyzeResult # noqa: F401
-from pyspark.sql.connect.udtf import _create_py_udtf
+from pyspark.sql.connect.udtf import _create_py_udtf, _create_pyarrow_udtf
from pyspark.sql import functions as pysparkfuncs
from pyspark.sql.types import (
_from_numpy_type,
@@ -73,7 +73,7 @@
# The implementation of pandas_udf is embedded in pyspark.sql.function.pandas_udf
# for code reuse.
-from pyspark.sql.functions import pandas_udf # noqa: F401
+from pyspark.sql.functions import arrow_udf, pandas_udf # noqa: F401
if TYPE_CHECKING:
@@ -3140,6 +3140,26 @@ def current_date() -> Column:
current_date.__doc__ = pysparkfuncs.current_date.__doc__
+@overload
+def current_time() -> Column:
+ ...
+
+
+@overload
+def current_time(precision: int) -> Column:
+ ...
+
+
+def current_time(precision: Optional[int] = None) -> Column:
+ if precision is None:
+ return _invoke_function("current_time")
+ else:
+ return _invoke_function("current_time", lit(precision))
+
+
+current_time.__doc__ = pysparkfuncs.current_time.__doc__
+
+
def current_timestamp() -> Column:
return _invoke_function("current_timestamp")
@@ -3375,6 +3395,16 @@ def to_date(col: "ColumnOrName", format: Optional[str] = None) -> Column:
to_date.__doc__ = pysparkfuncs.to_date.__doc__
+def try_to_date(col: "ColumnOrName", format: Optional[str] = None) -> Column:
+ if format is None:
+ return _invoke_function_over_columns("try_to_date", col)
+ else:
+ return _invoke_function("try_to_date", _to_col(col), lit(format))
+
+
+try_to_date.__doc__ = pysparkfuncs.try_to_date.__doc__
+
+
def unix_date(col: "ColumnOrName") -> Column:
return _invoke_function_over_columns("unix_date", col)
@@ -3403,6 +3433,26 @@ def unix_seconds(col: "ColumnOrName") -> Column:
unix_seconds.__doc__ = pysparkfuncs.unix_seconds.__doc__
+@overload
+def to_time(str: "ColumnOrName") -> Column:
+ ...
+
+
+@overload
+def to_time(str: "ColumnOrName", format: "ColumnOrName") -> Column:
+ ...
+
+
+def to_time(str: "ColumnOrName", format: Optional["ColumnOrName"] = None) -> Column:
+ if format is None:
+ return _invoke_function_over_columns("to_time", str)
+ else:
+ return _invoke_function_over_columns("to_time", str, format)
+
+
+to_time.__doc__ = pysparkfuncs.to_time.__doc__
+
+
@overload
def to_timestamp(col: "ColumnOrName") -> Column:
...
@@ -3423,6 +3473,26 @@ def to_timestamp(col: "ColumnOrName", format: Optional[str] = None) -> Column:
to_timestamp.__doc__ = pysparkfuncs.to_timestamp.__doc__
+@overload
+def try_to_time(str: "ColumnOrName") -> Column:
+ ...
+
+
+@overload
+def try_to_time(str: "ColumnOrName", format: "ColumnOrName") -> Column:
+ ...
+
+
+def try_to_time(str: "ColumnOrName", format: Optional["ColumnOrName"] = None) -> Column:
+ if format is None:
+ return _invoke_function_over_columns("try_to_time", str)
+ else:
+ return _invoke_function_over_columns("try_to_time", str, format)
+
+
+try_to_time.__doc__ = pysparkfuncs.try_to_time.__doc__
+
+
def try_to_timestamp(col: "ColumnOrName", format: Optional["ColumnOrName"] = None) -> Column:
if format is not None:
return _invoke_function_over_columns("try_to_timestamp", col, format)
@@ -3573,6 +3643,13 @@ def timestamp_seconds(col: "ColumnOrName") -> Column:
timestamp_seconds.__doc__ = pysparkfuncs.timestamp_seconds.__doc__
+def time_trunc(unit: "ColumnOrName", time: "ColumnOrName") -> Column:
+ return _invoke_function_over_columns("time_trunc", unit, time)
+
+
+time_trunc.__doc__ = pysparkfuncs.time_trunc.__doc__
+
+
def timestamp_millis(col: "ColumnOrName") -> Column:
return _invoke_function_over_columns("timestamp_millis", col)
@@ -3847,6 +3924,13 @@ def make_interval(
make_interval.__doc__ = pysparkfuncs.make_interval.__doc__
+def make_time(hour: "ColumnOrName", minute: "ColumnOrName", second: "ColumnOrName") -> Column:
+ return _invoke_function_over_columns("make_time", hour, minute, second)
+
+
+make_time.__doc__ = pysparkfuncs.make_time.__doc__
+
+
def make_timestamp(
years: "ColumnOrName",
months: "ColumnOrName",
@@ -4416,6 +4500,20 @@ def udtf(
udtf.__doc__ = pysparkfuncs.udtf.__doc__
+def arrow_udtf(
+ cls: Optional[Type] = None,
+ *,
+ returnType: Optional[Union[StructType, str]] = None,
+) -> Union["UserDefinedTableFunction", Callable[[Type], "UserDefinedTableFunction"]]:
+ if cls is None:
+ return functools.partial(_create_pyarrow_udtf, returnType=returnType)
+ else:
+ return _create_pyarrow_udtf(cls=cls, returnType=returnType)
+
+
+arrow_udtf.__doc__ = pysparkfuncs.arrow_udtf.__doc__
+
+
def call_function(funcName: str, *cols: "ColumnOrName") -> Column:
from pyspark.sql.connect.column import Column as ConnectColumn
diff --git a/python/pyspark/sql/connect/proto/expressions_pb2.py b/python/pyspark/sql/connect/proto/expressions_pb2.py
index 0cec23f4857df..3ce27567f1e04 100644
--- a/python/pyspark/sql/connect/proto/expressions_pb2.py
+++ b/python/pyspark/sql/connect/proto/expressions_pb2.py
@@ -40,7 +40,7 @@
DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(
- b'\n\x1fspark/connect/expressions.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x19spark/connect/types.proto\x1a\x1aspark/connect/common.proto"\xf3\x34\n\nExpression\x12\x37\n\x06\x63ommon\x18\x12 \x01(\x0b\x32\x1f.spark.connect.ExpressionCommonR\x06\x63ommon\x12=\n\x07literal\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x07literal\x12\x62\n\x14unresolved_attribute\x18\x02 \x01(\x0b\x32-.spark.connect.Expression.UnresolvedAttributeH\x00R\x13unresolvedAttribute\x12_\n\x13unresolved_function\x18\x03 \x01(\x0b\x32,.spark.connect.Expression.UnresolvedFunctionH\x00R\x12unresolvedFunction\x12Y\n\x11\x65xpression_string\x18\x04 \x01(\x0b\x32*.spark.connect.Expression.ExpressionStringH\x00R\x10\x65xpressionString\x12S\n\x0funresolved_star\x18\x05 \x01(\x0b\x32(.spark.connect.Expression.UnresolvedStarH\x00R\x0eunresolvedStar\x12\x37\n\x05\x61lias\x18\x06 \x01(\x0b\x32\x1f.spark.connect.Expression.AliasH\x00R\x05\x61lias\x12\x34\n\x04\x63\x61st\x18\x07 \x01(\x0b\x32\x1e.spark.connect.Expression.CastH\x00R\x04\x63\x61st\x12V\n\x10unresolved_regex\x18\x08 \x01(\x0b\x32).spark.connect.Expression.UnresolvedRegexH\x00R\x0funresolvedRegex\x12\x44\n\nsort_order\x18\t \x01(\x0b\x32#.spark.connect.Expression.SortOrderH\x00R\tsortOrder\x12S\n\x0flambda_function\x18\n \x01(\x0b\x32(.spark.connect.Expression.LambdaFunctionH\x00R\x0elambdaFunction\x12:\n\x06window\x18\x0b \x01(\x0b\x32 .spark.connect.Expression.WindowH\x00R\x06window\x12l\n\x18unresolved_extract_value\x18\x0c \x01(\x0b\x32\x30.spark.connect.Expression.UnresolvedExtractValueH\x00R\x16unresolvedExtractValue\x12M\n\rupdate_fields\x18\r \x01(\x0b\x32&.spark.connect.Expression.UpdateFieldsH\x00R\x0cupdateFields\x12\x82\x01\n unresolved_named_lambda_variable\x18\x0e \x01(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableH\x00R\x1dunresolvedNamedLambdaVariable\x12~\n#common_inline_user_defined_function\x18\x0f \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionH\x00R\x1f\x63ommonInlineUserDefinedFunction\x12\x42\n\rcall_function\x18\x10 \x01(\x0b\x32\x1b.spark.connect.CallFunctionH\x00R\x0c\x63\x61llFunction\x12\x64\n\x19named_argument_expression\x18\x11 \x01(\x0b\x32&.spark.connect.NamedArgumentExpressionH\x00R\x17namedArgumentExpression\x12?\n\x0cmerge_action\x18\x13 \x01(\x0b\x32\x1a.spark.connect.MergeActionH\x00R\x0bmergeAction\x12g\n\x1atyped_aggregate_expression\x18\x14 \x01(\x0b\x32\'.spark.connect.TypedAggregateExpressionH\x00R\x18typedAggregateExpression\x12T\n\x13subquery_expression\x18\x15 \x01(\x0b\x32!.spark.connect.SubqueryExpressionH\x00R\x12subqueryExpression\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a\x8f\x06\n\x06Window\x12\x42\n\x0fwindow_function\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0ewindowFunction\x12@\n\x0epartition_spec\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\rpartitionSpec\x12\x42\n\norder_spec\x18\x03 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\torderSpec\x12K\n\nframe_spec\x18\x04 \x01(\x0b\x32,.spark.connect.Expression.Window.WindowFrameR\tframeSpec\x1a\xed\x03\n\x0bWindowFrame\x12U\n\nframe_type\x18\x01 \x01(\x0e\x32\x36.spark.connect.Expression.Window.WindowFrame.FrameTypeR\tframeType\x12P\n\x05lower\x18\x02 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05lower\x12P\n\x05upper\x18\x03 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05upper\x1a\x91\x01\n\rFrameBoundary\x12!\n\x0b\x63urrent_row\x18\x01 \x01(\x08H\x00R\ncurrentRow\x12\x1e\n\tunbounded\x18\x02 \x01(\x08H\x00R\tunbounded\x12\x31\n\x05value\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionH\x00R\x05valueB\n\n\x08\x62oundary"O\n\tFrameType\x12\x18\n\x14\x46RAME_TYPE_UNDEFINED\x10\x00\x12\x12\n\x0e\x46RAME_TYPE_ROW\x10\x01\x12\x14\n\x10\x46RAME_TYPE_RANGE\x10\x02\x1a\xa9\x03\n\tSortOrder\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12O\n\tdirection\x18\x02 \x01(\x0e\x32\x31.spark.connect.Expression.SortOrder.SortDirectionR\tdirection\x12U\n\rnull_ordering\x18\x03 \x01(\x0e\x32\x30.spark.connect.Expression.SortOrder.NullOrderingR\x0cnullOrdering"l\n\rSortDirection\x12\x1e\n\x1aSORT_DIRECTION_UNSPECIFIED\x10\x00\x12\x1c\n\x18SORT_DIRECTION_ASCENDING\x10\x01\x12\x1d\n\x19SORT_DIRECTION_DESCENDING\x10\x02"U\n\x0cNullOrdering\x12\x1a\n\x16SORT_NULLS_UNSPECIFIED\x10\x00\x12\x14\n\x10SORT_NULLS_FIRST\x10\x01\x12\x13\n\x0fSORT_NULLS_LAST\x10\x02\x1a\xbb\x02\n\x04\x43\x61st\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12-\n\x04type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x04type\x12\x1b\n\x08type_str\x18\x03 \x01(\tH\x00R\x07typeStr\x12\x44\n\teval_mode\x18\x04 \x01(\x0e\x32\'.spark.connect.Expression.Cast.EvalModeR\x08\x65valMode"b\n\x08\x45valMode\x12\x19\n\x15\x45VAL_MODE_UNSPECIFIED\x10\x00\x12\x14\n\x10\x45VAL_MODE_LEGACY\x10\x01\x12\x12\n\x0e\x45VAL_MODE_ANSI\x10\x02\x12\x11\n\rEVAL_MODE_TRY\x10\x03\x42\x0e\n\x0c\x63\x61st_to_type\x1a\xc1\x0f\n\x07Literal\x12-\n\x04null\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x04null\x12\x18\n\x06\x62inary\x18\x02 \x01(\x0cH\x00R\x06\x62inary\x12\x1a\n\x07\x62oolean\x18\x03 \x01(\x08H\x00R\x07\x62oolean\x12\x14\n\x04\x62yte\x18\x04 \x01(\x05H\x00R\x04\x62yte\x12\x16\n\x05short\x18\x05 \x01(\x05H\x00R\x05short\x12\x1a\n\x07integer\x18\x06 \x01(\x05H\x00R\x07integer\x12\x14\n\x04long\x18\x07 \x01(\x03H\x00R\x04long\x12\x16\n\x05\x66loat\x18\n \x01(\x02H\x00R\x05\x66loat\x12\x18\n\x06\x64ouble\x18\x0b \x01(\x01H\x00R\x06\x64ouble\x12\x45\n\x07\x64\x65\x63imal\x18\x0c \x01(\x0b\x32).spark.connect.Expression.Literal.DecimalH\x00R\x07\x64\x65\x63imal\x12\x18\n\x06string\x18\r \x01(\tH\x00R\x06string\x12\x14\n\x04\x64\x61te\x18\x10 \x01(\x05H\x00R\x04\x64\x61te\x12\x1e\n\ttimestamp\x18\x11 \x01(\x03H\x00R\ttimestamp\x12%\n\rtimestamp_ntz\x18\x12 \x01(\x03H\x00R\x0ctimestampNtz\x12\x61\n\x11\x63\x61lendar_interval\x18\x13 \x01(\x0b\x32\x32.spark.connect.Expression.Literal.CalendarIntervalH\x00R\x10\x63\x61lendarInterval\x12\x30\n\x13year_month_interval\x18\x14 \x01(\x05H\x00R\x11yearMonthInterval\x12,\n\x11\x64\x61y_time_interval\x18\x15 \x01(\x03H\x00R\x0f\x64\x61yTimeInterval\x12?\n\x05\x61rray\x18\x16 \x01(\x0b\x32\'.spark.connect.Expression.Literal.ArrayH\x00R\x05\x61rray\x12\x39\n\x03map\x18\x17 \x01(\x0b\x32%.spark.connect.Expression.Literal.MapH\x00R\x03map\x12\x42\n\x06struct\x18\x18 \x01(\x0b\x32(.spark.connect.Expression.Literal.StructH\x00R\x06struct\x12\x61\n\x11specialized_array\x18\x19 \x01(\x0b\x32\x32.spark.connect.Expression.Literal.SpecializedArrayH\x00R\x10specializedArray\x1au\n\x07\x44\x65\x63imal\x12\x14\n\x05value\x18\x01 \x01(\tR\x05value\x12!\n\tprecision\x18\x02 \x01(\x05H\x00R\tprecision\x88\x01\x01\x12\x19\n\x05scale\x18\x03 \x01(\x05H\x01R\x05scale\x88\x01\x01\x42\x0c\n\n_precisionB\x08\n\x06_scale\x1a\x62\n\x10\x43\x61lendarInterval\x12\x16\n\x06months\x18\x01 \x01(\x05R\x06months\x12\x12\n\x04\x64\x61ys\x18\x02 \x01(\x05R\x04\x64\x61ys\x12"\n\x0cmicroseconds\x18\x03 \x01(\x03R\x0cmicroseconds\x1a\x82\x01\n\x05\x41rray\x12:\n\x0c\x65lement_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x0b\x65lementType\x12=\n\x08\x65lements\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x08\x65lements\x1a\xe3\x01\n\x03Map\x12\x32\n\x08key_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x07keyType\x12\x36\n\nvalue_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\tvalueType\x12\x35\n\x04keys\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x04keys\x12\x39\n\x06values\x18\x04 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1a\x81\x01\n\x06Struct\x12\x38\n\x0bstruct_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\nstructType\x12=\n\x08\x65lements\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x08\x65lements\x1a\xc0\x02\n\x10SpecializedArray\x12,\n\x05\x62ools\x18\x01 \x01(\x0b\x32\x14.spark.connect.BoolsH\x00R\x05\x62ools\x12)\n\x04ints\x18\x02 \x01(\x0b\x32\x13.spark.connect.IntsH\x00R\x04ints\x12,\n\x05longs\x18\x03 \x01(\x0b\x32\x14.spark.connect.LongsH\x00R\x05longs\x12/\n\x06\x66loats\x18\x04 \x01(\x0b\x32\x15.spark.connect.FloatsH\x00R\x06\x66loats\x12\x32\n\x07\x64oubles\x18\x05 \x01(\x0b\x32\x16.spark.connect.DoublesH\x00R\x07\x64oubles\x12\x32\n\x07strings\x18\x06 \x01(\x0b\x32\x16.spark.connect.StringsH\x00R\x07stringsB\x0c\n\nvalue_typeB\x0e\n\x0cliteral_type\x1a\xba\x01\n\x13UnresolvedAttribute\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12\x31\n\x12is_metadata_column\x18\x03 \x01(\x08H\x01R\x10isMetadataColumn\x88\x01\x01\x42\n\n\x08_plan_idB\x15\n\x13_is_metadata_column\x1a\x82\x02\n\x12UnresolvedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12\x1f\n\x0bis_distinct\x18\x03 \x01(\x08R\nisDistinct\x12\x37\n\x18is_user_defined_function\x18\x04 \x01(\x08R\x15isUserDefinedFunction\x12$\n\x0bis_internal\x18\x05 \x01(\x08H\x00R\nisInternal\x88\x01\x01\x42\x0e\n\x0c_is_internal\x1a\x32\n\x10\x45xpressionString\x12\x1e\n\nexpression\x18\x01 \x01(\tR\nexpression\x1a|\n\x0eUnresolvedStar\x12,\n\x0funparsed_target\x18\x01 \x01(\tH\x00R\x0eunparsedTarget\x88\x01\x01\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x01R\x06planId\x88\x01\x01\x42\x12\n\x10_unparsed_targetB\n\n\x08_plan_id\x1aV\n\x0fUnresolvedRegex\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x42\n\n\x08_plan_id\x1a\x84\x01\n\x16UnresolvedExtractValue\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12\x39\n\nextraction\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\nextraction\x1a\xbb\x01\n\x0cUpdateFields\x12\x46\n\x11struct_expression\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x10structExpression\x12\x1d\n\nfield_name\x18\x02 \x01(\tR\tfieldName\x12\x44\n\x10value_expression\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0fvalueExpression\x1ax\n\x05\x41lias\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12\x12\n\x04name\x18\x02 \x03(\tR\x04name\x12\x1f\n\x08metadata\x18\x03 \x01(\tH\x00R\x08metadata\x88\x01\x01\x42\x0b\n\t_metadata\x1a\x9e\x01\n\x0eLambdaFunction\x12\x35\n\x08\x66unction\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08\x66unction\x12U\n\targuments\x18\x02 \x03(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableR\targuments\x1a>\n\x1dUnresolvedNamedLambdaVariable\x12\x1d\n\nname_parts\x18\x01 \x03(\tR\tnamePartsB\x0b\n\texpr_type"A\n\x10\x45xpressionCommon\x12-\n\x06origin\x18\x01 \x01(\x0b\x32\x15.spark.connect.OriginR\x06origin"\x8d\x03\n\x1f\x43ommonInlineUserDefinedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12\x39\n\npython_udf\x18\x04 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\tpythonUdf\x12I\n\x10scalar_scala_udf\x18\x05 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFH\x00R\x0escalarScalaUdf\x12\x33\n\x08java_udf\x18\x06 \x01(\x0b\x32\x16.spark.connect.JavaUDFH\x00R\x07javaUdf\x12\x1f\n\x0bis_distinct\x18\x07 \x01(\x08R\nisDistinctB\n\n\x08\x66unction"\xcc\x01\n\tPythonUDF\x12\x38\n\x0boutput_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVer\x12/\n\x13\x61\x64\x64itional_includes\x18\x05 \x03(\tR\x12\x61\x64\x64itionalIncludes"\xd6\x01\n\x0eScalarScalaUDF\x12\x18\n\x07payload\x18\x01 \x01(\x0cR\x07payload\x12\x37\n\ninputTypes\x18\x02 \x03(\x0b\x32\x17.spark.connect.DataTypeR\ninputTypes\x12\x37\n\noutputType\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\x12\x1a\n\x08nullable\x18\x04 \x01(\x08R\x08nullable\x12\x1c\n\taggregate\x18\x05 \x01(\x08R\taggregate"\x95\x01\n\x07JavaUDF\x12\x1d\n\nclass_name\x18\x01 \x01(\tR\tclassName\x12=\n\x0boutput_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\noutputType\x88\x01\x01\x12\x1c\n\taggregate\x18\x03 \x01(\x08R\taggregateB\x0e\n\x0c_output_type"c\n\x18TypedAggregateExpression\x12G\n\x10scalar_scala_udf\x18\x01 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFR\x0escalarScalaUdf"l\n\x0c\x43\x61llFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"\\\n\x17NamedArgumentExpression\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value"\x80\x04\n\x0bMergeAction\x12\x46\n\x0b\x61\x63tion_type\x18\x01 \x01(\x0e\x32%.spark.connect.MergeAction.ActionTypeR\nactionType\x12<\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionH\x00R\tcondition\x88\x01\x01\x12G\n\x0b\x61ssignments\x18\x03 \x03(\x0b\x32%.spark.connect.MergeAction.AssignmentR\x0b\x61ssignments\x1aj\n\nAssignment\x12+\n\x03key\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value"\xa7\x01\n\nActionType\x12\x17\n\x13\x41\x43TION_TYPE_INVALID\x10\x00\x12\x16\n\x12\x41\x43TION_TYPE_DELETE\x10\x01\x12\x16\n\x12\x41\x43TION_TYPE_INSERT\x10\x02\x12\x1b\n\x17\x41\x43TION_TYPE_INSERT_STAR\x10\x03\x12\x16\n\x12\x41\x43TION_TYPE_UPDATE\x10\x04\x12\x1b\n\x17\x41\x43TION_TYPE_UPDATE_STAR\x10\x05\x42\x0c\n\n_condition"\xc5\x05\n\x12SubqueryExpression\x12\x17\n\x07plan_id\x18\x01 \x01(\x03R\x06planId\x12S\n\rsubquery_type\x18\x02 \x01(\x0e\x32..spark.connect.SubqueryExpression.SubqueryTypeR\x0csubqueryType\x12\x62\n\x11table_arg_options\x18\x03 \x01(\x0b\x32\x31.spark.connect.SubqueryExpression.TableArgOptionsH\x00R\x0ftableArgOptions\x88\x01\x01\x12G\n\x12in_subquery_values\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x10inSubqueryValues\x1a\xea\x01\n\x0fTableArgOptions\x12@\n\x0epartition_spec\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\rpartitionSpec\x12\x42\n\norder_spec\x18\x02 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\torderSpec\x12\x37\n\x15with_single_partition\x18\x03 \x01(\x08H\x00R\x13withSinglePartition\x88\x01\x01\x42\x18\n\x16_with_single_partition"\x90\x01\n\x0cSubqueryType\x12\x19\n\x15SUBQUERY_TYPE_UNKNOWN\x10\x00\x12\x18\n\x14SUBQUERY_TYPE_SCALAR\x10\x01\x12\x18\n\x14SUBQUERY_TYPE_EXISTS\x10\x02\x12\x1b\n\x17SUBQUERY_TYPE_TABLE_ARG\x10\x03\x12\x14\n\x10SUBQUERY_TYPE_IN\x10\x04\x42\x14\n\x12_table_arg_optionsB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3'
+ b'\n\x1fspark/connect/expressions.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x19spark/connect/types.proto\x1a\x1aspark/connect/common.proto"\xce\x37\n\nExpression\x12\x37\n\x06\x63ommon\x18\x12 \x01(\x0b\x32\x1f.spark.connect.ExpressionCommonR\x06\x63ommon\x12=\n\x07literal\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x07literal\x12\x62\n\x14unresolved_attribute\x18\x02 \x01(\x0b\x32-.spark.connect.Expression.UnresolvedAttributeH\x00R\x13unresolvedAttribute\x12_\n\x13unresolved_function\x18\x03 \x01(\x0b\x32,.spark.connect.Expression.UnresolvedFunctionH\x00R\x12unresolvedFunction\x12Y\n\x11\x65xpression_string\x18\x04 \x01(\x0b\x32*.spark.connect.Expression.ExpressionStringH\x00R\x10\x65xpressionString\x12S\n\x0funresolved_star\x18\x05 \x01(\x0b\x32(.spark.connect.Expression.UnresolvedStarH\x00R\x0eunresolvedStar\x12\x37\n\x05\x61lias\x18\x06 \x01(\x0b\x32\x1f.spark.connect.Expression.AliasH\x00R\x05\x61lias\x12\x34\n\x04\x63\x61st\x18\x07 \x01(\x0b\x32\x1e.spark.connect.Expression.CastH\x00R\x04\x63\x61st\x12V\n\x10unresolved_regex\x18\x08 \x01(\x0b\x32).spark.connect.Expression.UnresolvedRegexH\x00R\x0funresolvedRegex\x12\x44\n\nsort_order\x18\t \x01(\x0b\x32#.spark.connect.Expression.SortOrderH\x00R\tsortOrder\x12S\n\x0flambda_function\x18\n \x01(\x0b\x32(.spark.connect.Expression.LambdaFunctionH\x00R\x0elambdaFunction\x12:\n\x06window\x18\x0b \x01(\x0b\x32 .spark.connect.Expression.WindowH\x00R\x06window\x12l\n\x18unresolved_extract_value\x18\x0c \x01(\x0b\x32\x30.spark.connect.Expression.UnresolvedExtractValueH\x00R\x16unresolvedExtractValue\x12M\n\rupdate_fields\x18\r \x01(\x0b\x32&.spark.connect.Expression.UpdateFieldsH\x00R\x0cupdateFields\x12\x82\x01\n unresolved_named_lambda_variable\x18\x0e \x01(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableH\x00R\x1dunresolvedNamedLambdaVariable\x12~\n#common_inline_user_defined_function\x18\x0f \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionH\x00R\x1f\x63ommonInlineUserDefinedFunction\x12\x42\n\rcall_function\x18\x10 \x01(\x0b\x32\x1b.spark.connect.CallFunctionH\x00R\x0c\x63\x61llFunction\x12\x64\n\x19named_argument_expression\x18\x11 \x01(\x0b\x32&.spark.connect.NamedArgumentExpressionH\x00R\x17namedArgumentExpression\x12?\n\x0cmerge_action\x18\x13 \x01(\x0b\x32\x1a.spark.connect.MergeActionH\x00R\x0bmergeAction\x12g\n\x1atyped_aggregate_expression\x18\x14 \x01(\x0b\x32\'.spark.connect.TypedAggregateExpressionH\x00R\x18typedAggregateExpression\x12T\n\x13subquery_expression\x18\x15 \x01(\x0b\x32!.spark.connect.SubqueryExpressionH\x00R\x12subqueryExpression\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a\x8f\x06\n\x06Window\x12\x42\n\x0fwindow_function\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0ewindowFunction\x12@\n\x0epartition_spec\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\rpartitionSpec\x12\x42\n\norder_spec\x18\x03 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\torderSpec\x12K\n\nframe_spec\x18\x04 \x01(\x0b\x32,.spark.connect.Expression.Window.WindowFrameR\tframeSpec\x1a\xed\x03\n\x0bWindowFrame\x12U\n\nframe_type\x18\x01 \x01(\x0e\x32\x36.spark.connect.Expression.Window.WindowFrame.FrameTypeR\tframeType\x12P\n\x05lower\x18\x02 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05lower\x12P\n\x05upper\x18\x03 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05upper\x1a\x91\x01\n\rFrameBoundary\x12!\n\x0b\x63urrent_row\x18\x01 \x01(\x08H\x00R\ncurrentRow\x12\x1e\n\tunbounded\x18\x02 \x01(\x08H\x00R\tunbounded\x12\x31\n\x05value\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionH\x00R\x05valueB\n\n\x08\x62oundary"O\n\tFrameType\x12\x18\n\x14\x46RAME_TYPE_UNDEFINED\x10\x00\x12\x12\n\x0e\x46RAME_TYPE_ROW\x10\x01\x12\x14\n\x10\x46RAME_TYPE_RANGE\x10\x02\x1a\xa9\x03\n\tSortOrder\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12O\n\tdirection\x18\x02 \x01(\x0e\x32\x31.spark.connect.Expression.SortOrder.SortDirectionR\tdirection\x12U\n\rnull_ordering\x18\x03 \x01(\x0e\x32\x30.spark.connect.Expression.SortOrder.NullOrderingR\x0cnullOrdering"l\n\rSortDirection\x12\x1e\n\x1aSORT_DIRECTION_UNSPECIFIED\x10\x00\x12\x1c\n\x18SORT_DIRECTION_ASCENDING\x10\x01\x12\x1d\n\x19SORT_DIRECTION_DESCENDING\x10\x02"U\n\x0cNullOrdering\x12\x1a\n\x16SORT_NULLS_UNSPECIFIED\x10\x00\x12\x14\n\x10SORT_NULLS_FIRST\x10\x01\x12\x13\n\x0fSORT_NULLS_LAST\x10\x02\x1a\xbb\x02\n\x04\x43\x61st\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12-\n\x04type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x04type\x12\x1b\n\x08type_str\x18\x03 \x01(\tH\x00R\x07typeStr\x12\x44\n\teval_mode\x18\x04 \x01(\x0e\x32\'.spark.connect.Expression.Cast.EvalModeR\x08\x65valMode"b\n\x08\x45valMode\x12\x19\n\x15\x45VAL_MODE_UNSPECIFIED\x10\x00\x12\x14\n\x10\x45VAL_MODE_LEGACY\x10\x01\x12\x12\n\x0e\x45VAL_MODE_ANSI\x10\x02\x12\x11\n\rEVAL_MODE_TRY\x10\x03\x42\x0e\n\x0c\x63\x61st_to_type\x1a\x9c\x12\n\x07Literal\x12-\n\x04null\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x04null\x12\x18\n\x06\x62inary\x18\x02 \x01(\x0cH\x00R\x06\x62inary\x12\x1a\n\x07\x62oolean\x18\x03 \x01(\x08H\x00R\x07\x62oolean\x12\x14\n\x04\x62yte\x18\x04 \x01(\x05H\x00R\x04\x62yte\x12\x16\n\x05short\x18\x05 \x01(\x05H\x00R\x05short\x12\x1a\n\x07integer\x18\x06 \x01(\x05H\x00R\x07integer\x12\x14\n\x04long\x18\x07 \x01(\x03H\x00R\x04long\x12\x16\n\x05\x66loat\x18\n \x01(\x02H\x00R\x05\x66loat\x12\x18\n\x06\x64ouble\x18\x0b \x01(\x01H\x00R\x06\x64ouble\x12\x45\n\x07\x64\x65\x63imal\x18\x0c \x01(\x0b\x32).spark.connect.Expression.Literal.DecimalH\x00R\x07\x64\x65\x63imal\x12\x18\n\x06string\x18\r \x01(\tH\x00R\x06string\x12\x14\n\x04\x64\x61te\x18\x10 \x01(\x05H\x00R\x04\x64\x61te\x12\x1e\n\ttimestamp\x18\x11 \x01(\x03H\x00R\ttimestamp\x12%\n\rtimestamp_ntz\x18\x12 \x01(\x03H\x00R\x0ctimestampNtz\x12\x61\n\x11\x63\x61lendar_interval\x18\x13 \x01(\x0b\x32\x32.spark.connect.Expression.Literal.CalendarIntervalH\x00R\x10\x63\x61lendarInterval\x12\x30\n\x13year_month_interval\x18\x14 \x01(\x05H\x00R\x11yearMonthInterval\x12,\n\x11\x64\x61y_time_interval\x18\x15 \x01(\x03H\x00R\x0f\x64\x61yTimeInterval\x12?\n\x05\x61rray\x18\x16 \x01(\x0b\x32\'.spark.connect.Expression.Literal.ArrayH\x00R\x05\x61rray\x12\x39\n\x03map\x18\x17 \x01(\x0b\x32%.spark.connect.Expression.Literal.MapH\x00R\x03map\x12\x42\n\x06struct\x18\x18 \x01(\x0b\x32(.spark.connect.Expression.Literal.StructH\x00R\x06struct\x12\x61\n\x11specialized_array\x18\x19 \x01(\x0b\x32\x32.spark.connect.Expression.Literal.SpecializedArrayH\x00R\x10specializedArray\x12<\n\x04time\x18\x1a \x01(\x0b\x32&.spark.connect.Expression.Literal.TimeH\x00R\x04time\x1au\n\x07\x44\x65\x63imal\x12\x14\n\x05value\x18\x01 \x01(\tR\x05value\x12!\n\tprecision\x18\x02 \x01(\x05H\x00R\tprecision\x88\x01\x01\x12\x19\n\x05scale\x18\x03 \x01(\x05H\x01R\x05scale\x88\x01\x01\x42\x0c\n\n_precisionB\x08\n\x06_scale\x1a\x62\n\x10\x43\x61lendarInterval\x12\x16\n\x06months\x18\x01 \x01(\x05R\x06months\x12\x12\n\x04\x64\x61ys\x18\x02 \x01(\x05R\x04\x64\x61ys\x12"\n\x0cmicroseconds\x18\x03 \x01(\x03R\x0cmicroseconds\x1a\xc2\x01\n\x05\x41rray\x12>\n\x0c\x65lement_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeB\x02\x18\x01R\x0b\x65lementType\x12=\n\x08\x65lements\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x08\x65lements\x12:\n\tdata_type\x18\x03 \x01(\x0b\x32\x1d.spark.connect.DataType.ArrayR\x08\x64\x61taType\x1a\xa5\x02\n\x03Map\x12\x36\n\x08key_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeB\x02\x18\x01R\x07keyType\x12:\n\nvalue_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeB\x02\x18\x01R\tvalueType\x12\x35\n\x04keys\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x04keys\x12\x39\n\x06values\x18\x04 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x12\x38\n\tdata_type\x18\x05 \x01(\x0b\x32\x1b.spark.connect.DataType.MapR\x08\x64\x61taType\x1a\xcf\x01\n\x06Struct\x12<\n\x0bstruct_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeB\x02\x18\x01R\nstructType\x12=\n\x08\x65lements\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x08\x65lements\x12H\n\x10\x64\x61ta_type_struct\x18\x03 \x01(\x0b\x32\x1e.spark.connect.DataType.StructR\x0e\x64\x61taTypeStruct\x1a\xc0\x02\n\x10SpecializedArray\x12,\n\x05\x62ools\x18\x01 \x01(\x0b\x32\x14.spark.connect.BoolsH\x00R\x05\x62ools\x12)\n\x04ints\x18\x02 \x01(\x0b\x32\x13.spark.connect.IntsH\x00R\x04ints\x12,\n\x05longs\x18\x03 \x01(\x0b\x32\x14.spark.connect.LongsH\x00R\x05longs\x12/\n\x06\x66loats\x18\x04 \x01(\x0b\x32\x15.spark.connect.FloatsH\x00R\x06\x66loats\x12\x32\n\x07\x64oubles\x18\x05 \x01(\x0b\x32\x16.spark.connect.DoublesH\x00R\x07\x64oubles\x12\x32\n\x07strings\x18\x06 \x01(\x0b\x32\x16.spark.connect.StringsH\x00R\x07stringsB\x0c\n\nvalue_type\x1aK\n\x04Time\x12\x12\n\x04nano\x18\x01 \x01(\x03R\x04nano\x12!\n\tprecision\x18\x02 \x01(\x05H\x00R\tprecision\x88\x01\x01\x42\x0c\n\n_precisionB\x0e\n\x0cliteral_type\x1a\xba\x01\n\x13UnresolvedAttribute\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12\x31\n\x12is_metadata_column\x18\x03 \x01(\x08H\x01R\x10isMetadataColumn\x88\x01\x01\x42\n\n\x08_plan_idB\x15\n\x13_is_metadata_column\x1a\x82\x02\n\x12UnresolvedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12\x1f\n\x0bis_distinct\x18\x03 \x01(\x08R\nisDistinct\x12\x37\n\x18is_user_defined_function\x18\x04 \x01(\x08R\x15isUserDefinedFunction\x12$\n\x0bis_internal\x18\x05 \x01(\x08H\x00R\nisInternal\x88\x01\x01\x42\x0e\n\x0c_is_internal\x1a\x32\n\x10\x45xpressionString\x12\x1e\n\nexpression\x18\x01 \x01(\tR\nexpression\x1a|\n\x0eUnresolvedStar\x12,\n\x0funparsed_target\x18\x01 \x01(\tH\x00R\x0eunparsedTarget\x88\x01\x01\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x01R\x06planId\x88\x01\x01\x42\x12\n\x10_unparsed_targetB\n\n\x08_plan_id\x1aV\n\x0fUnresolvedRegex\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x42\n\n\x08_plan_id\x1a\x84\x01\n\x16UnresolvedExtractValue\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12\x39\n\nextraction\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\nextraction\x1a\xbb\x01\n\x0cUpdateFields\x12\x46\n\x11struct_expression\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x10structExpression\x12\x1d\n\nfield_name\x18\x02 \x01(\tR\tfieldName\x12\x44\n\x10value_expression\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0fvalueExpression\x1ax\n\x05\x41lias\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12\x12\n\x04name\x18\x02 \x03(\tR\x04name\x12\x1f\n\x08metadata\x18\x03 \x01(\tH\x00R\x08metadata\x88\x01\x01\x42\x0b\n\t_metadata\x1a\x9e\x01\n\x0eLambdaFunction\x12\x35\n\x08\x66unction\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08\x66unction\x12U\n\targuments\x18\x02 \x03(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableR\targuments\x1a>\n\x1dUnresolvedNamedLambdaVariable\x12\x1d\n\nname_parts\x18\x01 \x03(\tR\tnamePartsB\x0b\n\texpr_type"A\n\x10\x45xpressionCommon\x12-\n\x06origin\x18\x01 \x01(\x0b\x32\x15.spark.connect.OriginR\x06origin"\x8d\x03\n\x1f\x43ommonInlineUserDefinedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12\x39\n\npython_udf\x18\x04 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\tpythonUdf\x12I\n\x10scalar_scala_udf\x18\x05 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFH\x00R\x0escalarScalaUdf\x12\x33\n\x08java_udf\x18\x06 \x01(\x0b\x32\x16.spark.connect.JavaUDFH\x00R\x07javaUdf\x12\x1f\n\x0bis_distinct\x18\x07 \x01(\x08R\nisDistinctB\n\n\x08\x66unction"\xcc\x01\n\tPythonUDF\x12\x38\n\x0boutput_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVer\x12/\n\x13\x61\x64\x64itional_includes\x18\x05 \x03(\tR\x12\x61\x64\x64itionalIncludes"\xd6\x01\n\x0eScalarScalaUDF\x12\x18\n\x07payload\x18\x01 \x01(\x0cR\x07payload\x12\x37\n\ninputTypes\x18\x02 \x03(\x0b\x32\x17.spark.connect.DataTypeR\ninputTypes\x12\x37\n\noutputType\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\x12\x1a\n\x08nullable\x18\x04 \x01(\x08R\x08nullable\x12\x1c\n\taggregate\x18\x05 \x01(\x08R\taggregate"\x95\x01\n\x07JavaUDF\x12\x1d\n\nclass_name\x18\x01 \x01(\tR\tclassName\x12=\n\x0boutput_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\noutputType\x88\x01\x01\x12\x1c\n\taggregate\x18\x03 \x01(\x08R\taggregateB\x0e\n\x0c_output_type"c\n\x18TypedAggregateExpression\x12G\n\x10scalar_scala_udf\x18\x01 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFR\x0escalarScalaUdf"l\n\x0c\x43\x61llFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"\\\n\x17NamedArgumentExpression\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value"\x80\x04\n\x0bMergeAction\x12\x46\n\x0b\x61\x63tion_type\x18\x01 \x01(\x0e\x32%.spark.connect.MergeAction.ActionTypeR\nactionType\x12<\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionH\x00R\tcondition\x88\x01\x01\x12G\n\x0b\x61ssignments\x18\x03 \x03(\x0b\x32%.spark.connect.MergeAction.AssignmentR\x0b\x61ssignments\x1aj\n\nAssignment\x12+\n\x03key\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value"\xa7\x01\n\nActionType\x12\x17\n\x13\x41\x43TION_TYPE_INVALID\x10\x00\x12\x16\n\x12\x41\x43TION_TYPE_DELETE\x10\x01\x12\x16\n\x12\x41\x43TION_TYPE_INSERT\x10\x02\x12\x1b\n\x17\x41\x43TION_TYPE_INSERT_STAR\x10\x03\x12\x16\n\x12\x41\x43TION_TYPE_UPDATE\x10\x04\x12\x1b\n\x17\x41\x43TION_TYPE_UPDATE_STAR\x10\x05\x42\x0c\n\n_condition"\xc5\x05\n\x12SubqueryExpression\x12\x17\n\x07plan_id\x18\x01 \x01(\x03R\x06planId\x12S\n\rsubquery_type\x18\x02 \x01(\x0e\x32..spark.connect.SubqueryExpression.SubqueryTypeR\x0csubqueryType\x12\x62\n\x11table_arg_options\x18\x03 \x01(\x0b\x32\x31.spark.connect.SubqueryExpression.TableArgOptionsH\x00R\x0ftableArgOptions\x88\x01\x01\x12G\n\x12in_subquery_values\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x10inSubqueryValues\x1a\xea\x01\n\x0fTableArgOptions\x12@\n\x0epartition_spec\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\rpartitionSpec\x12\x42\n\norder_spec\x18\x02 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\torderSpec\x12\x37\n\x15with_single_partition\x18\x03 \x01(\x08H\x00R\x13withSinglePartition\x88\x01\x01\x42\x18\n\x16_with_single_partition"\x90\x01\n\x0cSubqueryType\x12\x19\n\x15SUBQUERY_TYPE_UNKNOWN\x10\x00\x12\x18\n\x14SUBQUERY_TYPE_SCALAR\x10\x01\x12\x18\n\x14SUBQUERY_TYPE_EXISTS\x10\x02\x12\x1b\n\x17SUBQUERY_TYPE_TABLE_ARG\x10\x03\x12\x14\n\x10SUBQUERY_TYPE_IN\x10\x04\x42\x14\n\x12_table_arg_optionsB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3'
)
_globals = globals()
@@ -53,8 +53,22 @@
_globals[
"DESCRIPTOR"
]._serialized_options = b"\n\036org.apache.spark.connect.protoP\001Z\022internal/generated"
+ _globals["_EXPRESSION_LITERAL_ARRAY"].fields_by_name["element_type"]._loaded_options = None
+ _globals["_EXPRESSION_LITERAL_ARRAY"].fields_by_name[
+ "element_type"
+ ]._serialized_options = b"\030\001"
+ _globals["_EXPRESSION_LITERAL_MAP"].fields_by_name["key_type"]._loaded_options = None
+ _globals["_EXPRESSION_LITERAL_MAP"].fields_by_name["key_type"]._serialized_options = b"\030\001"
+ _globals["_EXPRESSION_LITERAL_MAP"].fields_by_name["value_type"]._loaded_options = None
+ _globals["_EXPRESSION_LITERAL_MAP"].fields_by_name[
+ "value_type"
+ ]._serialized_options = b"\030\001"
+ _globals["_EXPRESSION_LITERAL_STRUCT"].fields_by_name["struct_type"]._loaded_options = None
+ _globals["_EXPRESSION_LITERAL_STRUCT"].fields_by_name[
+ "struct_type"
+ ]._serialized_options = b"\030\001"
_globals["_EXPRESSION"]._serialized_start = 133
- _globals["_EXPRESSION"]._serialized_end = 6904
+ _globals["_EXPRESSION"]._serialized_end = 7251
_globals["_EXPRESSION_WINDOW"]._serialized_start = 1986
_globals["_EXPRESSION_WINDOW"]._serialized_end = 2769
_globals["_EXPRESSION_WINDOW_WINDOWFRAME"]._serialized_start = 2276
@@ -74,65 +88,67 @@
_globals["_EXPRESSION_CAST_EVALMODE"]._serialized_start = 3401
_globals["_EXPRESSION_CAST_EVALMODE"]._serialized_end = 3499
_globals["_EXPRESSION_LITERAL"]._serialized_start = 3518
- _globals["_EXPRESSION_LITERAL"]._serialized_end = 5503
- _globals["_EXPRESSION_LITERAL_DECIMAL"]._serialized_start = 4452
- _globals["_EXPRESSION_LITERAL_DECIMAL"]._serialized_end = 4569
- _globals["_EXPRESSION_LITERAL_CALENDARINTERVAL"]._serialized_start = 4571
- _globals["_EXPRESSION_LITERAL_CALENDARINTERVAL"]._serialized_end = 4669
- _globals["_EXPRESSION_LITERAL_ARRAY"]._serialized_start = 4672
- _globals["_EXPRESSION_LITERAL_ARRAY"]._serialized_end = 4802
- _globals["_EXPRESSION_LITERAL_MAP"]._serialized_start = 4805
- _globals["_EXPRESSION_LITERAL_MAP"]._serialized_end = 5032
- _globals["_EXPRESSION_LITERAL_STRUCT"]._serialized_start = 5035
- _globals["_EXPRESSION_LITERAL_STRUCT"]._serialized_end = 5164
- _globals["_EXPRESSION_LITERAL_SPECIALIZEDARRAY"]._serialized_start = 5167
- _globals["_EXPRESSION_LITERAL_SPECIALIZEDARRAY"]._serialized_end = 5487
- _globals["_EXPRESSION_UNRESOLVEDATTRIBUTE"]._serialized_start = 5506
- _globals["_EXPRESSION_UNRESOLVEDATTRIBUTE"]._serialized_end = 5692
- _globals["_EXPRESSION_UNRESOLVEDFUNCTION"]._serialized_start = 5695
- _globals["_EXPRESSION_UNRESOLVEDFUNCTION"]._serialized_end = 5953
- _globals["_EXPRESSION_EXPRESSIONSTRING"]._serialized_start = 5955
- _globals["_EXPRESSION_EXPRESSIONSTRING"]._serialized_end = 6005
- _globals["_EXPRESSION_UNRESOLVEDSTAR"]._serialized_start = 6007
- _globals["_EXPRESSION_UNRESOLVEDSTAR"]._serialized_end = 6131
- _globals["_EXPRESSION_UNRESOLVEDREGEX"]._serialized_start = 6133
- _globals["_EXPRESSION_UNRESOLVEDREGEX"]._serialized_end = 6219
- _globals["_EXPRESSION_UNRESOLVEDEXTRACTVALUE"]._serialized_start = 6222
- _globals["_EXPRESSION_UNRESOLVEDEXTRACTVALUE"]._serialized_end = 6354
- _globals["_EXPRESSION_UPDATEFIELDS"]._serialized_start = 6357
- _globals["_EXPRESSION_UPDATEFIELDS"]._serialized_end = 6544
- _globals["_EXPRESSION_ALIAS"]._serialized_start = 6546
- _globals["_EXPRESSION_ALIAS"]._serialized_end = 6666
- _globals["_EXPRESSION_LAMBDAFUNCTION"]._serialized_start = 6669
- _globals["_EXPRESSION_LAMBDAFUNCTION"]._serialized_end = 6827
- _globals["_EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE"]._serialized_start = 6829
- _globals["_EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE"]._serialized_end = 6891
- _globals["_EXPRESSIONCOMMON"]._serialized_start = 6906
- _globals["_EXPRESSIONCOMMON"]._serialized_end = 6971
- _globals["_COMMONINLINEUSERDEFINEDFUNCTION"]._serialized_start = 6974
- _globals["_COMMONINLINEUSERDEFINEDFUNCTION"]._serialized_end = 7371
- _globals["_PYTHONUDF"]._serialized_start = 7374
- _globals["_PYTHONUDF"]._serialized_end = 7578
- _globals["_SCALARSCALAUDF"]._serialized_start = 7581
- _globals["_SCALARSCALAUDF"]._serialized_end = 7795
- _globals["_JAVAUDF"]._serialized_start = 7798
- _globals["_JAVAUDF"]._serialized_end = 7947
- _globals["_TYPEDAGGREGATEEXPRESSION"]._serialized_start = 7949
- _globals["_TYPEDAGGREGATEEXPRESSION"]._serialized_end = 8048
- _globals["_CALLFUNCTION"]._serialized_start = 8050
- _globals["_CALLFUNCTION"]._serialized_end = 8158
- _globals["_NAMEDARGUMENTEXPRESSION"]._serialized_start = 8160
- _globals["_NAMEDARGUMENTEXPRESSION"]._serialized_end = 8252
- _globals["_MERGEACTION"]._serialized_start = 8255
- _globals["_MERGEACTION"]._serialized_end = 8767
- _globals["_MERGEACTION_ASSIGNMENT"]._serialized_start = 8477
- _globals["_MERGEACTION_ASSIGNMENT"]._serialized_end = 8583
- _globals["_MERGEACTION_ACTIONTYPE"]._serialized_start = 8586
- _globals["_MERGEACTION_ACTIONTYPE"]._serialized_end = 8753
- _globals["_SUBQUERYEXPRESSION"]._serialized_start = 8770
- _globals["_SUBQUERYEXPRESSION"]._serialized_end = 9479
- _globals["_SUBQUERYEXPRESSION_TABLEARGOPTIONS"]._serialized_start = 9076
- _globals["_SUBQUERYEXPRESSION_TABLEARGOPTIONS"]._serialized_end = 9310
- _globals["_SUBQUERYEXPRESSION_SUBQUERYTYPE"]._serialized_start = 9313
- _globals["_SUBQUERYEXPRESSION_SUBQUERYTYPE"]._serialized_end = 9457
+ _globals["_EXPRESSION_LITERAL"]._serialized_end = 5850
+ _globals["_EXPRESSION_LITERAL_DECIMAL"]._serialized_start = 4514
+ _globals["_EXPRESSION_LITERAL_DECIMAL"]._serialized_end = 4631
+ _globals["_EXPRESSION_LITERAL_CALENDARINTERVAL"]._serialized_start = 4633
+ _globals["_EXPRESSION_LITERAL_CALENDARINTERVAL"]._serialized_end = 4731
+ _globals["_EXPRESSION_LITERAL_ARRAY"]._serialized_start = 4734
+ _globals["_EXPRESSION_LITERAL_ARRAY"]._serialized_end = 4928
+ _globals["_EXPRESSION_LITERAL_MAP"]._serialized_start = 4931
+ _globals["_EXPRESSION_LITERAL_MAP"]._serialized_end = 5224
+ _globals["_EXPRESSION_LITERAL_STRUCT"]._serialized_start = 5227
+ _globals["_EXPRESSION_LITERAL_STRUCT"]._serialized_end = 5434
+ _globals["_EXPRESSION_LITERAL_SPECIALIZEDARRAY"]._serialized_start = 5437
+ _globals["_EXPRESSION_LITERAL_SPECIALIZEDARRAY"]._serialized_end = 5757
+ _globals["_EXPRESSION_LITERAL_TIME"]._serialized_start = 5759
+ _globals["_EXPRESSION_LITERAL_TIME"]._serialized_end = 5834
+ _globals["_EXPRESSION_UNRESOLVEDATTRIBUTE"]._serialized_start = 5853
+ _globals["_EXPRESSION_UNRESOLVEDATTRIBUTE"]._serialized_end = 6039
+ _globals["_EXPRESSION_UNRESOLVEDFUNCTION"]._serialized_start = 6042
+ _globals["_EXPRESSION_UNRESOLVEDFUNCTION"]._serialized_end = 6300
+ _globals["_EXPRESSION_EXPRESSIONSTRING"]._serialized_start = 6302
+ _globals["_EXPRESSION_EXPRESSIONSTRING"]._serialized_end = 6352
+ _globals["_EXPRESSION_UNRESOLVEDSTAR"]._serialized_start = 6354
+ _globals["_EXPRESSION_UNRESOLVEDSTAR"]._serialized_end = 6478
+ _globals["_EXPRESSION_UNRESOLVEDREGEX"]._serialized_start = 6480
+ _globals["_EXPRESSION_UNRESOLVEDREGEX"]._serialized_end = 6566
+ _globals["_EXPRESSION_UNRESOLVEDEXTRACTVALUE"]._serialized_start = 6569
+ _globals["_EXPRESSION_UNRESOLVEDEXTRACTVALUE"]._serialized_end = 6701
+ _globals["_EXPRESSION_UPDATEFIELDS"]._serialized_start = 6704
+ _globals["_EXPRESSION_UPDATEFIELDS"]._serialized_end = 6891
+ _globals["_EXPRESSION_ALIAS"]._serialized_start = 6893
+ _globals["_EXPRESSION_ALIAS"]._serialized_end = 7013
+ _globals["_EXPRESSION_LAMBDAFUNCTION"]._serialized_start = 7016
+ _globals["_EXPRESSION_LAMBDAFUNCTION"]._serialized_end = 7174
+ _globals["_EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE"]._serialized_start = 7176
+ _globals["_EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE"]._serialized_end = 7238
+ _globals["_EXPRESSIONCOMMON"]._serialized_start = 7253
+ _globals["_EXPRESSIONCOMMON"]._serialized_end = 7318
+ _globals["_COMMONINLINEUSERDEFINEDFUNCTION"]._serialized_start = 7321
+ _globals["_COMMONINLINEUSERDEFINEDFUNCTION"]._serialized_end = 7718
+ _globals["_PYTHONUDF"]._serialized_start = 7721
+ _globals["_PYTHONUDF"]._serialized_end = 7925
+ _globals["_SCALARSCALAUDF"]._serialized_start = 7928
+ _globals["_SCALARSCALAUDF"]._serialized_end = 8142
+ _globals["_JAVAUDF"]._serialized_start = 8145
+ _globals["_JAVAUDF"]._serialized_end = 8294
+ _globals["_TYPEDAGGREGATEEXPRESSION"]._serialized_start = 8296
+ _globals["_TYPEDAGGREGATEEXPRESSION"]._serialized_end = 8395
+ _globals["_CALLFUNCTION"]._serialized_start = 8397
+ _globals["_CALLFUNCTION"]._serialized_end = 8505
+ _globals["_NAMEDARGUMENTEXPRESSION"]._serialized_start = 8507
+ _globals["_NAMEDARGUMENTEXPRESSION"]._serialized_end = 8599
+ _globals["_MERGEACTION"]._serialized_start = 8602
+ _globals["_MERGEACTION"]._serialized_end = 9114
+ _globals["_MERGEACTION_ASSIGNMENT"]._serialized_start = 8824
+ _globals["_MERGEACTION_ASSIGNMENT"]._serialized_end = 8930
+ _globals["_MERGEACTION_ACTIONTYPE"]._serialized_start = 8933
+ _globals["_MERGEACTION_ACTIONTYPE"]._serialized_end = 9100
+ _globals["_SUBQUERYEXPRESSION"]._serialized_start = 9117
+ _globals["_SUBQUERYEXPRESSION"]._serialized_end = 9826
+ _globals["_SUBQUERYEXPRESSION_TABLEARGOPTIONS"]._serialized_start = 9423
+ _globals["_SUBQUERYEXPRESSION_TABLEARGOPTIONS"]._serialized_end = 9657
+ _globals["_SUBQUERYEXPRESSION_SUBQUERYTYPE"]._serialized_start = 9660
+ _globals["_SUBQUERYEXPRESSION_SUBQUERYTYPE"]._serialized_end = 9804
# @@protoc_insertion_point(module_scope)
diff --git a/python/pyspark/sql/connect/proto/expressions_pb2.pyi b/python/pyspark/sql/connect/proto/expressions_pb2.pyi
index 25fc04c0319e6..508a11a01c85e 100644
--- a/python/pyspark/sql/connect/proto/expressions_pb2.pyi
+++ b/python/pyspark/sql/connect/proto/expressions_pb2.pyi
@@ -474,27 +474,51 @@ class Expression(google.protobuf.message.Message):
ELEMENT_TYPE_FIELD_NUMBER: builtins.int
ELEMENTS_FIELD_NUMBER: builtins.int
+ DATA_TYPE_FIELD_NUMBER: builtins.int
@property
- def element_type(self) -> pyspark.sql.connect.proto.types_pb2.DataType: ...
+ def element_type(self) -> pyspark.sql.connect.proto.types_pb2.DataType:
+ """(Deprecated) The element type of the array.
+
+ This field is deprecated since Spark 4.1+ and should only be set
+ if the data_type field is not set. Use data_type field instead.
+ """
@property
def elements(
self,
) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[
global___Expression.Literal
- ]: ...
+ ]:
+ """The literal values that make up the array elements."""
+ @property
+ def data_type(self) -> pyspark.sql.connect.proto.types_pb2.DataType.Array:
+ """The type of the array.
+
+ If the element type can be inferred from the first element of the elements field,
+ then you don't need to set data_type.element_type to save space. On the other hand,
+ redundant type information is also acceptable.
+ """
def __init__(
self,
*,
element_type: pyspark.sql.connect.proto.types_pb2.DataType | None = ...,
elements: collections.abc.Iterable[global___Expression.Literal] | None = ...,
+ data_type: pyspark.sql.connect.proto.types_pb2.DataType.Array | None = ...,
) -> None: ...
def HasField(
- self, field_name: typing_extensions.Literal["element_type", b"element_type"]
+ self,
+ field_name: typing_extensions.Literal[
+ "data_type", b"data_type", "element_type", b"element_type"
+ ],
) -> builtins.bool: ...
def ClearField(
self,
field_name: typing_extensions.Literal[
- "element_type", b"element_type", "elements", b"elements"
+ "data_type",
+ b"data_type",
+ "element_type",
+ b"element_type",
+ "elements",
+ b"elements",
],
) -> None: ...
@@ -505,22 +529,43 @@ class Expression(google.protobuf.message.Message):
VALUE_TYPE_FIELD_NUMBER: builtins.int
KEYS_FIELD_NUMBER: builtins.int
VALUES_FIELD_NUMBER: builtins.int
+ DATA_TYPE_FIELD_NUMBER: builtins.int
@property
- def key_type(self) -> pyspark.sql.connect.proto.types_pb2.DataType: ...
+ def key_type(self) -> pyspark.sql.connect.proto.types_pb2.DataType:
+ """(Deprecated) The key type of the map.
+
+ This field is deprecated since Spark 4.1+ and should only be set
+ if the data_type field is not set. Use data_type field instead.
+ """
@property
- def value_type(self) -> pyspark.sql.connect.proto.types_pb2.DataType: ...
+ def value_type(self) -> pyspark.sql.connect.proto.types_pb2.DataType:
+ """(Deprecated) The value type of the map.
+
+ This field is deprecated since Spark 4.1+ and should only be set
+ if the data_type field is not set. Use data_type field instead.
+ """
@property
def keys(
self,
) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[
global___Expression.Literal
- ]: ...
+ ]:
+ """The literal keys that make up the map."""
@property
def values(
self,
) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[
global___Expression.Literal
- ]: ...
+ ]:
+ """The literal values that make up the map."""
+ @property
+ def data_type(self) -> pyspark.sql.connect.proto.types_pb2.DataType.Map:
+ """The type of the map.
+
+ If the key/value types can be inferred from the first element of the keys/values fields,
+ then you don't need to set data_type.key_type/data_type.value_type to save space.
+ On the other hand, redundant type information is also acceptable.
+ """
def __init__(
self,
*,
@@ -528,16 +573,19 @@ class Expression(google.protobuf.message.Message):
value_type: pyspark.sql.connect.proto.types_pb2.DataType | None = ...,
keys: collections.abc.Iterable[global___Expression.Literal] | None = ...,
values: collections.abc.Iterable[global___Expression.Literal] | None = ...,
+ data_type: pyspark.sql.connect.proto.types_pb2.DataType.Map | None = ...,
) -> None: ...
def HasField(
self,
field_name: typing_extensions.Literal[
- "key_type", b"key_type", "value_type", b"value_type"
+ "data_type", b"data_type", "key_type", b"key_type", "value_type", b"value_type"
],
) -> builtins.bool: ...
def ClearField(
self,
field_name: typing_extensions.Literal[
+ "data_type",
+ b"data_type",
"key_type",
b"key_type",
"keys",
@@ -554,27 +602,51 @@ class Expression(google.protobuf.message.Message):
STRUCT_TYPE_FIELD_NUMBER: builtins.int
ELEMENTS_FIELD_NUMBER: builtins.int
+ DATA_TYPE_STRUCT_FIELD_NUMBER: builtins.int
@property
- def struct_type(self) -> pyspark.sql.connect.proto.types_pb2.DataType: ...
+ def struct_type(self) -> pyspark.sql.connect.proto.types_pb2.DataType:
+ """(Deprecated) The type of the struct.
+
+ This field is deprecated since Spark 4.1+ because using DataType as the type of a struct
+ is ambiguous. This field should only be set if the data_type_struct field is not set.
+ Use data_type_struct field instead.
+ """
@property
def elements(
self,
) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[
global___Expression.Literal
- ]: ...
+ ]:
+ """(Required) The literal values that make up the struct elements."""
+ @property
+ def data_type_struct(self) -> pyspark.sql.connect.proto.types_pb2.DataType.Struct:
+ """The type of the struct.
+
+ Whether data_type_struct.fields.data_type should be set depends on
+ whether each field's type can be inferred from the elements field.
+ """
def __init__(
self,
*,
struct_type: pyspark.sql.connect.proto.types_pb2.DataType | None = ...,
elements: collections.abc.Iterable[global___Expression.Literal] | None = ...,
+ data_type_struct: pyspark.sql.connect.proto.types_pb2.DataType.Struct | None = ...,
) -> None: ...
def HasField(
- self, field_name: typing_extensions.Literal["struct_type", b"struct_type"]
+ self,
+ field_name: typing_extensions.Literal[
+ "data_type_struct", b"data_type_struct", "struct_type", b"struct_type"
+ ],
) -> builtins.bool: ...
def ClearField(
self,
field_name: typing_extensions.Literal[
- "elements", b"elements", "struct_type", b"struct_type"
+ "data_type_struct",
+ b"data_type_struct",
+ "elements",
+ b"elements",
+ "struct_type",
+ b"struct_type",
],
) -> None: ...
@@ -654,6 +726,36 @@ class Expression(google.protobuf.message.Message):
| None
): ...
+ class Time(google.protobuf.message.Message):
+ DESCRIPTOR: google.protobuf.descriptor.Descriptor
+
+ NANO_FIELD_NUMBER: builtins.int
+ PRECISION_FIELD_NUMBER: builtins.int
+ nano: builtins.int
+ precision: builtins.int
+ """The precision of this time, if omitted, uses the default value of MICROS_PRECISION."""
+ def __init__(
+ self,
+ *,
+ nano: builtins.int = ...,
+ precision: builtins.int | None = ...,
+ ) -> None: ...
+ def HasField(
+ self,
+ field_name: typing_extensions.Literal[
+ "_precision", b"_precision", "precision", b"precision"
+ ],
+ ) -> builtins.bool: ...
+ def ClearField(
+ self,
+ field_name: typing_extensions.Literal[
+ "_precision", b"_precision", "nano", b"nano", "precision", b"precision"
+ ],
+ ) -> None: ...
+ def WhichOneof(
+ self, oneof_group: typing_extensions.Literal["_precision", b"_precision"]
+ ) -> typing_extensions.Literal["precision"] | None: ...
+
NULL_FIELD_NUMBER: builtins.int
BINARY_FIELD_NUMBER: builtins.int
BOOLEAN_FIELD_NUMBER: builtins.int
@@ -675,6 +777,7 @@ class Expression(google.protobuf.message.Message):
MAP_FIELD_NUMBER: builtins.int
STRUCT_FIELD_NUMBER: builtins.int
SPECIALIZED_ARRAY_FIELD_NUMBER: builtins.int
+ TIME_FIELD_NUMBER: builtins.int
@property
def null(self) -> pyspark.sql.connect.proto.types_pb2.DataType: ...
binary: builtins.bytes
@@ -706,6 +809,8 @@ class Expression(google.protobuf.message.Message):
def struct(self) -> global___Expression.Literal.Struct: ...
@property
def specialized_array(self) -> global___Expression.Literal.SpecializedArray: ...
+ @property
+ def time(self) -> global___Expression.Literal.Time: ...
def __init__(
self,
*,
@@ -730,6 +835,7 @@ class Expression(google.protobuf.message.Message):
map: global___Expression.Literal.Map | None = ...,
struct: global___Expression.Literal.Struct | None = ...,
specialized_array: global___Expression.Literal.SpecializedArray | None = ...,
+ time: global___Expression.Literal.Time | None = ...,
) -> None: ...
def HasField(
self,
@@ -772,6 +878,8 @@ class Expression(google.protobuf.message.Message):
b"string",
"struct",
b"struct",
+ "time",
+ b"time",
"timestamp",
b"timestamp",
"timestamp_ntz",
@@ -821,6 +929,8 @@ class Expression(google.protobuf.message.Message):
b"string",
"struct",
b"struct",
+ "time",
+ b"time",
"timestamp",
b"timestamp",
"timestamp_ntz",
@@ -854,6 +964,7 @@ class Expression(google.protobuf.message.Message):
"map",
"struct",
"specialized_array",
+ "time",
]
| None
): ...
diff --git a/python/pyspark/sql/connect/proto/ml_pb2.py b/python/pyspark/sql/connect/proto/ml_pb2.py
index 46fc82131a9e7..4c1b4038c35e3 100644
--- a/python/pyspark/sql/connect/proto/ml_pb2.py
+++ b/python/pyspark/sql/connect/proto/ml_pb2.py
@@ -40,7 +40,7 @@
DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(
- b'\n\x16spark/connect/ml.proto\x12\rspark.connect\x1a\x1dspark/connect/relations.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/ml_common.proto"\xb2\x0b\n\tMlCommand\x12\x30\n\x03\x66it\x18\x01 \x01(\x0b\x32\x1c.spark.connect.MlCommand.FitH\x00R\x03\x66it\x12,\n\x05\x66\x65tch\x18\x02 \x01(\x0b\x32\x14.spark.connect.FetchH\x00R\x05\x66\x65tch\x12\x39\n\x06\x64\x65lete\x18\x03 \x01(\x0b\x32\x1f.spark.connect.MlCommand.DeleteH\x00R\x06\x64\x65lete\x12\x36\n\x05write\x18\x04 \x01(\x0b\x32\x1e.spark.connect.MlCommand.WriteH\x00R\x05write\x12\x33\n\x04read\x18\x05 \x01(\x0b\x32\x1d.spark.connect.MlCommand.ReadH\x00R\x04read\x12?\n\x08\x65valuate\x18\x06 \x01(\x0b\x32!.spark.connect.MlCommand.EvaluateH\x00R\x08\x65valuate\x12\x46\n\x0b\x63lean_cache\x18\x07 \x01(\x0b\x32#.spark.connect.MlCommand.CleanCacheH\x00R\ncleanCache\x12M\n\x0eget_cache_info\x18\x08 \x01(\x0b\x32%.spark.connect.MlCommand.GetCacheInfoH\x00R\x0cgetCacheInfo\x1a\xb2\x01\n\x03\x46it\x12\x37\n\testimator\x18\x01 \x01(\x0b\x32\x19.spark.connect.MlOperatorR\testimator\x12\x34\n\x06params\x18\x02 \x01(\x0b\x32\x17.spark.connect.MlParamsH\x00R\x06params\x88\x01\x01\x12\x31\n\x07\x64\x61taset\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x07\x64\x61tasetB\t\n\x07_params\x1a=\n\x06\x44\x65lete\x12\x33\n\x08obj_refs\x18\x01 \x03(\x0b\x32\x18.spark.connect.ObjectRefR\x07objRefs\x1a\x0c\n\nCleanCache\x1a\x0e\n\x0cGetCacheInfo\x1a\x9a\x03\n\x05Write\x12\x37\n\x08operator\x18\x01 \x01(\x0b\x32\x19.spark.connect.MlOperatorH\x00R\x08operator\x12\x33\n\x07obj_ref\x18\x02 \x01(\x0b\x32\x18.spark.connect.ObjectRefH\x00R\x06objRef\x12\x34\n\x06params\x18\x03 \x01(\x0b\x32\x17.spark.connect.MlParamsH\x01R\x06params\x88\x01\x01\x12\x12\n\x04path\x18\x04 \x01(\tR\x04path\x12.\n\x10should_overwrite\x18\x05 \x01(\x08H\x02R\x0fshouldOverwrite\x88\x01\x01\x12\x45\n\x07options\x18\x06 \x03(\x0b\x32+.spark.connect.MlCommand.Write.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x06\n\x04typeB\t\n\x07_paramsB\x13\n\x11_should_overwrite\x1aQ\n\x04Read\x12\x35\n\x08operator\x18\x01 \x01(\x0b\x32\x19.spark.connect.MlOperatorR\x08operator\x12\x12\n\x04path\x18\x02 \x01(\tR\x04path\x1a\xb7\x01\n\x08\x45valuate\x12\x37\n\tevaluator\x18\x01 \x01(\x0b\x32\x19.spark.connect.MlOperatorR\tevaluator\x12\x34\n\x06params\x18\x02 \x01(\x0b\x32\x17.spark.connect.MlParamsH\x00R\x06params\x88\x01\x01\x12\x31\n\x07\x64\x61taset\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x07\x64\x61tasetB\t\n\x07_paramsB\t\n\x07\x63ommand"\xd5\x03\n\x0fMlCommandResult\x12\x39\n\x05param\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x05param\x12\x1a\n\x07summary\x18\x02 \x01(\tH\x00R\x07summary\x12T\n\roperator_info\x18\x03 \x01(\x0b\x32-.spark.connect.MlCommandResult.MlOperatorInfoH\x00R\x0coperatorInfo\x1a\x85\x02\n\x0eMlOperatorInfo\x12\x33\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefH\x00R\x06objRef\x12\x14\n\x04name\x18\x02 \x01(\tH\x00R\x04name\x12\x15\n\x03uid\x18\x03 \x01(\tH\x01R\x03uid\x88\x01\x01\x12\x34\n\x06params\x18\x04 \x01(\x0b\x32\x17.spark.connect.MlParamsH\x02R\x06params\x88\x01\x01\x12,\n\x0fwarning_message\x18\x05 \x01(\tH\x03R\x0ewarningMessage\x88\x01\x01\x42\x06\n\x04typeB\x06\n\x04_uidB\t\n\x07_paramsB\x12\n\x10_warning_messageB\r\n\x0bresult_typeB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3'
+ b'\n\x16spark/connect/ml.proto\x12\rspark.connect\x1a\x1dspark/connect/relations.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/ml_common.proto"\xc7\x0e\n\tMlCommand\x12\x30\n\x03\x66it\x18\x01 \x01(\x0b\x32\x1c.spark.connect.MlCommand.FitH\x00R\x03\x66it\x12,\n\x05\x66\x65tch\x18\x02 \x01(\x0b\x32\x14.spark.connect.FetchH\x00R\x05\x66\x65tch\x12\x39\n\x06\x64\x65lete\x18\x03 \x01(\x0b\x32\x1f.spark.connect.MlCommand.DeleteH\x00R\x06\x64\x65lete\x12\x36\n\x05write\x18\x04 \x01(\x0b\x32\x1e.spark.connect.MlCommand.WriteH\x00R\x05write\x12\x33\n\x04read\x18\x05 \x01(\x0b\x32\x1d.spark.connect.MlCommand.ReadH\x00R\x04read\x12?\n\x08\x65valuate\x18\x06 \x01(\x0b\x32!.spark.connect.MlCommand.EvaluateH\x00R\x08\x65valuate\x12\x46\n\x0b\x63lean_cache\x18\x07 \x01(\x0b\x32#.spark.connect.MlCommand.CleanCacheH\x00R\ncleanCache\x12M\n\x0eget_cache_info\x18\x08 \x01(\x0b\x32%.spark.connect.MlCommand.GetCacheInfoH\x00R\x0cgetCacheInfo\x12O\n\x0e\x63reate_summary\x18\t \x01(\x0b\x32&.spark.connect.MlCommand.CreateSummaryH\x00R\rcreateSummary\x12M\n\x0eget_model_size\x18\n \x01(\x0b\x32%.spark.connect.MlCommand.GetModelSizeH\x00R\x0cgetModelSize\x1a\xb2\x01\n\x03\x46it\x12\x37\n\testimator\x18\x01 \x01(\x0b\x32\x19.spark.connect.MlOperatorR\testimator\x12\x34\n\x06params\x18\x02 \x01(\x0b\x32\x17.spark.connect.MlParamsH\x00R\x06params\x88\x01\x01\x12\x31\n\x07\x64\x61taset\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x07\x64\x61tasetB\t\n\x07_params\x1ap\n\x06\x44\x65lete\x12\x33\n\x08obj_refs\x18\x01 \x03(\x0b\x32\x18.spark.connect.ObjectRefR\x07objRefs\x12"\n\nevict_only\x18\x02 \x01(\x08H\x00R\tevictOnly\x88\x01\x01\x42\r\n\x0b_evict_only\x1a\x0c\n\nCleanCache\x1a\x0e\n\x0cGetCacheInfo\x1a\x9a\x03\n\x05Write\x12\x37\n\x08operator\x18\x01 \x01(\x0b\x32\x19.spark.connect.MlOperatorH\x00R\x08operator\x12\x33\n\x07obj_ref\x18\x02 \x01(\x0b\x32\x18.spark.connect.ObjectRefH\x00R\x06objRef\x12\x34\n\x06params\x18\x03 \x01(\x0b\x32\x17.spark.connect.MlParamsH\x01R\x06params\x88\x01\x01\x12\x12\n\x04path\x18\x04 \x01(\tR\x04path\x12.\n\x10should_overwrite\x18\x05 \x01(\x08H\x02R\x0fshouldOverwrite\x88\x01\x01\x12\x45\n\x07options\x18\x06 \x03(\x0b\x32+.spark.connect.MlCommand.Write.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x06\n\x04typeB\t\n\x07_paramsB\x13\n\x11_should_overwrite\x1aQ\n\x04Read\x12\x35\n\x08operator\x18\x01 \x01(\x0b\x32\x19.spark.connect.MlOperatorR\x08operator\x12\x12\n\x04path\x18\x02 \x01(\tR\x04path\x1a\xb7\x01\n\x08\x45valuate\x12\x37\n\tevaluator\x18\x01 \x01(\x0b\x32\x19.spark.connect.MlOperatorR\tevaluator\x12\x34\n\x06params\x18\x02 \x01(\x0b\x32\x17.spark.connect.MlParamsH\x00R\x06params\x88\x01\x01\x12\x31\n\x07\x64\x61taset\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x07\x64\x61tasetB\t\n\x07_params\x1ay\n\rCreateSummary\x12\x35\n\tmodel_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefR\x08modelRef\x12\x31\n\x07\x64\x61taset\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x07\x64\x61taset\x1a\x45\n\x0cGetModelSize\x12\x35\n\tmodel_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefR\x08modelRefB\t\n\x07\x63ommand"\xd5\x03\n\x0fMlCommandResult\x12\x39\n\x05param\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x05param\x12\x1a\n\x07summary\x18\x02 \x01(\tH\x00R\x07summary\x12T\n\roperator_info\x18\x03 \x01(\x0b\x32-.spark.connect.MlCommandResult.MlOperatorInfoH\x00R\x0coperatorInfo\x1a\x85\x02\n\x0eMlOperatorInfo\x12\x33\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefH\x00R\x06objRef\x12\x14\n\x04name\x18\x02 \x01(\tH\x00R\x04name\x12\x15\n\x03uid\x18\x03 \x01(\tH\x01R\x03uid\x88\x01\x01\x12\x34\n\x06params\x18\x04 \x01(\x0b\x32\x17.spark.connect.MlParamsH\x02R\x06params\x88\x01\x01\x12,\n\x0fwarning_message\x18\x05 \x01(\tH\x03R\x0ewarningMessage\x88\x01\x01\x42\x06\n\x04typeB\x06\n\x04_uidB\t\n\x07_paramsB\x12\n\x10_warning_messageB\r\n\x0bresult_typeB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3'
)
_globals = globals()
@@ -54,25 +54,29 @@
_globals["_MLCOMMAND_WRITE_OPTIONSENTRY"]._loaded_options = None
_globals["_MLCOMMAND_WRITE_OPTIONSENTRY"]._serialized_options = b"8\001"
_globals["_MLCOMMAND"]._serialized_start = 137
- _globals["_MLCOMMAND"]._serialized_end = 1595
- _globals["_MLCOMMAND_FIT"]._serialized_start = 631
- _globals["_MLCOMMAND_FIT"]._serialized_end = 809
- _globals["_MLCOMMAND_DELETE"]._serialized_start = 811
- _globals["_MLCOMMAND_DELETE"]._serialized_end = 872
- _globals["_MLCOMMAND_CLEANCACHE"]._serialized_start = 874
- _globals["_MLCOMMAND_CLEANCACHE"]._serialized_end = 886
- _globals["_MLCOMMAND_GETCACHEINFO"]._serialized_start = 888
- _globals["_MLCOMMAND_GETCACHEINFO"]._serialized_end = 902
- _globals["_MLCOMMAND_WRITE"]._serialized_start = 905
- _globals["_MLCOMMAND_WRITE"]._serialized_end = 1315
- _globals["_MLCOMMAND_WRITE_OPTIONSENTRY"]._serialized_start = 1217
- _globals["_MLCOMMAND_WRITE_OPTIONSENTRY"]._serialized_end = 1275
- _globals["_MLCOMMAND_READ"]._serialized_start = 1317
- _globals["_MLCOMMAND_READ"]._serialized_end = 1398
- _globals["_MLCOMMAND_EVALUATE"]._serialized_start = 1401
- _globals["_MLCOMMAND_EVALUATE"]._serialized_end = 1584
- _globals["_MLCOMMANDRESULT"]._serialized_start = 1598
- _globals["_MLCOMMANDRESULT"]._serialized_end = 2067
- _globals["_MLCOMMANDRESULT_MLOPERATORINFO"]._serialized_start = 1791
- _globals["_MLCOMMANDRESULT_MLOPERATORINFO"]._serialized_end = 2052
+ _globals["_MLCOMMAND"]._serialized_end = 2000
+ _globals["_MLCOMMAND_FIT"]._serialized_start = 791
+ _globals["_MLCOMMAND_FIT"]._serialized_end = 969
+ _globals["_MLCOMMAND_DELETE"]._serialized_start = 971
+ _globals["_MLCOMMAND_DELETE"]._serialized_end = 1083
+ _globals["_MLCOMMAND_CLEANCACHE"]._serialized_start = 1085
+ _globals["_MLCOMMAND_CLEANCACHE"]._serialized_end = 1097
+ _globals["_MLCOMMAND_GETCACHEINFO"]._serialized_start = 1099
+ _globals["_MLCOMMAND_GETCACHEINFO"]._serialized_end = 1113
+ _globals["_MLCOMMAND_WRITE"]._serialized_start = 1116
+ _globals["_MLCOMMAND_WRITE"]._serialized_end = 1526
+ _globals["_MLCOMMAND_WRITE_OPTIONSENTRY"]._serialized_start = 1428
+ _globals["_MLCOMMAND_WRITE_OPTIONSENTRY"]._serialized_end = 1486
+ _globals["_MLCOMMAND_READ"]._serialized_start = 1528
+ _globals["_MLCOMMAND_READ"]._serialized_end = 1609
+ _globals["_MLCOMMAND_EVALUATE"]._serialized_start = 1612
+ _globals["_MLCOMMAND_EVALUATE"]._serialized_end = 1795
+ _globals["_MLCOMMAND_CREATESUMMARY"]._serialized_start = 1797
+ _globals["_MLCOMMAND_CREATESUMMARY"]._serialized_end = 1918
+ _globals["_MLCOMMAND_GETMODELSIZE"]._serialized_start = 1920
+ _globals["_MLCOMMAND_GETMODELSIZE"]._serialized_end = 1989
+ _globals["_MLCOMMANDRESULT"]._serialized_start = 2003
+ _globals["_MLCOMMANDRESULT"]._serialized_end = 2472
+ _globals["_MLCOMMANDRESULT_MLOPERATORINFO"]._serialized_start = 2196
+ _globals["_MLCOMMANDRESULT_MLOPERATORINFO"]._serialized_end = 2457
# @@protoc_insertion_point(module_scope)
diff --git a/python/pyspark/sql/connect/proto/ml_pb2.pyi b/python/pyspark/sql/connect/proto/ml_pb2.pyi
index 88cc6cb625ded..156ef846a8d10 100644
--- a/python/pyspark/sql/connect/proto/ml_pb2.pyi
+++ b/python/pyspark/sql/connect/proto/ml_pb2.pyi
@@ -118,21 +118,39 @@ class MlCommand(google.protobuf.message.Message):
DESCRIPTOR: google.protobuf.descriptor.Descriptor
OBJ_REFS_FIELD_NUMBER: builtins.int
+ EVICT_ONLY_FIELD_NUMBER: builtins.int
@property
def obj_refs(
self,
) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[
pyspark.sql.connect.proto.ml_common_pb2.ObjectRef
]: ...
+ evict_only: builtins.bool
+ """if set `evict_only` to true, only evict the cached model from memory,
+ but keep the offloaded model in Spark driver local disk.
+ """
def __init__(
self,
*,
obj_refs: collections.abc.Iterable[pyspark.sql.connect.proto.ml_common_pb2.ObjectRef]
| None = ...,
+ evict_only: builtins.bool | None = ...,
) -> None: ...
+ def HasField(
+ self,
+ field_name: typing_extensions.Literal[
+ "_evict_only", b"_evict_only", "evict_only", b"evict_only"
+ ],
+ ) -> builtins.bool: ...
def ClearField(
- self, field_name: typing_extensions.Literal["obj_refs", b"obj_refs"]
+ self,
+ field_name: typing_extensions.Literal[
+ "_evict_only", b"_evict_only", "evict_only", b"evict_only", "obj_refs", b"obj_refs"
+ ],
) -> None: ...
+ def WhichOneof(
+ self, oneof_group: typing_extensions.Literal["_evict_only", b"_evict_only"]
+ ) -> typing_extensions.Literal["evict_only"] | None: ...
class CleanCache(google.protobuf.message.Message):
"""Force to clean up all the ML cached objects"""
@@ -342,6 +360,54 @@ class MlCommand(google.protobuf.message.Message):
self, oneof_group: typing_extensions.Literal["_params", b"_params"]
) -> typing_extensions.Literal["params"] | None: ...
+ class CreateSummary(google.protobuf.message.Message):
+ """This is for re-creating the model summary when the model summary is lost
+ (model summary is lost when the model is offloaded and then loaded back)
+ """
+
+ DESCRIPTOR: google.protobuf.descriptor.Descriptor
+
+ MODEL_REF_FIELD_NUMBER: builtins.int
+ DATASET_FIELD_NUMBER: builtins.int
+ @property
+ def model_ref(self) -> pyspark.sql.connect.proto.ml_common_pb2.ObjectRef: ...
+ @property
+ def dataset(self) -> pyspark.sql.connect.proto.relations_pb2.Relation: ...
+ def __init__(
+ self,
+ *,
+ model_ref: pyspark.sql.connect.proto.ml_common_pb2.ObjectRef | None = ...,
+ dataset: pyspark.sql.connect.proto.relations_pb2.Relation | None = ...,
+ ) -> None: ...
+ def HasField(
+ self,
+ field_name: typing_extensions.Literal["dataset", b"dataset", "model_ref", b"model_ref"],
+ ) -> builtins.bool: ...
+ def ClearField(
+ self,
+ field_name: typing_extensions.Literal["dataset", b"dataset", "model_ref", b"model_ref"],
+ ) -> None: ...
+
+ class GetModelSize(google.protobuf.message.Message):
+ """This is for query the model estimated in-memory size"""
+
+ DESCRIPTOR: google.protobuf.descriptor.Descriptor
+
+ MODEL_REF_FIELD_NUMBER: builtins.int
+ @property
+ def model_ref(self) -> pyspark.sql.connect.proto.ml_common_pb2.ObjectRef: ...
+ def __init__(
+ self,
+ *,
+ model_ref: pyspark.sql.connect.proto.ml_common_pb2.ObjectRef | None = ...,
+ ) -> None: ...
+ def HasField(
+ self, field_name: typing_extensions.Literal["model_ref", b"model_ref"]
+ ) -> builtins.bool: ...
+ def ClearField(
+ self, field_name: typing_extensions.Literal["model_ref", b"model_ref"]
+ ) -> None: ...
+
FIT_FIELD_NUMBER: builtins.int
FETCH_FIELD_NUMBER: builtins.int
DELETE_FIELD_NUMBER: builtins.int
@@ -350,6 +416,8 @@ class MlCommand(google.protobuf.message.Message):
EVALUATE_FIELD_NUMBER: builtins.int
CLEAN_CACHE_FIELD_NUMBER: builtins.int
GET_CACHE_INFO_FIELD_NUMBER: builtins.int
+ CREATE_SUMMARY_FIELD_NUMBER: builtins.int
+ GET_MODEL_SIZE_FIELD_NUMBER: builtins.int
@property
def fit(self) -> global___MlCommand.Fit: ...
@property
@@ -366,6 +434,10 @@ class MlCommand(google.protobuf.message.Message):
def clean_cache(self) -> global___MlCommand.CleanCache: ...
@property
def get_cache_info(self) -> global___MlCommand.GetCacheInfo: ...
+ @property
+ def create_summary(self) -> global___MlCommand.CreateSummary: ...
+ @property
+ def get_model_size(self) -> global___MlCommand.GetModelSize: ...
def __init__(
self,
*,
@@ -377,6 +449,8 @@ class MlCommand(google.protobuf.message.Message):
evaluate: global___MlCommand.Evaluate | None = ...,
clean_cache: global___MlCommand.CleanCache | None = ...,
get_cache_info: global___MlCommand.GetCacheInfo | None = ...,
+ create_summary: global___MlCommand.CreateSummary | None = ...,
+ get_model_size: global___MlCommand.GetModelSize | None = ...,
) -> None: ...
def HasField(
self,
@@ -385,6 +459,8 @@ class MlCommand(google.protobuf.message.Message):
b"clean_cache",
"command",
b"command",
+ "create_summary",
+ b"create_summary",
"delete",
b"delete",
"evaluate",
@@ -395,6 +471,8 @@ class MlCommand(google.protobuf.message.Message):
b"fit",
"get_cache_info",
b"get_cache_info",
+ "get_model_size",
+ b"get_model_size",
"read",
b"read",
"write",
@@ -408,6 +486,8 @@ class MlCommand(google.protobuf.message.Message):
b"clean_cache",
"command",
b"command",
+ "create_summary",
+ b"create_summary",
"delete",
b"delete",
"evaluate",
@@ -418,6 +498,8 @@ class MlCommand(google.protobuf.message.Message):
b"fit",
"get_cache_info",
b"get_cache_info",
+ "get_model_size",
+ b"get_model_size",
"read",
b"read",
"write",
@@ -428,7 +510,16 @@ class MlCommand(google.protobuf.message.Message):
self, oneof_group: typing_extensions.Literal["command", b"command"]
) -> (
typing_extensions.Literal[
- "fit", "fetch", "delete", "write", "read", "evaluate", "clean_cache", "get_cache_info"
+ "fit",
+ "fetch",
+ "delete",
+ "write",
+ "read",
+ "evaluate",
+ "clean_cache",
+ "get_cache_info",
+ "create_summary",
+ "get_model_size",
]
| None
): ...
diff --git a/python/pyspark/sql/connect/proto/pipelines_pb2.py b/python/pyspark/sql/connect/proto/pipelines_pb2.py
index 017e7285f8848..08b39a39e831a 100644
--- a/python/pyspark/sql/connect/proto/pipelines_pb2.py
+++ b/python/pyspark/sql/connect/proto/pipelines_pb2.py
@@ -40,7 +40,7 @@
DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(
- b'\n\x1dspark/connect/pipelines.proto\x12\rspark.connect\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"\xe6\x12\n\x0fPipelineCommand\x12h\n\x15\x63reate_dataflow_graph\x18\x01 \x01(\x0b\x32\x32.spark.connect.PipelineCommand.CreateDataflowGraphH\x00R\x13\x63reateDataflowGraph\x12U\n\x0e\x64\x65\x66ine_dataset\x18\x02 \x01(\x0b\x32,.spark.connect.PipelineCommand.DefineDatasetH\x00R\rdefineDataset\x12L\n\x0b\x64\x65\x66ine_flow\x18\x03 \x01(\x0b\x32).spark.connect.PipelineCommand.DefineFlowH\x00R\ndefineFlow\x12\x62\n\x13\x64rop_dataflow_graph\x18\x04 \x01(\x0b\x32\x30.spark.connect.PipelineCommand.DropDataflowGraphH\x00R\x11\x64ropDataflowGraph\x12\x46\n\tstart_run\x18\x05 \x01(\x0b\x32\'.spark.connect.PipelineCommand.StartRunH\x00R\x08startRun\x12r\n\x19\x64\x65\x66ine_sql_graph_elements\x18\x06 \x01(\x0b\x32\x35.spark.connect.PipelineCommand.DefineSqlGraphElementsH\x00R\x16\x64\x65\x66ineSqlGraphElements\x1a\x87\x03\n\x13\x43reateDataflowGraph\x12,\n\x0f\x64\x65\x66\x61ult_catalog\x18\x01 \x01(\tH\x00R\x0e\x64\x65\x66\x61ultCatalog\x88\x01\x01\x12.\n\x10\x64\x65\x66\x61ult_database\x18\x02 \x01(\tH\x01R\x0f\x64\x65\x66\x61ultDatabase\x88\x01\x01\x12Z\n\x08sql_conf\x18\x05 \x03(\x0b\x32?.spark.connect.PipelineCommand.CreateDataflowGraph.SqlConfEntryR\x07sqlConf\x1a:\n\x0cSqlConfEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1aQ\n\x08Response\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x42\x14\n\x12_dataflow_graph_idB\x12\n\x10_default_catalogB\x13\n\x11_default_database\x1aZ\n\x11\x44ropDataflowGraph\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x42\x14\n\x12_dataflow_graph_id\x1a\xd1\x04\n\rDefineDataset\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12&\n\x0c\x64\x61taset_name\x18\x02 \x01(\tH\x01R\x0b\x64\x61tasetName\x88\x01\x01\x12\x42\n\x0c\x64\x61taset_type\x18\x03 \x01(\x0e\x32\x1a.spark.connect.DatasetTypeH\x02R\x0b\x64\x61tasetType\x88\x01\x01\x12\x1d\n\x07\x63omment\x18\x04 \x01(\tH\x03R\x07\x63omment\x88\x01\x01\x12l\n\x10table_properties\x18\x05 \x03(\x0b\x32\x41.spark.connect.PipelineCommand.DefineDataset.TablePropertiesEntryR\x0ftableProperties\x12%\n\x0epartition_cols\x18\x06 \x03(\tR\rpartitionCols\x12\x34\n\x06schema\x18\x07 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x04R\x06schema\x88\x01\x01\x12\x1b\n\x06\x66ormat\x18\x08 \x01(\tH\x05R\x06\x66ormat\x88\x01\x01\x1a\x42\n\x14TablePropertiesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x14\n\x12_dataflow_graph_idB\x0f\n\r_dataset_nameB\x0f\n\r_dataset_typeB\n\n\x08_commentB\t\n\x07_schemaB\t\n\x07_format\x1a\xbc\x03\n\nDefineFlow\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12 \n\tflow_name\x18\x02 \x01(\tH\x01R\x08\x66lowName\x88\x01\x01\x12\x33\n\x13target_dataset_name\x18\x03 \x01(\tH\x02R\x11targetDatasetName\x88\x01\x01\x12\x30\n\x04plan\x18\x04 \x01(\x0b\x32\x17.spark.connect.RelationH\x03R\x04plan\x88\x01\x01\x12Q\n\x08sql_conf\x18\x05 \x03(\x0b\x32\x36.spark.connect.PipelineCommand.DefineFlow.SqlConfEntryR\x07sqlConf\x12\x17\n\x04once\x18\x06 \x01(\x08H\x04R\x04once\x88\x01\x01\x1a:\n\x0cSqlConfEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x14\n\x12_dataflow_graph_idB\x0c\n\n_flow_nameB\x16\n\x14_target_dataset_nameB\x07\n\x05_planB\x07\n\x05_once\x1aQ\n\x08StartRun\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x42\x14\n\x12_dataflow_graph_id\x1a\xc7\x01\n\x16\x44\x65\x66ineSqlGraphElements\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12\'\n\rsql_file_path\x18\x02 \x01(\tH\x01R\x0bsqlFilePath\x88\x01\x01\x12\x1e\n\x08sql_text\x18\x03 \x01(\tH\x02R\x07sqlText\x88\x01\x01\x42\x14\n\x12_dataflow_graph_idB\x10\n\x0e_sql_file_pathB\x0b\n\t_sql_textB\x0e\n\x0c\x63ommand_type"\x8e\x02\n\x15PipelineCommandResult\x12\x81\x01\n\x1c\x63reate_dataflow_graph_result\x18\x01 \x01(\x0b\x32>.spark.connect.PipelineCommandResult.CreateDataflowGraphResultH\x00R\x19\x63reateDataflowGraphResult\x1a\x62\n\x19\x43reateDataflowGraphResult\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x42\x14\n\x12_dataflow_graph_idB\r\n\x0bresult_type"I\n\x13PipelineEventResult\x12\x32\n\x05\x65vent\x18\x01 \x01(\x0b\x32\x1c.spark.connect.PipelineEventR\x05\x65vent"t\n\rPipelineEvent\x12\x38\n\ttimestamp\x18\x01 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\ttimestamp\x12\x1d\n\x07message\x18\x02 \x01(\tH\x00R\x07message\x88\x01\x01\x42\n\n\x08_message*a\n\x0b\x44\x61tasetType\x12\x1c\n\x18\x44\x41TASET_TYPE_UNSPECIFIED\x10\x00\x12\x15\n\x11MATERIALIZED_VIEW\x10\x01\x12\t\n\x05TABLE\x10\x02\x12\x12\n\x0eTEMPORARY_VIEW\x10\x03\x42"\n\x1eorg.apache.spark.connect.protoP\x01\x62\x06proto3'
+ b'\n\x1dspark/connect/pipelines.proto\x12\rspark.connect\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"\x97\x14\n\x0fPipelineCommand\x12h\n\x15\x63reate_dataflow_graph\x18\x01 \x01(\x0b\x32\x32.spark.connect.PipelineCommand.CreateDataflowGraphH\x00R\x13\x63reateDataflowGraph\x12U\n\x0e\x64\x65\x66ine_dataset\x18\x02 \x01(\x0b\x32,.spark.connect.PipelineCommand.DefineDatasetH\x00R\rdefineDataset\x12L\n\x0b\x64\x65\x66ine_flow\x18\x03 \x01(\x0b\x32).spark.connect.PipelineCommand.DefineFlowH\x00R\ndefineFlow\x12\x62\n\x13\x64rop_dataflow_graph\x18\x04 \x01(\x0b\x32\x30.spark.connect.PipelineCommand.DropDataflowGraphH\x00R\x11\x64ropDataflowGraph\x12\x46\n\tstart_run\x18\x05 \x01(\x0b\x32\'.spark.connect.PipelineCommand.StartRunH\x00R\x08startRun\x12r\n\x19\x64\x65\x66ine_sql_graph_elements\x18\x06 \x01(\x0b\x32\x35.spark.connect.PipelineCommand.DefineSqlGraphElementsH\x00R\x16\x64\x65\x66ineSqlGraphElements\x1a\x87\x03\n\x13\x43reateDataflowGraph\x12,\n\x0f\x64\x65\x66\x61ult_catalog\x18\x01 \x01(\tH\x00R\x0e\x64\x65\x66\x61ultCatalog\x88\x01\x01\x12.\n\x10\x64\x65\x66\x61ult_database\x18\x02 \x01(\tH\x01R\x0f\x64\x65\x66\x61ultDatabase\x88\x01\x01\x12Z\n\x08sql_conf\x18\x05 \x03(\x0b\x32?.spark.connect.PipelineCommand.CreateDataflowGraph.SqlConfEntryR\x07sqlConf\x1a:\n\x0cSqlConfEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1aQ\n\x08Response\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x42\x14\n\x12_dataflow_graph_idB\x12\n\x10_default_catalogB\x13\n\x11_default_database\x1aZ\n\x11\x44ropDataflowGraph\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x42\x14\n\x12_dataflow_graph_id\x1a\xd1\x04\n\rDefineDataset\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12&\n\x0c\x64\x61taset_name\x18\x02 \x01(\tH\x01R\x0b\x64\x61tasetName\x88\x01\x01\x12\x42\n\x0c\x64\x61taset_type\x18\x03 \x01(\x0e\x32\x1a.spark.connect.DatasetTypeH\x02R\x0b\x64\x61tasetType\x88\x01\x01\x12\x1d\n\x07\x63omment\x18\x04 \x01(\tH\x03R\x07\x63omment\x88\x01\x01\x12l\n\x10table_properties\x18\x05 \x03(\x0b\x32\x41.spark.connect.PipelineCommand.DefineDataset.TablePropertiesEntryR\x0ftableProperties\x12%\n\x0epartition_cols\x18\x06 \x03(\tR\rpartitionCols\x12\x34\n\x06schema\x18\x07 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x04R\x06schema\x88\x01\x01\x12\x1b\n\x06\x66ormat\x18\x08 \x01(\tH\x05R\x06\x66ormat\x88\x01\x01\x1a\x42\n\x14TablePropertiesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x14\n\x12_dataflow_graph_idB\x0f\n\r_dataset_nameB\x0f\n\r_dataset_typeB\n\n\x08_commentB\t\n\x07_schemaB\t\n\x07_format\x1a\xa6\x03\n\nDefineFlow\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12 \n\tflow_name\x18\x02 \x01(\tH\x01R\x08\x66lowName\x88\x01\x01\x12\x33\n\x13target_dataset_name\x18\x03 \x01(\tH\x02R\x11targetDatasetName\x88\x01\x01\x12\x38\n\x08relation\x18\x04 \x01(\x0b\x32\x17.spark.connect.RelationH\x03R\x08relation\x88\x01\x01\x12Q\n\x08sql_conf\x18\x05 \x03(\x0b\x32\x36.spark.connect.PipelineCommand.DefineFlow.SqlConfEntryR\x07sqlConf\x1a:\n\x0cSqlConfEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x14\n\x12_dataflow_graph_idB\x0c\n\n_flow_nameB\x16\n\x14_target_dataset_nameB\x0b\n\t_relation\x1a\x97\x02\n\x08StartRun\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12\x34\n\x16\x66ull_refresh_selection\x18\x02 \x03(\tR\x14\x66ullRefreshSelection\x12-\n\x10\x66ull_refresh_all\x18\x03 \x01(\x08H\x01R\x0e\x66ullRefreshAll\x88\x01\x01\x12+\n\x11refresh_selection\x18\x04 \x03(\tR\x10refreshSelection\x12\x15\n\x03\x64ry\x18\x05 \x01(\x08H\x02R\x03\x64ry\x88\x01\x01\x42\x14\n\x12_dataflow_graph_idB\x13\n\x11_full_refresh_allB\x06\n\x04_dry\x1a\xc7\x01\n\x16\x44\x65\x66ineSqlGraphElements\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x12\'\n\rsql_file_path\x18\x02 \x01(\tH\x01R\x0bsqlFilePath\x88\x01\x01\x12\x1e\n\x08sql_text\x18\x03 \x01(\tH\x02R\x07sqlText\x88\x01\x01\x42\x14\n\x12_dataflow_graph_idB\x10\n\x0e_sql_file_pathB\x0b\n\t_sql_textB\x0e\n\x0c\x63ommand_type"\x8e\x02\n\x15PipelineCommandResult\x12\x81\x01\n\x1c\x63reate_dataflow_graph_result\x18\x01 \x01(\x0b\x32>.spark.connect.PipelineCommandResult.CreateDataflowGraphResultH\x00R\x19\x63reateDataflowGraphResult\x1a\x62\n\x19\x43reateDataflowGraphResult\x12/\n\x11\x64\x61taflow_graph_id\x18\x01 \x01(\tH\x00R\x0f\x64\x61taflowGraphId\x88\x01\x01\x42\x14\n\x12_dataflow_graph_idB\r\n\x0bresult_type"I\n\x13PipelineEventResult\x12\x32\n\x05\x65vent\x18\x01 \x01(\x0b\x32\x1c.spark.connect.PipelineEventR\x05\x65vent"t\n\rPipelineEvent\x12\x38\n\ttimestamp\x18\x01 \x01(\x0b\x32\x1a.google.protobuf.TimestampR\ttimestamp\x12\x1d\n\x07message\x18\x02 \x01(\tH\x00R\x07message\x88\x01\x01\x42\n\n\x08_message*a\n\x0b\x44\x61tasetType\x12\x1c\n\x18\x44\x41TASET_TYPE_UNSPECIFIED\x10\x00\x12\x15\n\x11MATERIALIZED_VIEW\x10\x01\x12\t\n\x05TABLE\x10\x02\x12\x12\n\x0eTEMPORARY_VIEW\x10\x03\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3'
)
_globals = globals()
@@ -50,17 +50,19 @@
)
if not _descriptor._USE_C_DESCRIPTORS:
_globals["DESCRIPTOR"]._loaded_options = None
- _globals["DESCRIPTOR"]._serialized_options = b"\n\036org.apache.spark.connect.protoP\001"
+ _globals[
+ "DESCRIPTOR"
+ ]._serialized_options = b"\n\036org.apache.spark.connect.protoP\001Z\022internal/generated"
_globals["_PIPELINECOMMAND_CREATEDATAFLOWGRAPH_SQLCONFENTRY"]._loaded_options = None
_globals["_PIPELINECOMMAND_CREATEDATAFLOWGRAPH_SQLCONFENTRY"]._serialized_options = b"8\001"
_globals["_PIPELINECOMMAND_DEFINEDATASET_TABLEPROPERTIESENTRY"]._loaded_options = None
_globals["_PIPELINECOMMAND_DEFINEDATASET_TABLEPROPERTIESENTRY"]._serialized_options = b"8\001"
_globals["_PIPELINECOMMAND_DEFINEFLOW_SQLCONFENTRY"]._loaded_options = None
_globals["_PIPELINECOMMAND_DEFINEFLOW_SQLCONFENTRY"]._serialized_options = b"8\001"
- _globals["_DATASETTYPE"]._serialized_start = 3014
- _globals["_DATASETTYPE"]._serialized_end = 3111
+ _globals["_DATASETTYPE"]._serialized_start = 3191
+ _globals["_DATASETTYPE"]._serialized_end = 3288
_globals["_PIPELINECOMMAND"]._serialized_start = 140
- _globals["_PIPELINECOMMAND"]._serialized_end = 2546
+ _globals["_PIPELINECOMMAND"]._serialized_end = 2723
_globals["_PIPELINECOMMAND_CREATEDATAFLOWGRAPH"]._serialized_start = 719
_globals["_PIPELINECOMMAND_CREATEDATAFLOWGRAPH"]._serialized_end = 1110
_globals["_PIPELINECOMMAND_CREATEDATAFLOWGRAPH_SQLCONFENTRY"]._serialized_start = 928
@@ -74,19 +76,19 @@
_globals["_PIPELINECOMMAND_DEFINEDATASET_TABLEPROPERTIESENTRY"]._serialized_start = 1642
_globals["_PIPELINECOMMAND_DEFINEDATASET_TABLEPROPERTIESENTRY"]._serialized_end = 1708
_globals["_PIPELINECOMMAND_DEFINEFLOW"]._serialized_start = 1801
- _globals["_PIPELINECOMMAND_DEFINEFLOW"]._serialized_end = 2245
+ _globals["_PIPELINECOMMAND_DEFINEFLOW"]._serialized_end = 2223
_globals["_PIPELINECOMMAND_DEFINEFLOW_SQLCONFENTRY"]._serialized_start = 928
_globals["_PIPELINECOMMAND_DEFINEFLOW_SQLCONFENTRY"]._serialized_end = 986
- _globals["_PIPELINECOMMAND_STARTRUN"]._serialized_start = 2247
- _globals["_PIPELINECOMMAND_STARTRUN"]._serialized_end = 2328
- _globals["_PIPELINECOMMAND_DEFINESQLGRAPHELEMENTS"]._serialized_start = 2331
- _globals["_PIPELINECOMMAND_DEFINESQLGRAPHELEMENTS"]._serialized_end = 2530
- _globals["_PIPELINECOMMANDRESULT"]._serialized_start = 2549
- _globals["_PIPELINECOMMANDRESULT"]._serialized_end = 2819
- _globals["_PIPELINECOMMANDRESULT_CREATEDATAFLOWGRAPHRESULT"]._serialized_start = 2706
- _globals["_PIPELINECOMMANDRESULT_CREATEDATAFLOWGRAPHRESULT"]._serialized_end = 2804
- _globals["_PIPELINEEVENTRESULT"]._serialized_start = 2821
- _globals["_PIPELINEEVENTRESULT"]._serialized_end = 2894
- _globals["_PIPELINEEVENT"]._serialized_start = 2896
- _globals["_PIPELINEEVENT"]._serialized_end = 3012
+ _globals["_PIPELINECOMMAND_STARTRUN"]._serialized_start = 2226
+ _globals["_PIPELINECOMMAND_STARTRUN"]._serialized_end = 2505
+ _globals["_PIPELINECOMMAND_DEFINESQLGRAPHELEMENTS"]._serialized_start = 2508
+ _globals["_PIPELINECOMMAND_DEFINESQLGRAPHELEMENTS"]._serialized_end = 2707
+ _globals["_PIPELINECOMMANDRESULT"]._serialized_start = 2726
+ _globals["_PIPELINECOMMANDRESULT"]._serialized_end = 2996
+ _globals["_PIPELINECOMMANDRESULT_CREATEDATAFLOWGRAPHRESULT"]._serialized_start = 2883
+ _globals["_PIPELINECOMMANDRESULT_CREATEDATAFLOWGRAPHRESULT"]._serialized_end = 2981
+ _globals["_PIPELINEEVENTRESULT"]._serialized_start = 2998
+ _globals["_PIPELINEEVENTRESULT"]._serialized_end = 3071
+ _globals["_PIPELINEEVENT"]._serialized_start = 3073
+ _globals["_PIPELINEEVENT"]._serialized_end = 3189
# @@protoc_insertion_point(module_scope)
diff --git a/python/pyspark/sql/connect/proto/pipelines_pb2.pyi b/python/pyspark/sql/connect/proto/pipelines_pb2.pyi
index cf2cb8d3053b7..6287aabafc6b2 100644
--- a/python/pyspark/sql/connect/proto/pipelines_pb2.pyi
+++ b/python/pyspark/sql/connect/proto/pipelines_pb2.pyi
@@ -418,9 +418,8 @@ class PipelineCommand(google.protobuf.message.Message):
DATAFLOW_GRAPH_ID_FIELD_NUMBER: builtins.int
FLOW_NAME_FIELD_NUMBER: builtins.int
TARGET_DATASET_NAME_FIELD_NUMBER: builtins.int
- PLAN_FIELD_NUMBER: builtins.int
+ RELATION_FIELD_NUMBER: builtins.int
SQL_CONF_FIELD_NUMBER: builtins.int
- ONCE_FIELD_NUMBER: builtins.int
dataflow_graph_id: builtins.str
"""The graph to attach this flow to."""
flow_name: builtins.str
@@ -428,24 +427,21 @@ class PipelineCommand(google.protobuf.message.Message):
target_dataset_name: builtins.str
"""Name of the dataset this flow writes to. Can be partially or fully qualified."""
@property
- def plan(self) -> pyspark.sql.connect.proto.relations_pb2.Relation:
+ def relation(self) -> pyspark.sql.connect.proto.relations_pb2.Relation:
"""An unresolved relation that defines the dataset's flow."""
@property
def sql_conf(
self,
) -> google.protobuf.internal.containers.ScalarMap[builtins.str, builtins.str]:
"""SQL configurations set when running this flow."""
- once: builtins.bool
- """If true, this flow will only be run once per full refresh."""
def __init__(
self,
*,
dataflow_graph_id: builtins.str | None = ...,
flow_name: builtins.str | None = ...,
target_dataset_name: builtins.str | None = ...,
- plan: pyspark.sql.connect.proto.relations_pb2.Relation | None = ...,
+ relation: pyspark.sql.connect.proto.relations_pb2.Relation | None = ...,
sql_conf: collections.abc.Mapping[builtins.str, builtins.str] | None = ...,
- once: builtins.bool | None = ...,
) -> None: ...
def HasField(
self,
@@ -454,20 +450,16 @@ class PipelineCommand(google.protobuf.message.Message):
b"_dataflow_graph_id",
"_flow_name",
b"_flow_name",
- "_once",
- b"_once",
- "_plan",
- b"_plan",
+ "_relation",
+ b"_relation",
"_target_dataset_name",
b"_target_dataset_name",
"dataflow_graph_id",
b"dataflow_graph_id",
"flow_name",
b"flow_name",
- "once",
- b"once",
- "plan",
- b"plan",
+ "relation",
+ b"relation",
"target_dataset_name",
b"target_dataset_name",
],
@@ -479,20 +471,16 @@ class PipelineCommand(google.protobuf.message.Message):
b"_dataflow_graph_id",
"_flow_name",
b"_flow_name",
- "_once",
- b"_once",
- "_plan",
- b"_plan",
+ "_relation",
+ b"_relation",
"_target_dataset_name",
b"_target_dataset_name",
"dataflow_graph_id",
b"dataflow_graph_id",
"flow_name",
b"flow_name",
- "once",
- b"once",
- "plan",
- b"plan",
+ "relation",
+ b"relation",
"sql_conf",
b"sql_conf",
"target_dataset_name",
@@ -510,12 +498,8 @@ class PipelineCommand(google.protobuf.message.Message):
) -> typing_extensions.Literal["flow_name"] | None: ...
@typing.overload
def WhichOneof(
- self, oneof_group: typing_extensions.Literal["_once", b"_once"]
- ) -> typing_extensions.Literal["once"] | None: ...
- @typing.overload
- def WhichOneof(
- self, oneof_group: typing_extensions.Literal["_plan", b"_plan"]
- ) -> typing_extensions.Literal["plan"] | None: ...
+ self, oneof_group: typing_extensions.Literal["_relation", b"_relation"]
+ ) -> typing_extensions.Literal["relation"] | None: ...
@typing.overload
def WhichOneof(
self,
@@ -530,20 +514,52 @@ class PipelineCommand(google.protobuf.message.Message):
DESCRIPTOR: google.protobuf.descriptor.Descriptor
DATAFLOW_GRAPH_ID_FIELD_NUMBER: builtins.int
+ FULL_REFRESH_SELECTION_FIELD_NUMBER: builtins.int
+ FULL_REFRESH_ALL_FIELD_NUMBER: builtins.int
+ REFRESH_SELECTION_FIELD_NUMBER: builtins.int
+ DRY_FIELD_NUMBER: builtins.int
dataflow_graph_id: builtins.str
"""The graph to start."""
+ @property
+ def full_refresh_selection(
+ self,
+ ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.str]:
+ """List of dataset to reset and recompute."""
+ full_refresh_all: builtins.bool
+ """Perform a full graph reset and recompute."""
+ @property
+ def refresh_selection(
+ self,
+ ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.str]:
+ """List of dataset to update."""
+ dry: builtins.bool
+ """If true, the run will not actually execute any flows, but will only validate the graph and
+ check for any errors. This is useful for testing and validation purposes.
+ """
def __init__(
self,
*,
dataflow_graph_id: builtins.str | None = ...,
+ full_refresh_selection: collections.abc.Iterable[builtins.str] | None = ...,
+ full_refresh_all: builtins.bool | None = ...,
+ refresh_selection: collections.abc.Iterable[builtins.str] | None = ...,
+ dry: builtins.bool | None = ...,
) -> None: ...
def HasField(
self,
field_name: typing_extensions.Literal[
"_dataflow_graph_id",
b"_dataflow_graph_id",
+ "_dry",
+ b"_dry",
+ "_full_refresh_all",
+ b"_full_refresh_all",
"dataflow_graph_id",
b"dataflow_graph_id",
+ "dry",
+ b"dry",
+ "full_refresh_all",
+ b"full_refresh_all",
],
) -> builtins.bool: ...
def ClearField(
@@ -551,14 +567,35 @@ class PipelineCommand(google.protobuf.message.Message):
field_name: typing_extensions.Literal[
"_dataflow_graph_id",
b"_dataflow_graph_id",
+ "_dry",
+ b"_dry",
+ "_full_refresh_all",
+ b"_full_refresh_all",
"dataflow_graph_id",
b"dataflow_graph_id",
+ "dry",
+ b"dry",
+ "full_refresh_all",
+ b"full_refresh_all",
+ "full_refresh_selection",
+ b"full_refresh_selection",
+ "refresh_selection",
+ b"refresh_selection",
],
) -> None: ...
+ @typing.overload
def WhichOneof(
self,
oneof_group: typing_extensions.Literal["_dataflow_graph_id", b"_dataflow_graph_id"],
) -> typing_extensions.Literal["dataflow_graph_id"] | None: ...
+ @typing.overload
+ def WhichOneof(
+ self, oneof_group: typing_extensions.Literal["_dry", b"_dry"]
+ ) -> typing_extensions.Literal["dry"] | None: ...
+ @typing.overload
+ def WhichOneof(
+ self, oneof_group: typing_extensions.Literal["_full_refresh_all", b"_full_refresh_all"]
+ ) -> typing_extensions.Literal["full_refresh_all"] | None: ...
class DefineSqlGraphElements(google.protobuf.message.Message):
"""Parses the SQL file and registers all datasets and flows."""
diff --git a/python/pyspark/sql/connect/proto/relations_pb2.py b/python/pyspark/sql/connect/proto/relations_pb2.py
index 525ba88ff67c6..3774bcbdbfb0e 100644
--- a/python/pyspark/sql/connect/proto/relations_pb2.py
+++ b/python/pyspark/sql/connect/proto/relations_pb2.py
@@ -43,7 +43,7 @@
DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(
- b'\n\x1dspark/connect/relations.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1fspark/connect/expressions.proto\x1a\x19spark/connect/types.proto\x1a\x1bspark/connect/catalog.proto\x1a\x1aspark/connect/common.proto\x1a\x1dspark/connect/ml_common.proto"\x9c\x1d\n\x08Relation\x12\x35\n\x06\x63ommon\x18\x01 \x01(\x0b\x32\x1d.spark.connect.RelationCommonR\x06\x63ommon\x12)\n\x04read\x18\x02 \x01(\x0b\x32\x13.spark.connect.ReadH\x00R\x04read\x12\x32\n\x07project\x18\x03 \x01(\x0b\x32\x16.spark.connect.ProjectH\x00R\x07project\x12/\n\x06\x66ilter\x18\x04 \x01(\x0b\x32\x15.spark.connect.FilterH\x00R\x06\x66ilter\x12)\n\x04join\x18\x05 \x01(\x0b\x32\x13.spark.connect.JoinH\x00R\x04join\x12\x34\n\x06set_op\x18\x06 \x01(\x0b\x32\x1b.spark.connect.SetOperationH\x00R\x05setOp\x12)\n\x04sort\x18\x07 \x01(\x0b\x32\x13.spark.connect.SortH\x00R\x04sort\x12,\n\x05limit\x18\x08 \x01(\x0b\x32\x14.spark.connect.LimitH\x00R\x05limit\x12\x38\n\taggregate\x18\t \x01(\x0b\x32\x18.spark.connect.AggregateH\x00R\taggregate\x12&\n\x03sql\x18\n \x01(\x0b\x32\x12.spark.connect.SQLH\x00R\x03sql\x12\x45\n\x0elocal_relation\x18\x0b \x01(\x0b\x32\x1c.spark.connect.LocalRelationH\x00R\rlocalRelation\x12/\n\x06sample\x18\x0c \x01(\x0b\x32\x15.spark.connect.SampleH\x00R\x06sample\x12/\n\x06offset\x18\r \x01(\x0b\x32\x15.spark.connect.OffsetH\x00R\x06offset\x12>\n\x0b\x64\x65\x64uplicate\x18\x0e \x01(\x0b\x32\x1a.spark.connect.DeduplicateH\x00R\x0b\x64\x65\x64uplicate\x12,\n\x05range\x18\x0f \x01(\x0b\x32\x14.spark.connect.RangeH\x00R\x05range\x12\x45\n\x0esubquery_alias\x18\x10 \x01(\x0b\x32\x1c.spark.connect.SubqueryAliasH\x00R\rsubqueryAlias\x12>\n\x0brepartition\x18\x11 \x01(\x0b\x32\x1a.spark.connect.RepartitionH\x00R\x0brepartition\x12*\n\x05to_df\x18\x12 \x01(\x0b\x32\x13.spark.connect.ToDFH\x00R\x04toDf\x12U\n\x14with_columns_renamed\x18\x13 \x01(\x0b\x32!.spark.connect.WithColumnsRenamedH\x00R\x12withColumnsRenamed\x12<\n\x0bshow_string\x18\x14 \x01(\x0b\x32\x19.spark.connect.ShowStringH\x00R\nshowString\x12)\n\x04\x64rop\x18\x15 \x01(\x0b\x32\x13.spark.connect.DropH\x00R\x04\x64rop\x12)\n\x04tail\x18\x16 \x01(\x0b\x32\x13.spark.connect.TailH\x00R\x04tail\x12?\n\x0cwith_columns\x18\x17 \x01(\x0b\x32\x1a.spark.connect.WithColumnsH\x00R\x0bwithColumns\x12)\n\x04hint\x18\x18 \x01(\x0b\x32\x13.spark.connect.HintH\x00R\x04hint\x12\x32\n\x07unpivot\x18\x19 \x01(\x0b\x32\x16.spark.connect.UnpivotH\x00R\x07unpivot\x12\x36\n\tto_schema\x18\x1a \x01(\x0b\x32\x17.spark.connect.ToSchemaH\x00R\x08toSchema\x12\x64\n\x19repartition_by_expression\x18\x1b \x01(\x0b\x32&.spark.connect.RepartitionByExpressionH\x00R\x17repartitionByExpression\x12\x45\n\x0emap_partitions\x18\x1c \x01(\x0b\x32\x1c.spark.connect.MapPartitionsH\x00R\rmapPartitions\x12H\n\x0f\x63ollect_metrics\x18\x1d \x01(\x0b\x32\x1d.spark.connect.CollectMetricsH\x00R\x0e\x63ollectMetrics\x12,\n\x05parse\x18\x1e \x01(\x0b\x32\x14.spark.connect.ParseH\x00R\x05parse\x12\x36\n\tgroup_map\x18\x1f \x01(\x0b\x32\x17.spark.connect.GroupMapH\x00R\x08groupMap\x12=\n\x0c\x63o_group_map\x18 \x01(\x0b\x32\x19.spark.connect.CoGroupMapH\x00R\ncoGroupMap\x12\x45\n\x0ewith_watermark\x18! \x01(\x0b\x32\x1c.spark.connect.WithWatermarkH\x00R\rwithWatermark\x12\x63\n\x1a\x61pply_in_pandas_with_state\x18" \x01(\x0b\x32%.spark.connect.ApplyInPandasWithStateH\x00R\x16\x61pplyInPandasWithState\x12<\n\x0bhtml_string\x18# \x01(\x0b\x32\x19.spark.connect.HtmlStringH\x00R\nhtmlString\x12X\n\x15\x63\x61\x63hed_local_relation\x18$ \x01(\x0b\x32".spark.connect.CachedLocalRelationH\x00R\x13\x63\x61\x63hedLocalRelation\x12[\n\x16\x63\x61\x63hed_remote_relation\x18% \x01(\x0b\x32#.spark.connect.CachedRemoteRelationH\x00R\x14\x63\x61\x63hedRemoteRelation\x12\x8e\x01\n)common_inline_user_defined_table_function\x18& \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R$commonInlineUserDefinedTableFunction\x12\x37\n\nas_of_join\x18\' \x01(\x0b\x32\x17.spark.connect.AsOfJoinH\x00R\x08\x61sOfJoin\x12\x85\x01\n&common_inline_user_defined_data_source\x18( \x01(\x0b\x32\x30.spark.connect.CommonInlineUserDefinedDataSourceH\x00R!commonInlineUserDefinedDataSource\x12\x45\n\x0ewith_relations\x18) \x01(\x0b\x32\x1c.spark.connect.WithRelationsH\x00R\rwithRelations\x12\x38\n\ttranspose\x18* \x01(\x0b\x32\x18.spark.connect.TransposeH\x00R\ttranspose\x12w\n unresolved_table_valued_function\x18+ \x01(\x0b\x32,.spark.connect.UnresolvedTableValuedFunctionH\x00R\x1dunresolvedTableValuedFunction\x12?\n\x0clateral_join\x18, \x01(\x0b\x32\x1a.spark.connect.LateralJoinH\x00R\x0blateralJoin\x12\x30\n\x07\x66ill_na\x18Z \x01(\x0b\x32\x15.spark.connect.NAFillH\x00R\x06\x66illNa\x12\x30\n\x07\x64rop_na\x18[ \x01(\x0b\x32\x15.spark.connect.NADropH\x00R\x06\x64ropNa\x12\x34\n\x07replace\x18\\ \x01(\x0b\x32\x18.spark.connect.NAReplaceH\x00R\x07replace\x12\x36\n\x07summary\x18\x64 \x01(\x0b\x32\x1a.spark.connect.StatSummaryH\x00R\x07summary\x12\x39\n\x08\x63rosstab\x18\x65 \x01(\x0b\x32\x1b.spark.connect.StatCrosstabH\x00R\x08\x63rosstab\x12\x39\n\x08\x64\x65scribe\x18\x66 \x01(\x0b\x32\x1b.spark.connect.StatDescribeH\x00R\x08\x64\x65scribe\x12*\n\x03\x63ov\x18g \x01(\x0b\x32\x16.spark.connect.StatCovH\x00R\x03\x63ov\x12-\n\x04\x63orr\x18h \x01(\x0b\x32\x17.spark.connect.StatCorrH\x00R\x04\x63orr\x12L\n\x0f\x61pprox_quantile\x18i \x01(\x0b\x32!.spark.connect.StatApproxQuantileH\x00R\x0e\x61pproxQuantile\x12=\n\nfreq_items\x18j \x01(\x0b\x32\x1c.spark.connect.StatFreqItemsH\x00R\tfreqItems\x12:\n\tsample_by\x18k \x01(\x0b\x32\x1b.spark.connect.StatSampleByH\x00R\x08sampleBy\x12\x33\n\x07\x63\x61talog\x18\xc8\x01 \x01(\x0b\x32\x16.spark.connect.CatalogH\x00R\x07\x63\x61talog\x12=\n\x0bml_relation\x18\xac\x02 \x01(\x0b\x32\x19.spark.connect.MlRelationH\x00R\nmlRelation\x12\x35\n\textension\x18\xe6\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x33\n\x07unknown\x18\xe7\x07 \x01(\x0b\x32\x16.spark.connect.UnknownH\x00R\x07unknownB\n\n\x08rel_type"\xf8\x02\n\nMlRelation\x12\x43\n\ttransform\x18\x01 \x01(\x0b\x32#.spark.connect.MlRelation.TransformH\x00R\ttransform\x12,\n\x05\x66\x65tch\x18\x02 \x01(\x0b\x32\x14.spark.connect.FetchH\x00R\x05\x66\x65tch\x1a\xeb\x01\n\tTransform\x12\x33\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefH\x00R\x06objRef\x12=\n\x0btransformer\x18\x02 \x01(\x0b\x32\x19.spark.connect.MlOperatorH\x00R\x0btransformer\x12-\n\x05input\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06params\x18\x04 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06paramsB\n\n\x08operatorB\t\n\x07ml_type"\xcb\x02\n\x05\x46\x65tch\x12\x31\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefR\x06objRef\x12\x35\n\x07methods\x18\x02 \x03(\x0b\x32\x1b.spark.connect.Fetch.MethodR\x07methods\x1a\xd7\x01\n\x06Method\x12\x16\n\x06method\x18\x01 \x01(\tR\x06method\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32 .spark.connect.Fetch.Method.ArgsR\x04\x61rgs\x1a\x7f\n\x04\x41rgs\x12\x39\n\x05param\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x05param\x12/\n\x05input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x05inputB\x0b\n\targs_type"\t\n\x07Unknown"\x8e\x01\n\x0eRelationCommon\x12#\n\x0bsource_info\x18\x01 \x01(\tB\x02\x18\x01R\nsourceInfo\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12-\n\x06origin\x18\x03 \x01(\x0b\x32\x15.spark.connect.OriginR\x06originB\n\n\x08_plan_id"\xde\x03\n\x03SQL\x12\x14\n\x05query\x18\x01 \x01(\tR\x05query\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1c.spark.connect.SQL.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12O\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32&.spark.connect.SQL.NamedArgumentsEntryR\x0enamedArguments\x12>\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cposArguments\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01\x1a\\\n\x13NamedArgumentsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value:\x02\x38\x01"u\n\rWithRelations\x12+\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04root\x12\x37\n\nreferences\x18\x02 \x03(\x0b\x32\x17.spark.connect.RelationR\nreferences"\x97\x05\n\x04Read\x12\x41\n\x0bnamed_table\x18\x01 \x01(\x0b\x32\x1e.spark.connect.Read.NamedTableH\x00R\nnamedTable\x12\x41\n\x0b\x64\x61ta_source\x18\x02 \x01(\x0b\x32\x1e.spark.connect.Read.DataSourceH\x00R\ndataSource\x12!\n\x0cis_streaming\x18\x03 \x01(\x08R\x0bisStreaming\x1a\xc0\x01\n\nNamedTable\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x45\n\x07options\x18\x02 \x03(\x0b\x32+.spark.connect.Read.NamedTable.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x95\x02\n\nDataSource\x12\x1b\n\x06\x66ormat\x18\x01 \x01(\tH\x00R\x06\x66ormat\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x12\x45\n\x07options\x18\x03 \x03(\x0b\x32+.spark.connect.Read.DataSource.OptionsEntryR\x07options\x12\x14\n\x05paths\x18\x04 \x03(\tR\x05paths\x12\x1e\n\npredicates\x18\x05 \x03(\tR\npredicates\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07_formatB\t\n\x07_schemaB\x0b\n\tread_type"u\n\x07Project\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12;\n\x0b\x65xpressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0b\x65xpressions"p\n\x06\x46ilter\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x37\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\tcondition"\x95\x05\n\x04Join\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinType\x12#\n\rusing_columns\x18\x05 \x03(\tR\x0cusingColumns\x12K\n\x0ejoin_data_type\x18\x06 \x01(\x0b\x32 .spark.connect.Join.JoinDataTypeH\x00R\x0cjoinDataType\x88\x01\x01\x1a\\\n\x0cJoinDataType\x12$\n\x0eis_left_struct\x18\x01 \x01(\x08R\x0cisLeftStruct\x12&\n\x0fis_right_struct\x18\x02 \x01(\x08R\risRightStruct"\xd0\x01\n\x08JoinType\x12\x19\n\x15JOIN_TYPE_UNSPECIFIED\x10\x00\x12\x13\n\x0fJOIN_TYPE_INNER\x10\x01\x12\x18\n\x14JOIN_TYPE_FULL_OUTER\x10\x02\x12\x18\n\x14JOIN_TYPE_LEFT_OUTER\x10\x03\x12\x19\n\x15JOIN_TYPE_RIGHT_OUTER\x10\x04\x12\x17\n\x13JOIN_TYPE_LEFT_ANTI\x10\x05\x12\x17\n\x13JOIN_TYPE_LEFT_SEMI\x10\x06\x12\x13\n\x0fJOIN_TYPE_CROSS\x10\x07\x42\x11\n\x0f_join_data_type"\xdf\x03\n\x0cSetOperation\x12\x36\n\nleft_input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\tleftInput\x12\x38\n\x0bright_input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\nrightInput\x12\x45\n\x0bset_op_type\x18\x03 \x01(\x0e\x32%.spark.connect.SetOperation.SetOpTypeR\tsetOpType\x12\x1a\n\x06is_all\x18\x04 \x01(\x08H\x00R\x05isAll\x88\x01\x01\x12\x1c\n\x07\x62y_name\x18\x05 \x01(\x08H\x01R\x06\x62yName\x88\x01\x01\x12\x37\n\x15\x61llow_missing_columns\x18\x06 \x01(\x08H\x02R\x13\x61llowMissingColumns\x88\x01\x01"r\n\tSetOpType\x12\x1b\n\x17SET_OP_TYPE_UNSPECIFIED\x10\x00\x12\x19\n\x15SET_OP_TYPE_INTERSECT\x10\x01\x12\x15\n\x11SET_OP_TYPE_UNION\x10\x02\x12\x16\n\x12SET_OP_TYPE_EXCEPT\x10\x03\x42\t\n\x07_is_allB\n\n\x08_by_nameB\x18\n\x16_allow_missing_columns"L\n\x05Limit\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"O\n\x06Offset\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06offset\x18\x02 \x01(\x05R\x06offset"K\n\x04Tail\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"\xfe\x05\n\tAggregate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x41\n\ngroup_type\x18\x02 \x01(\x0e\x32".spark.connect.Aggregate.GroupTypeR\tgroupType\x12L\n\x14grouping_expressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12N\n\x15\x61ggregate_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x14\x61ggregateExpressions\x12\x34\n\x05pivot\x18\x05 \x01(\x0b\x32\x1e.spark.connect.Aggregate.PivotR\x05pivot\x12J\n\rgrouping_sets\x18\x06 \x03(\x0b\x32%.spark.connect.Aggregate.GroupingSetsR\x0cgroupingSets\x1ao\n\x05Pivot\x12+\n\x03\x63ol\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1aL\n\x0cGroupingSets\x12<\n\x0cgrouping_set\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0bgroupingSet"\x9f\x01\n\tGroupType\x12\x1a\n\x16GROUP_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12GROUP_TYPE_GROUPBY\x10\x01\x12\x15\n\x11GROUP_TYPE_ROLLUP\x10\x02\x12\x13\n\x0fGROUP_TYPE_CUBE\x10\x03\x12\x14\n\x10GROUP_TYPE_PIVOT\x10\x04\x12\x1c\n\x18GROUP_TYPE_GROUPING_SETS\x10\x05"\xa0\x01\n\x04Sort\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x05order\x18\x02 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\x05order\x12 \n\tis_global\x18\x03 \x01(\x08H\x00R\x08isGlobal\x88\x01\x01\x42\x0c\n\n_is_global"\x8d\x01\n\x04\x44rop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x33\n\x07\x63olumns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07\x63olumns\x12!\n\x0c\x63olumn_names\x18\x03 \x03(\tR\x0b\x63olumnNames"\xf0\x01\n\x0b\x44\x65\x64uplicate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames\x12\x32\n\x13\x61ll_columns_as_keys\x18\x03 \x01(\x08H\x00R\x10\x61llColumnsAsKeys\x88\x01\x01\x12.\n\x10within_watermark\x18\x04 \x01(\x08H\x01R\x0fwithinWatermark\x88\x01\x01\x42\x16\n\x14_all_columns_as_keysB\x13\n\x11_within_watermark"Y\n\rLocalRelation\x12\x17\n\x04\x64\x61ta\x18\x01 \x01(\x0cH\x00R\x04\x64\x61ta\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x42\x07\n\x05_dataB\t\n\x07_schema"H\n\x13\x43\x61\x63hedLocalRelation\x12\x12\n\x04hash\x18\x03 \x01(\tR\x04hashJ\x04\x08\x01\x10\x02J\x04\x08\x02\x10\x03R\x06userIdR\tsessionId"7\n\x14\x43\x61\x63hedRemoteRelation\x12\x1f\n\x0brelation_id\x18\x01 \x01(\tR\nrelationId"\x91\x02\n\x06Sample\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1f\n\x0blower_bound\x18\x02 \x01(\x01R\nlowerBound\x12\x1f\n\x0bupper_bound\x18\x03 \x01(\x01R\nupperBound\x12.\n\x10with_replacement\x18\x04 \x01(\x08H\x00R\x0fwithReplacement\x88\x01\x01\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x01R\x04seed\x88\x01\x01\x12/\n\x13\x64\x65terministic_order\x18\x06 \x01(\x08R\x12\x64\x65terministicOrderB\x13\n\x11_with_replacementB\x07\n\x05_seed"\x91\x01\n\x05Range\x12\x19\n\x05start\x18\x01 \x01(\x03H\x00R\x05start\x88\x01\x01\x12\x10\n\x03\x65nd\x18\x02 \x01(\x03R\x03\x65nd\x12\x12\n\x04step\x18\x03 \x01(\x03R\x04step\x12*\n\x0enum_partitions\x18\x04 \x01(\x05H\x01R\rnumPartitions\x88\x01\x01\x42\x08\n\x06_startB\x11\n\x0f_num_partitions"r\n\rSubqueryAlias\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05\x61lias\x18\x02 \x01(\tR\x05\x61lias\x12\x1c\n\tqualifier\x18\x03 \x03(\tR\tqualifier"\x8e\x01\n\x0bRepartition\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12%\n\x0enum_partitions\x18\x02 \x01(\x05R\rnumPartitions\x12\x1d\n\x07shuffle\x18\x03 \x01(\x08H\x00R\x07shuffle\x88\x01\x01\x42\n\n\x08_shuffle"\x8e\x01\n\nShowString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate\x12\x1a\n\x08vertical\x18\x04 \x01(\x08R\x08vertical"r\n\nHtmlString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate"\\\n\x0bStatSummary\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1e\n\nstatistics\x18\x02 \x03(\tR\nstatistics"Q\n\x0cStatDescribe\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols"e\n\x0cStatCrosstab\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"`\n\x07StatCov\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"\x89\x01\n\x08StatCorr\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2\x12\x1b\n\x06method\x18\x04 \x01(\tH\x00R\x06method\x88\x01\x01\x42\t\n\x07_method"\xa4\x01\n\x12StatApproxQuantile\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12$\n\rprobabilities\x18\x03 \x03(\x01R\rprobabilities\x12%\n\x0erelative_error\x18\x04 \x01(\x01R\rrelativeError"}\n\rStatFreqItems\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x1d\n\x07support\x18\x03 \x01(\x01H\x00R\x07support\x88\x01\x01\x42\n\n\x08_support"\xb5\x02\n\x0cStatSampleBy\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03\x63ol\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x42\n\tfractions\x18\x03 \x03(\x0b\x32$.spark.connect.StatSampleBy.FractionR\tfractions\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x00R\x04seed\x88\x01\x01\x1a\x63\n\x08\x46raction\x12;\n\x07stratum\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x07stratum\x12\x1a\n\x08\x66raction\x18\x02 \x01(\x01R\x08\x66ractionB\x07\n\x05_seed"\x86\x01\n\x06NAFill\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x39\n\x06values\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values"\x86\x01\n\x06NADrop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\'\n\rmin_non_nulls\x18\x03 \x01(\x05H\x00R\x0bminNonNulls\x88\x01\x01\x42\x10\n\x0e_min_non_nulls"\xa8\x02\n\tNAReplace\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12H\n\x0creplacements\x18\x03 \x03(\x0b\x32$.spark.connect.NAReplace.ReplacementR\x0creplacements\x1a\x8d\x01\n\x0bReplacement\x12>\n\told_value\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08oldValue\x12>\n\tnew_value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08newValue"X\n\x04ToDF\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames"\xfe\x02\n\x12WithColumnsRenamed\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12i\n\x12rename_columns_map\x18\x02 \x03(\x0b\x32\x37.spark.connect.WithColumnsRenamed.RenameColumnsMapEntryB\x02\x18\x01R\x10renameColumnsMap\x12\x42\n\x07renames\x18\x03 \x03(\x0b\x32(.spark.connect.WithColumnsRenamed.RenameR\x07renames\x1a\x43\n\x15RenameColumnsMapEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x45\n\x06Rename\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12 \n\x0cnew_col_name\x18\x02 \x01(\tR\nnewColName"w\n\x0bWithColumns\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x07\x61liases\x18\x02 \x03(\x0b\x32\x1f.spark.connect.Expression.AliasR\x07\x61liases"\x86\x01\n\rWithWatermark\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\nevent_time\x18\x02 \x01(\tR\teventTime\x12\'\n\x0f\x64\x65lay_threshold\x18\x03 \x01(\tR\x0e\x64\x65layThreshold"\x84\x01\n\x04Hint\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x39\n\nparameters\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\nparameters"\xc7\x02\n\x07Unpivot\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03ids\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x03ids\x12:\n\x06values\x18\x03 \x01(\x0b\x32\x1d.spark.connect.Unpivot.ValuesH\x00R\x06values\x88\x01\x01\x12\x30\n\x14variable_column_name\x18\x04 \x01(\tR\x12variableColumnName\x12*\n\x11value_column_name\x18\x05 \x01(\tR\x0fvalueColumnName\x1a;\n\x06Values\x12\x31\n\x06values\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x06valuesB\t\n\x07_values"z\n\tTranspose\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12>\n\rindex_columns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cindexColumns"}\n\x1dUnresolvedTableValuedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"j\n\x08ToSchema\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema"\xcb\x01\n\x17RepartitionByExpression\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x0fpartition_exprs\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0epartitionExprs\x12*\n\x0enum_partitions\x18\x03 \x01(\x05H\x00R\rnumPartitions\x88\x01\x01\x42\x11\n\x0f_num_partitions"\xe8\x01\n\rMapPartitions\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x04\x66unc\x18\x02 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12"\n\nis_barrier\x18\x03 \x01(\x08H\x00R\tisBarrier\x88\x01\x01\x12"\n\nprofile_id\x18\x04 \x01(\x05H\x01R\tprofileId\x88\x01\x01\x42\r\n\x0b_is_barrierB\r\n\x0b_profile_id"\xd2\x06\n\x08GroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12J\n\x13sorting_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x12sortingExpressions\x12<\n\rinitial_input\x18\x05 \x01(\x0b\x32\x17.spark.connect.RelationR\x0cinitialInput\x12[\n\x1cinitial_grouping_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x1ainitialGroupingExpressions\x12;\n\x18is_map_groups_with_state\x18\x07 \x01(\x08H\x00R\x14isMapGroupsWithState\x88\x01\x01\x12$\n\x0boutput_mode\x18\x08 \x01(\tH\x01R\noutputMode\x88\x01\x01\x12&\n\x0ctimeout_conf\x18\t \x01(\tH\x02R\x0btimeoutConf\x88\x01\x01\x12?\n\x0cstate_schema\x18\n \x01(\x0b\x32\x17.spark.connect.DataTypeH\x03R\x0bstateSchema\x88\x01\x01\x12\x65\n\x19transform_with_state_info\x18\x0b \x01(\x0b\x32%.spark.connect.TransformWithStateInfoH\x04R\x16transformWithStateInfo\x88\x01\x01\x42\x1b\n\x19_is_map_groups_with_stateB\x0e\n\x0c_output_modeB\x0f\n\r_timeout_confB\x0f\n\r_state_schemaB\x1c\n\x1a_transform_with_state_info"\xdf\x01\n\x16TransformWithStateInfo\x12\x1b\n\ttime_mode\x18\x01 \x01(\tR\x08timeMode\x12\x38\n\x16\x65vent_time_column_name\x18\x02 \x01(\tH\x00R\x13\x65ventTimeColumnName\x88\x01\x01\x12\x41\n\routput_schema\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x01R\x0coutputSchema\x88\x01\x01\x42\x19\n\x17_event_time_column_nameB\x10\n\x0e_output_schema"\x8e\x04\n\nCoGroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12W\n\x1ainput_grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18inputGroupingExpressions\x12-\n\x05other\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05other\x12W\n\x1aother_grouping_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18otherGroupingExpressions\x12\x42\n\x04\x66unc\x18\x05 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12U\n\x19input_sorting_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17inputSortingExpressions\x12U\n\x19other_sorting_expressions\x18\x07 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17otherSortingExpressions"\xe5\x02\n\x16\x41pplyInPandasWithState\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12#\n\routput_schema\x18\x04 \x01(\tR\x0coutputSchema\x12!\n\x0cstate_schema\x18\x05 \x01(\tR\x0bstateSchema\x12\x1f\n\x0boutput_mode\x18\x06 \x01(\tR\noutputMode\x12!\n\x0ctimeout_conf\x18\x07 \x01(\tR\x0btimeoutConf"\xf4\x01\n$CommonInlineUserDefinedTableFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12<\n\x0bpython_udtf\x18\x04 \x01(\x0b\x32\x19.spark.connect.PythonUDTFH\x00R\npythonUdtfB\n\n\x08\x66unction"\xb1\x01\n\nPythonUDTF\x12=\n\x0breturn_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\nreturnType\x88\x01\x01\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVerB\x0e\n\x0c_return_type"\x97\x01\n!CommonInlineUserDefinedDataSource\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12O\n\x12python_data_source\x18\x02 \x01(\x0b\x32\x1f.spark.connect.PythonDataSourceH\x00R\x10pythonDataSourceB\r\n\x0b\x64\x61ta_source"K\n\x10PythonDataSource\x12\x18\n\x07\x63ommand\x18\x01 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x02 \x01(\tR\tpythonVer"\x88\x01\n\x0e\x43ollectMetrics\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x33\n\x07metrics\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07metrics"\x84\x03\n\x05Parse\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x38\n\x06\x66ormat\x18\x02 \x01(\x0e\x32 .spark.connect.Parse.ParseFormatR\x06\x66ormat\x12\x34\n\x06schema\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x06schema\x88\x01\x01\x12;\n\x07options\x18\x04 \x03(\x0b\x32!.spark.connect.Parse.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"X\n\x0bParseFormat\x12\x1c\n\x18PARSE_FORMAT_UNSPECIFIED\x10\x00\x12\x14\n\x10PARSE_FORMAT_CSV\x10\x01\x12\x15\n\x11PARSE_FORMAT_JSON\x10\x02\x42\t\n\x07_schema"\xdb\x03\n\x08\x41sOfJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12\x37\n\nleft_as_of\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08leftAsOf\x12\x39\n\x0bright_as_of\x18\x04 \x01(\x0b\x32\x19.spark.connect.ExpressionR\trightAsOf\x12\x36\n\tjoin_expr\x18\x05 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08joinExpr\x12#\n\rusing_columns\x18\x06 \x03(\tR\x0cusingColumns\x12\x1b\n\tjoin_type\x18\x07 \x01(\tR\x08joinType\x12\x37\n\ttolerance\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\ttolerance\x12.\n\x13\x61llow_exact_matches\x18\t \x01(\x08R\x11\x61llowExactMatches\x12\x1c\n\tdirection\x18\n \x01(\tR\tdirection"\xe6\x01\n\x0bLateralJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinTypeB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3'
+ b'\n\x1dspark/connect/relations.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1fspark/connect/expressions.proto\x1a\x19spark/connect/types.proto\x1a\x1bspark/connect/catalog.proto\x1a\x1aspark/connect/common.proto\x1a\x1dspark/connect/ml_common.proto"\x9c\x1d\n\x08Relation\x12\x35\n\x06\x63ommon\x18\x01 \x01(\x0b\x32\x1d.spark.connect.RelationCommonR\x06\x63ommon\x12)\n\x04read\x18\x02 \x01(\x0b\x32\x13.spark.connect.ReadH\x00R\x04read\x12\x32\n\x07project\x18\x03 \x01(\x0b\x32\x16.spark.connect.ProjectH\x00R\x07project\x12/\n\x06\x66ilter\x18\x04 \x01(\x0b\x32\x15.spark.connect.FilterH\x00R\x06\x66ilter\x12)\n\x04join\x18\x05 \x01(\x0b\x32\x13.spark.connect.JoinH\x00R\x04join\x12\x34\n\x06set_op\x18\x06 \x01(\x0b\x32\x1b.spark.connect.SetOperationH\x00R\x05setOp\x12)\n\x04sort\x18\x07 \x01(\x0b\x32\x13.spark.connect.SortH\x00R\x04sort\x12,\n\x05limit\x18\x08 \x01(\x0b\x32\x14.spark.connect.LimitH\x00R\x05limit\x12\x38\n\taggregate\x18\t \x01(\x0b\x32\x18.spark.connect.AggregateH\x00R\taggregate\x12&\n\x03sql\x18\n \x01(\x0b\x32\x12.spark.connect.SQLH\x00R\x03sql\x12\x45\n\x0elocal_relation\x18\x0b \x01(\x0b\x32\x1c.spark.connect.LocalRelationH\x00R\rlocalRelation\x12/\n\x06sample\x18\x0c \x01(\x0b\x32\x15.spark.connect.SampleH\x00R\x06sample\x12/\n\x06offset\x18\r \x01(\x0b\x32\x15.spark.connect.OffsetH\x00R\x06offset\x12>\n\x0b\x64\x65\x64uplicate\x18\x0e \x01(\x0b\x32\x1a.spark.connect.DeduplicateH\x00R\x0b\x64\x65\x64uplicate\x12,\n\x05range\x18\x0f \x01(\x0b\x32\x14.spark.connect.RangeH\x00R\x05range\x12\x45\n\x0esubquery_alias\x18\x10 \x01(\x0b\x32\x1c.spark.connect.SubqueryAliasH\x00R\rsubqueryAlias\x12>\n\x0brepartition\x18\x11 \x01(\x0b\x32\x1a.spark.connect.RepartitionH\x00R\x0brepartition\x12*\n\x05to_df\x18\x12 \x01(\x0b\x32\x13.spark.connect.ToDFH\x00R\x04toDf\x12U\n\x14with_columns_renamed\x18\x13 \x01(\x0b\x32!.spark.connect.WithColumnsRenamedH\x00R\x12withColumnsRenamed\x12<\n\x0bshow_string\x18\x14 \x01(\x0b\x32\x19.spark.connect.ShowStringH\x00R\nshowString\x12)\n\x04\x64rop\x18\x15 \x01(\x0b\x32\x13.spark.connect.DropH\x00R\x04\x64rop\x12)\n\x04tail\x18\x16 \x01(\x0b\x32\x13.spark.connect.TailH\x00R\x04tail\x12?\n\x0cwith_columns\x18\x17 \x01(\x0b\x32\x1a.spark.connect.WithColumnsH\x00R\x0bwithColumns\x12)\n\x04hint\x18\x18 \x01(\x0b\x32\x13.spark.connect.HintH\x00R\x04hint\x12\x32\n\x07unpivot\x18\x19 \x01(\x0b\x32\x16.spark.connect.UnpivotH\x00R\x07unpivot\x12\x36\n\tto_schema\x18\x1a \x01(\x0b\x32\x17.spark.connect.ToSchemaH\x00R\x08toSchema\x12\x64\n\x19repartition_by_expression\x18\x1b \x01(\x0b\x32&.spark.connect.RepartitionByExpressionH\x00R\x17repartitionByExpression\x12\x45\n\x0emap_partitions\x18\x1c \x01(\x0b\x32\x1c.spark.connect.MapPartitionsH\x00R\rmapPartitions\x12H\n\x0f\x63ollect_metrics\x18\x1d \x01(\x0b\x32\x1d.spark.connect.CollectMetricsH\x00R\x0e\x63ollectMetrics\x12,\n\x05parse\x18\x1e \x01(\x0b\x32\x14.spark.connect.ParseH\x00R\x05parse\x12\x36\n\tgroup_map\x18\x1f \x01(\x0b\x32\x17.spark.connect.GroupMapH\x00R\x08groupMap\x12=\n\x0c\x63o_group_map\x18 \x01(\x0b\x32\x19.spark.connect.CoGroupMapH\x00R\ncoGroupMap\x12\x45\n\x0ewith_watermark\x18! \x01(\x0b\x32\x1c.spark.connect.WithWatermarkH\x00R\rwithWatermark\x12\x63\n\x1a\x61pply_in_pandas_with_state\x18" \x01(\x0b\x32%.spark.connect.ApplyInPandasWithStateH\x00R\x16\x61pplyInPandasWithState\x12<\n\x0bhtml_string\x18# \x01(\x0b\x32\x19.spark.connect.HtmlStringH\x00R\nhtmlString\x12X\n\x15\x63\x61\x63hed_local_relation\x18$ \x01(\x0b\x32".spark.connect.CachedLocalRelationH\x00R\x13\x63\x61\x63hedLocalRelation\x12[\n\x16\x63\x61\x63hed_remote_relation\x18% \x01(\x0b\x32#.spark.connect.CachedRemoteRelationH\x00R\x14\x63\x61\x63hedRemoteRelation\x12\x8e\x01\n)common_inline_user_defined_table_function\x18& \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R$commonInlineUserDefinedTableFunction\x12\x37\n\nas_of_join\x18\' \x01(\x0b\x32\x17.spark.connect.AsOfJoinH\x00R\x08\x61sOfJoin\x12\x85\x01\n&common_inline_user_defined_data_source\x18( \x01(\x0b\x32\x30.spark.connect.CommonInlineUserDefinedDataSourceH\x00R!commonInlineUserDefinedDataSource\x12\x45\n\x0ewith_relations\x18) \x01(\x0b\x32\x1c.spark.connect.WithRelationsH\x00R\rwithRelations\x12\x38\n\ttranspose\x18* \x01(\x0b\x32\x18.spark.connect.TransposeH\x00R\ttranspose\x12w\n unresolved_table_valued_function\x18+ \x01(\x0b\x32,.spark.connect.UnresolvedTableValuedFunctionH\x00R\x1dunresolvedTableValuedFunction\x12?\n\x0clateral_join\x18, \x01(\x0b\x32\x1a.spark.connect.LateralJoinH\x00R\x0blateralJoin\x12\x30\n\x07\x66ill_na\x18Z \x01(\x0b\x32\x15.spark.connect.NAFillH\x00R\x06\x66illNa\x12\x30\n\x07\x64rop_na\x18[ \x01(\x0b\x32\x15.spark.connect.NADropH\x00R\x06\x64ropNa\x12\x34\n\x07replace\x18\\ \x01(\x0b\x32\x18.spark.connect.NAReplaceH\x00R\x07replace\x12\x36\n\x07summary\x18\x64 \x01(\x0b\x32\x1a.spark.connect.StatSummaryH\x00R\x07summary\x12\x39\n\x08\x63rosstab\x18\x65 \x01(\x0b\x32\x1b.spark.connect.StatCrosstabH\x00R\x08\x63rosstab\x12\x39\n\x08\x64\x65scribe\x18\x66 \x01(\x0b\x32\x1b.spark.connect.StatDescribeH\x00R\x08\x64\x65scribe\x12*\n\x03\x63ov\x18g \x01(\x0b\x32\x16.spark.connect.StatCovH\x00R\x03\x63ov\x12-\n\x04\x63orr\x18h \x01(\x0b\x32\x17.spark.connect.StatCorrH\x00R\x04\x63orr\x12L\n\x0f\x61pprox_quantile\x18i \x01(\x0b\x32!.spark.connect.StatApproxQuantileH\x00R\x0e\x61pproxQuantile\x12=\n\nfreq_items\x18j \x01(\x0b\x32\x1c.spark.connect.StatFreqItemsH\x00R\tfreqItems\x12:\n\tsample_by\x18k \x01(\x0b\x32\x1b.spark.connect.StatSampleByH\x00R\x08sampleBy\x12\x33\n\x07\x63\x61talog\x18\xc8\x01 \x01(\x0b\x32\x16.spark.connect.CatalogH\x00R\x07\x63\x61talog\x12=\n\x0bml_relation\x18\xac\x02 \x01(\x0b\x32\x19.spark.connect.MlRelationH\x00R\nmlRelation\x12\x35\n\textension\x18\xe6\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x33\n\x07unknown\x18\xe7\x07 \x01(\x0b\x32\x16.spark.connect.UnknownH\x00R\x07unknownB\n\n\x08rel_type"\xe4\x03\n\nMlRelation\x12\x43\n\ttransform\x18\x01 \x01(\x0b\x32#.spark.connect.MlRelation.TransformH\x00R\ttransform\x12,\n\x05\x66\x65tch\x18\x02 \x01(\x0b\x32\x14.spark.connect.FetchH\x00R\x05\x66\x65tch\x12P\n\x15model_summary_dataset\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationH\x01R\x13modelSummaryDataset\x88\x01\x01\x1a\xeb\x01\n\tTransform\x12\x33\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefH\x00R\x06objRef\x12=\n\x0btransformer\x18\x02 \x01(\x0b\x32\x19.spark.connect.MlOperatorH\x00R\x0btransformer\x12-\n\x05input\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06params\x18\x04 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06paramsB\n\n\x08operatorB\t\n\x07ml_typeB\x18\n\x16_model_summary_dataset"\xcb\x02\n\x05\x46\x65tch\x12\x31\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefR\x06objRef\x12\x35\n\x07methods\x18\x02 \x03(\x0b\x32\x1b.spark.connect.Fetch.MethodR\x07methods\x1a\xd7\x01\n\x06Method\x12\x16\n\x06method\x18\x01 \x01(\tR\x06method\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32 .spark.connect.Fetch.Method.ArgsR\x04\x61rgs\x1a\x7f\n\x04\x41rgs\x12\x39\n\x05param\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x05param\x12/\n\x05input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x05inputB\x0b\n\targs_type"\t\n\x07Unknown"\x8e\x01\n\x0eRelationCommon\x12#\n\x0bsource_info\x18\x01 \x01(\tB\x02\x18\x01R\nsourceInfo\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12-\n\x06origin\x18\x03 \x01(\x0b\x32\x15.spark.connect.OriginR\x06originB\n\n\x08_plan_id"\xde\x03\n\x03SQL\x12\x14\n\x05query\x18\x01 \x01(\tR\x05query\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1c.spark.connect.SQL.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12O\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32&.spark.connect.SQL.NamedArgumentsEntryR\x0enamedArguments\x12>\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cposArguments\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01\x1a\\\n\x13NamedArgumentsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value:\x02\x38\x01"u\n\rWithRelations\x12+\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04root\x12\x37\n\nreferences\x18\x02 \x03(\x0b\x32\x17.spark.connect.RelationR\nreferences"\x97\x05\n\x04Read\x12\x41\n\x0bnamed_table\x18\x01 \x01(\x0b\x32\x1e.spark.connect.Read.NamedTableH\x00R\nnamedTable\x12\x41\n\x0b\x64\x61ta_source\x18\x02 \x01(\x0b\x32\x1e.spark.connect.Read.DataSourceH\x00R\ndataSource\x12!\n\x0cis_streaming\x18\x03 \x01(\x08R\x0bisStreaming\x1a\xc0\x01\n\nNamedTable\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x45\n\x07options\x18\x02 \x03(\x0b\x32+.spark.connect.Read.NamedTable.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x95\x02\n\nDataSource\x12\x1b\n\x06\x66ormat\x18\x01 \x01(\tH\x00R\x06\x66ormat\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x12\x45\n\x07options\x18\x03 \x03(\x0b\x32+.spark.connect.Read.DataSource.OptionsEntryR\x07options\x12\x14\n\x05paths\x18\x04 \x03(\tR\x05paths\x12\x1e\n\npredicates\x18\x05 \x03(\tR\npredicates\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07_formatB\t\n\x07_schemaB\x0b\n\tread_type"u\n\x07Project\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12;\n\x0b\x65xpressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0b\x65xpressions"p\n\x06\x46ilter\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x37\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\tcondition"\x95\x05\n\x04Join\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinType\x12#\n\rusing_columns\x18\x05 \x03(\tR\x0cusingColumns\x12K\n\x0ejoin_data_type\x18\x06 \x01(\x0b\x32 .spark.connect.Join.JoinDataTypeH\x00R\x0cjoinDataType\x88\x01\x01\x1a\\\n\x0cJoinDataType\x12$\n\x0eis_left_struct\x18\x01 \x01(\x08R\x0cisLeftStruct\x12&\n\x0fis_right_struct\x18\x02 \x01(\x08R\risRightStruct"\xd0\x01\n\x08JoinType\x12\x19\n\x15JOIN_TYPE_UNSPECIFIED\x10\x00\x12\x13\n\x0fJOIN_TYPE_INNER\x10\x01\x12\x18\n\x14JOIN_TYPE_FULL_OUTER\x10\x02\x12\x18\n\x14JOIN_TYPE_LEFT_OUTER\x10\x03\x12\x19\n\x15JOIN_TYPE_RIGHT_OUTER\x10\x04\x12\x17\n\x13JOIN_TYPE_LEFT_ANTI\x10\x05\x12\x17\n\x13JOIN_TYPE_LEFT_SEMI\x10\x06\x12\x13\n\x0fJOIN_TYPE_CROSS\x10\x07\x42\x11\n\x0f_join_data_type"\xdf\x03\n\x0cSetOperation\x12\x36\n\nleft_input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\tleftInput\x12\x38\n\x0bright_input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\nrightInput\x12\x45\n\x0bset_op_type\x18\x03 \x01(\x0e\x32%.spark.connect.SetOperation.SetOpTypeR\tsetOpType\x12\x1a\n\x06is_all\x18\x04 \x01(\x08H\x00R\x05isAll\x88\x01\x01\x12\x1c\n\x07\x62y_name\x18\x05 \x01(\x08H\x01R\x06\x62yName\x88\x01\x01\x12\x37\n\x15\x61llow_missing_columns\x18\x06 \x01(\x08H\x02R\x13\x61llowMissingColumns\x88\x01\x01"r\n\tSetOpType\x12\x1b\n\x17SET_OP_TYPE_UNSPECIFIED\x10\x00\x12\x19\n\x15SET_OP_TYPE_INTERSECT\x10\x01\x12\x15\n\x11SET_OP_TYPE_UNION\x10\x02\x12\x16\n\x12SET_OP_TYPE_EXCEPT\x10\x03\x42\t\n\x07_is_allB\n\n\x08_by_nameB\x18\n\x16_allow_missing_columns"L\n\x05Limit\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"O\n\x06Offset\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06offset\x18\x02 \x01(\x05R\x06offset"K\n\x04Tail\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"\xfe\x05\n\tAggregate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x41\n\ngroup_type\x18\x02 \x01(\x0e\x32".spark.connect.Aggregate.GroupTypeR\tgroupType\x12L\n\x14grouping_expressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12N\n\x15\x61ggregate_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x14\x61ggregateExpressions\x12\x34\n\x05pivot\x18\x05 \x01(\x0b\x32\x1e.spark.connect.Aggregate.PivotR\x05pivot\x12J\n\rgrouping_sets\x18\x06 \x03(\x0b\x32%.spark.connect.Aggregate.GroupingSetsR\x0cgroupingSets\x1ao\n\x05Pivot\x12+\n\x03\x63ol\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1aL\n\x0cGroupingSets\x12<\n\x0cgrouping_set\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0bgroupingSet"\x9f\x01\n\tGroupType\x12\x1a\n\x16GROUP_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12GROUP_TYPE_GROUPBY\x10\x01\x12\x15\n\x11GROUP_TYPE_ROLLUP\x10\x02\x12\x13\n\x0fGROUP_TYPE_CUBE\x10\x03\x12\x14\n\x10GROUP_TYPE_PIVOT\x10\x04\x12\x1c\n\x18GROUP_TYPE_GROUPING_SETS\x10\x05"\xa0\x01\n\x04Sort\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x05order\x18\x02 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\x05order\x12 \n\tis_global\x18\x03 \x01(\x08H\x00R\x08isGlobal\x88\x01\x01\x42\x0c\n\n_is_global"\x8d\x01\n\x04\x44rop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x33\n\x07\x63olumns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07\x63olumns\x12!\n\x0c\x63olumn_names\x18\x03 \x03(\tR\x0b\x63olumnNames"\xf0\x01\n\x0b\x44\x65\x64uplicate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames\x12\x32\n\x13\x61ll_columns_as_keys\x18\x03 \x01(\x08H\x00R\x10\x61llColumnsAsKeys\x88\x01\x01\x12.\n\x10within_watermark\x18\x04 \x01(\x08H\x01R\x0fwithinWatermark\x88\x01\x01\x42\x16\n\x14_all_columns_as_keysB\x13\n\x11_within_watermark"Y\n\rLocalRelation\x12\x17\n\x04\x64\x61ta\x18\x01 \x01(\x0cH\x00R\x04\x64\x61ta\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x42\x07\n\x05_dataB\t\n\x07_schema"H\n\x13\x43\x61\x63hedLocalRelation\x12\x12\n\x04hash\x18\x03 \x01(\tR\x04hashJ\x04\x08\x01\x10\x02J\x04\x08\x02\x10\x03R\x06userIdR\tsessionId"7\n\x14\x43\x61\x63hedRemoteRelation\x12\x1f\n\x0brelation_id\x18\x01 \x01(\tR\nrelationId"\x91\x02\n\x06Sample\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1f\n\x0blower_bound\x18\x02 \x01(\x01R\nlowerBound\x12\x1f\n\x0bupper_bound\x18\x03 \x01(\x01R\nupperBound\x12.\n\x10with_replacement\x18\x04 \x01(\x08H\x00R\x0fwithReplacement\x88\x01\x01\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x01R\x04seed\x88\x01\x01\x12/\n\x13\x64\x65terministic_order\x18\x06 \x01(\x08R\x12\x64\x65terministicOrderB\x13\n\x11_with_replacementB\x07\n\x05_seed"\x91\x01\n\x05Range\x12\x19\n\x05start\x18\x01 \x01(\x03H\x00R\x05start\x88\x01\x01\x12\x10\n\x03\x65nd\x18\x02 \x01(\x03R\x03\x65nd\x12\x12\n\x04step\x18\x03 \x01(\x03R\x04step\x12*\n\x0enum_partitions\x18\x04 \x01(\x05H\x01R\rnumPartitions\x88\x01\x01\x42\x08\n\x06_startB\x11\n\x0f_num_partitions"r\n\rSubqueryAlias\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05\x61lias\x18\x02 \x01(\tR\x05\x61lias\x12\x1c\n\tqualifier\x18\x03 \x03(\tR\tqualifier"\x8e\x01\n\x0bRepartition\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12%\n\x0enum_partitions\x18\x02 \x01(\x05R\rnumPartitions\x12\x1d\n\x07shuffle\x18\x03 \x01(\x08H\x00R\x07shuffle\x88\x01\x01\x42\n\n\x08_shuffle"\x8e\x01\n\nShowString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate\x12\x1a\n\x08vertical\x18\x04 \x01(\x08R\x08vertical"r\n\nHtmlString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate"\\\n\x0bStatSummary\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1e\n\nstatistics\x18\x02 \x03(\tR\nstatistics"Q\n\x0cStatDescribe\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols"e\n\x0cStatCrosstab\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"`\n\x07StatCov\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"\x89\x01\n\x08StatCorr\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2\x12\x1b\n\x06method\x18\x04 \x01(\tH\x00R\x06method\x88\x01\x01\x42\t\n\x07_method"\xa4\x01\n\x12StatApproxQuantile\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12$\n\rprobabilities\x18\x03 \x03(\x01R\rprobabilities\x12%\n\x0erelative_error\x18\x04 \x01(\x01R\rrelativeError"}\n\rStatFreqItems\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x1d\n\x07support\x18\x03 \x01(\x01H\x00R\x07support\x88\x01\x01\x42\n\n\x08_support"\xb5\x02\n\x0cStatSampleBy\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03\x63ol\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x42\n\tfractions\x18\x03 \x03(\x0b\x32$.spark.connect.StatSampleBy.FractionR\tfractions\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x00R\x04seed\x88\x01\x01\x1a\x63\n\x08\x46raction\x12;\n\x07stratum\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x07stratum\x12\x1a\n\x08\x66raction\x18\x02 \x01(\x01R\x08\x66ractionB\x07\n\x05_seed"\x86\x01\n\x06NAFill\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x39\n\x06values\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values"\x86\x01\n\x06NADrop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\'\n\rmin_non_nulls\x18\x03 \x01(\x05H\x00R\x0bminNonNulls\x88\x01\x01\x42\x10\n\x0e_min_non_nulls"\xa8\x02\n\tNAReplace\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12H\n\x0creplacements\x18\x03 \x03(\x0b\x32$.spark.connect.NAReplace.ReplacementR\x0creplacements\x1a\x8d\x01\n\x0bReplacement\x12>\n\told_value\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08oldValue\x12>\n\tnew_value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08newValue"X\n\x04ToDF\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames"\xfe\x02\n\x12WithColumnsRenamed\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12i\n\x12rename_columns_map\x18\x02 \x03(\x0b\x32\x37.spark.connect.WithColumnsRenamed.RenameColumnsMapEntryB\x02\x18\x01R\x10renameColumnsMap\x12\x42\n\x07renames\x18\x03 \x03(\x0b\x32(.spark.connect.WithColumnsRenamed.RenameR\x07renames\x1a\x43\n\x15RenameColumnsMapEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x45\n\x06Rename\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12 \n\x0cnew_col_name\x18\x02 \x01(\tR\nnewColName"w\n\x0bWithColumns\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x07\x61liases\x18\x02 \x03(\x0b\x32\x1f.spark.connect.Expression.AliasR\x07\x61liases"\x86\x01\n\rWithWatermark\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\nevent_time\x18\x02 \x01(\tR\teventTime\x12\'\n\x0f\x64\x65lay_threshold\x18\x03 \x01(\tR\x0e\x64\x65layThreshold"\x84\x01\n\x04Hint\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x39\n\nparameters\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\nparameters"\xc7\x02\n\x07Unpivot\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03ids\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x03ids\x12:\n\x06values\x18\x03 \x01(\x0b\x32\x1d.spark.connect.Unpivot.ValuesH\x00R\x06values\x88\x01\x01\x12\x30\n\x14variable_column_name\x18\x04 \x01(\tR\x12variableColumnName\x12*\n\x11value_column_name\x18\x05 \x01(\tR\x0fvalueColumnName\x1a;\n\x06Values\x12\x31\n\x06values\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x06valuesB\t\n\x07_values"z\n\tTranspose\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12>\n\rindex_columns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cindexColumns"}\n\x1dUnresolvedTableValuedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"j\n\x08ToSchema\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema"\xcb\x01\n\x17RepartitionByExpression\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x0fpartition_exprs\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0epartitionExprs\x12*\n\x0enum_partitions\x18\x03 \x01(\x05H\x00R\rnumPartitions\x88\x01\x01\x42\x11\n\x0f_num_partitions"\xe8\x01\n\rMapPartitions\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x04\x66unc\x18\x02 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12"\n\nis_barrier\x18\x03 \x01(\x08H\x00R\tisBarrier\x88\x01\x01\x12"\n\nprofile_id\x18\x04 \x01(\x05H\x01R\tprofileId\x88\x01\x01\x42\r\n\x0b_is_barrierB\r\n\x0b_profile_id"\xd2\x06\n\x08GroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12J\n\x13sorting_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x12sortingExpressions\x12<\n\rinitial_input\x18\x05 \x01(\x0b\x32\x17.spark.connect.RelationR\x0cinitialInput\x12[\n\x1cinitial_grouping_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x1ainitialGroupingExpressions\x12;\n\x18is_map_groups_with_state\x18\x07 \x01(\x08H\x00R\x14isMapGroupsWithState\x88\x01\x01\x12$\n\x0boutput_mode\x18\x08 \x01(\tH\x01R\noutputMode\x88\x01\x01\x12&\n\x0ctimeout_conf\x18\t \x01(\tH\x02R\x0btimeoutConf\x88\x01\x01\x12?\n\x0cstate_schema\x18\n \x01(\x0b\x32\x17.spark.connect.DataTypeH\x03R\x0bstateSchema\x88\x01\x01\x12\x65\n\x19transform_with_state_info\x18\x0b \x01(\x0b\x32%.spark.connect.TransformWithStateInfoH\x04R\x16transformWithStateInfo\x88\x01\x01\x42\x1b\n\x19_is_map_groups_with_stateB\x0e\n\x0c_output_modeB\x0f\n\r_timeout_confB\x0f\n\r_state_schemaB\x1c\n\x1a_transform_with_state_info"\xdf\x01\n\x16TransformWithStateInfo\x12\x1b\n\ttime_mode\x18\x01 \x01(\tR\x08timeMode\x12\x38\n\x16\x65vent_time_column_name\x18\x02 \x01(\tH\x00R\x13\x65ventTimeColumnName\x88\x01\x01\x12\x41\n\routput_schema\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x01R\x0coutputSchema\x88\x01\x01\x42\x19\n\x17_event_time_column_nameB\x10\n\x0e_output_schema"\x8e\x04\n\nCoGroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12W\n\x1ainput_grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18inputGroupingExpressions\x12-\n\x05other\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05other\x12W\n\x1aother_grouping_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18otherGroupingExpressions\x12\x42\n\x04\x66unc\x18\x05 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12U\n\x19input_sorting_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17inputSortingExpressions\x12U\n\x19other_sorting_expressions\x18\x07 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17otherSortingExpressions"\xe5\x02\n\x16\x41pplyInPandasWithState\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12#\n\routput_schema\x18\x04 \x01(\tR\x0coutputSchema\x12!\n\x0cstate_schema\x18\x05 \x01(\tR\x0bstateSchema\x12\x1f\n\x0boutput_mode\x18\x06 \x01(\tR\noutputMode\x12!\n\x0ctimeout_conf\x18\x07 \x01(\tR\x0btimeoutConf"\xf4\x01\n$CommonInlineUserDefinedTableFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12<\n\x0bpython_udtf\x18\x04 \x01(\x0b\x32\x19.spark.connect.PythonUDTFH\x00R\npythonUdtfB\n\n\x08\x66unction"\xb1\x01\n\nPythonUDTF\x12=\n\x0breturn_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\nreturnType\x88\x01\x01\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVerB\x0e\n\x0c_return_type"\x97\x01\n!CommonInlineUserDefinedDataSource\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12O\n\x12python_data_source\x18\x02 \x01(\x0b\x32\x1f.spark.connect.PythonDataSourceH\x00R\x10pythonDataSourceB\r\n\x0b\x64\x61ta_source"K\n\x10PythonDataSource\x12\x18\n\x07\x63ommand\x18\x01 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x02 \x01(\tR\tpythonVer"\x88\x01\n\x0e\x43ollectMetrics\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x33\n\x07metrics\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07metrics"\x84\x03\n\x05Parse\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x38\n\x06\x66ormat\x18\x02 \x01(\x0e\x32 .spark.connect.Parse.ParseFormatR\x06\x66ormat\x12\x34\n\x06schema\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x06schema\x88\x01\x01\x12;\n\x07options\x18\x04 \x03(\x0b\x32!.spark.connect.Parse.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"X\n\x0bParseFormat\x12\x1c\n\x18PARSE_FORMAT_UNSPECIFIED\x10\x00\x12\x14\n\x10PARSE_FORMAT_CSV\x10\x01\x12\x15\n\x11PARSE_FORMAT_JSON\x10\x02\x42\t\n\x07_schema"\xdb\x03\n\x08\x41sOfJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12\x37\n\nleft_as_of\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08leftAsOf\x12\x39\n\x0bright_as_of\x18\x04 \x01(\x0b\x32\x19.spark.connect.ExpressionR\trightAsOf\x12\x36\n\tjoin_expr\x18\x05 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08joinExpr\x12#\n\rusing_columns\x18\x06 \x03(\tR\x0cusingColumns\x12\x1b\n\tjoin_type\x18\x07 \x01(\tR\x08joinType\x12\x37\n\ttolerance\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\ttolerance\x12.\n\x13\x61llow_exact_matches\x18\t \x01(\x08R\x11\x61llowExactMatches\x12\x1c\n\tdirection\x18\n \x01(\tR\tdirection"\xe6\x01\n\x0bLateralJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinTypeB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3'
)
_globals = globals()
@@ -81,169 +81,169 @@
_globals["_RELATION"]._serialized_start = 224
_globals["_RELATION"]._serialized_end = 3964
_globals["_MLRELATION"]._serialized_start = 3967
- _globals["_MLRELATION"]._serialized_end = 4343
- _globals["_MLRELATION_TRANSFORM"]._serialized_start = 4097
- _globals["_MLRELATION_TRANSFORM"]._serialized_end = 4332
- _globals["_FETCH"]._serialized_start = 4346
- _globals["_FETCH"]._serialized_end = 4677
- _globals["_FETCH_METHOD"]._serialized_start = 4462
- _globals["_FETCH_METHOD"]._serialized_end = 4677
- _globals["_FETCH_METHOD_ARGS"]._serialized_start = 4550
- _globals["_FETCH_METHOD_ARGS"]._serialized_end = 4677
- _globals["_UNKNOWN"]._serialized_start = 4679
- _globals["_UNKNOWN"]._serialized_end = 4688
- _globals["_RELATIONCOMMON"]._serialized_start = 4691
- _globals["_RELATIONCOMMON"]._serialized_end = 4833
- _globals["_SQL"]._serialized_start = 4836
- _globals["_SQL"]._serialized_end = 5314
- _globals["_SQL_ARGSENTRY"]._serialized_start = 5130
- _globals["_SQL_ARGSENTRY"]._serialized_end = 5220
- _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_start = 5222
- _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_end = 5314
- _globals["_WITHRELATIONS"]._serialized_start = 5316
- _globals["_WITHRELATIONS"]._serialized_end = 5433
- _globals["_READ"]._serialized_start = 5436
- _globals["_READ"]._serialized_end = 6099
- _globals["_READ_NAMEDTABLE"]._serialized_start = 5614
- _globals["_READ_NAMEDTABLE"]._serialized_end = 5806
- _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_start = 5748
- _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_end = 5806
- _globals["_READ_DATASOURCE"]._serialized_start = 5809
- _globals["_READ_DATASOURCE"]._serialized_end = 6086
- _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_start = 5748
- _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_end = 5806
- _globals["_PROJECT"]._serialized_start = 6101
- _globals["_PROJECT"]._serialized_end = 6218
- _globals["_FILTER"]._serialized_start = 6220
- _globals["_FILTER"]._serialized_end = 6332
- _globals["_JOIN"]._serialized_start = 6335
- _globals["_JOIN"]._serialized_end = 6996
- _globals["_JOIN_JOINDATATYPE"]._serialized_start = 6674
- _globals["_JOIN_JOINDATATYPE"]._serialized_end = 6766
- _globals["_JOIN_JOINTYPE"]._serialized_start = 6769
- _globals["_JOIN_JOINTYPE"]._serialized_end = 6977
- _globals["_SETOPERATION"]._serialized_start = 6999
- _globals["_SETOPERATION"]._serialized_end = 7478
- _globals["_SETOPERATION_SETOPTYPE"]._serialized_start = 7315
- _globals["_SETOPERATION_SETOPTYPE"]._serialized_end = 7429
- _globals["_LIMIT"]._serialized_start = 7480
- _globals["_LIMIT"]._serialized_end = 7556
- _globals["_OFFSET"]._serialized_start = 7558
- _globals["_OFFSET"]._serialized_end = 7637
- _globals["_TAIL"]._serialized_start = 7639
- _globals["_TAIL"]._serialized_end = 7714
- _globals["_AGGREGATE"]._serialized_start = 7717
- _globals["_AGGREGATE"]._serialized_end = 8483
- _globals["_AGGREGATE_PIVOT"]._serialized_start = 8132
- _globals["_AGGREGATE_PIVOT"]._serialized_end = 8243
- _globals["_AGGREGATE_GROUPINGSETS"]._serialized_start = 8245
- _globals["_AGGREGATE_GROUPINGSETS"]._serialized_end = 8321
- _globals["_AGGREGATE_GROUPTYPE"]._serialized_start = 8324
- _globals["_AGGREGATE_GROUPTYPE"]._serialized_end = 8483
- _globals["_SORT"]._serialized_start = 8486
- _globals["_SORT"]._serialized_end = 8646
- _globals["_DROP"]._serialized_start = 8649
- _globals["_DROP"]._serialized_end = 8790
- _globals["_DEDUPLICATE"]._serialized_start = 8793
- _globals["_DEDUPLICATE"]._serialized_end = 9033
- _globals["_LOCALRELATION"]._serialized_start = 9035
- _globals["_LOCALRELATION"]._serialized_end = 9124
- _globals["_CACHEDLOCALRELATION"]._serialized_start = 9126
- _globals["_CACHEDLOCALRELATION"]._serialized_end = 9198
- _globals["_CACHEDREMOTERELATION"]._serialized_start = 9200
- _globals["_CACHEDREMOTERELATION"]._serialized_end = 9255
- _globals["_SAMPLE"]._serialized_start = 9258
- _globals["_SAMPLE"]._serialized_end = 9531
- _globals["_RANGE"]._serialized_start = 9534
- _globals["_RANGE"]._serialized_end = 9679
- _globals["_SUBQUERYALIAS"]._serialized_start = 9681
- _globals["_SUBQUERYALIAS"]._serialized_end = 9795
- _globals["_REPARTITION"]._serialized_start = 9798
- _globals["_REPARTITION"]._serialized_end = 9940
- _globals["_SHOWSTRING"]._serialized_start = 9943
- _globals["_SHOWSTRING"]._serialized_end = 10085
- _globals["_HTMLSTRING"]._serialized_start = 10087
- _globals["_HTMLSTRING"]._serialized_end = 10201
- _globals["_STATSUMMARY"]._serialized_start = 10203
- _globals["_STATSUMMARY"]._serialized_end = 10295
- _globals["_STATDESCRIBE"]._serialized_start = 10297
- _globals["_STATDESCRIBE"]._serialized_end = 10378
- _globals["_STATCROSSTAB"]._serialized_start = 10380
- _globals["_STATCROSSTAB"]._serialized_end = 10481
- _globals["_STATCOV"]._serialized_start = 10483
- _globals["_STATCOV"]._serialized_end = 10579
- _globals["_STATCORR"]._serialized_start = 10582
- _globals["_STATCORR"]._serialized_end = 10719
- _globals["_STATAPPROXQUANTILE"]._serialized_start = 10722
- _globals["_STATAPPROXQUANTILE"]._serialized_end = 10886
- _globals["_STATFREQITEMS"]._serialized_start = 10888
- _globals["_STATFREQITEMS"]._serialized_end = 11013
- _globals["_STATSAMPLEBY"]._serialized_start = 11016
- _globals["_STATSAMPLEBY"]._serialized_end = 11325
- _globals["_STATSAMPLEBY_FRACTION"]._serialized_start = 11217
- _globals["_STATSAMPLEBY_FRACTION"]._serialized_end = 11316
- _globals["_NAFILL"]._serialized_start = 11328
- _globals["_NAFILL"]._serialized_end = 11462
- _globals["_NADROP"]._serialized_start = 11465
- _globals["_NADROP"]._serialized_end = 11599
- _globals["_NAREPLACE"]._serialized_start = 11602
- _globals["_NAREPLACE"]._serialized_end = 11898
- _globals["_NAREPLACE_REPLACEMENT"]._serialized_start = 11757
- _globals["_NAREPLACE_REPLACEMENT"]._serialized_end = 11898
- _globals["_TODF"]._serialized_start = 11900
- _globals["_TODF"]._serialized_end = 11988
- _globals["_WITHCOLUMNSRENAMED"]._serialized_start = 11991
- _globals["_WITHCOLUMNSRENAMED"]._serialized_end = 12373
- _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_start = 12235
- _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_end = 12302
- _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_start = 12304
- _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_end = 12373
- _globals["_WITHCOLUMNS"]._serialized_start = 12375
- _globals["_WITHCOLUMNS"]._serialized_end = 12494
- _globals["_WITHWATERMARK"]._serialized_start = 12497
- _globals["_WITHWATERMARK"]._serialized_end = 12631
- _globals["_HINT"]._serialized_start = 12634
- _globals["_HINT"]._serialized_end = 12766
- _globals["_UNPIVOT"]._serialized_start = 12769
- _globals["_UNPIVOT"]._serialized_end = 13096
- _globals["_UNPIVOT_VALUES"]._serialized_start = 13026
- _globals["_UNPIVOT_VALUES"]._serialized_end = 13085
- _globals["_TRANSPOSE"]._serialized_start = 13098
- _globals["_TRANSPOSE"]._serialized_end = 13220
- _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_start = 13222
- _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_end = 13347
- _globals["_TOSCHEMA"]._serialized_start = 13349
- _globals["_TOSCHEMA"]._serialized_end = 13455
- _globals["_REPARTITIONBYEXPRESSION"]._serialized_start = 13458
- _globals["_REPARTITIONBYEXPRESSION"]._serialized_end = 13661
- _globals["_MAPPARTITIONS"]._serialized_start = 13664
- _globals["_MAPPARTITIONS"]._serialized_end = 13896
- _globals["_GROUPMAP"]._serialized_start = 13899
- _globals["_GROUPMAP"]._serialized_end = 14749
- _globals["_TRANSFORMWITHSTATEINFO"]._serialized_start = 14752
- _globals["_TRANSFORMWITHSTATEINFO"]._serialized_end = 14975
- _globals["_COGROUPMAP"]._serialized_start = 14978
- _globals["_COGROUPMAP"]._serialized_end = 15504
- _globals["_APPLYINPANDASWITHSTATE"]._serialized_start = 15507
- _globals["_APPLYINPANDASWITHSTATE"]._serialized_end = 15864
- _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_start = 15867
- _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_end = 16111
- _globals["_PYTHONUDTF"]._serialized_start = 16114
- _globals["_PYTHONUDTF"]._serialized_end = 16291
- _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_start = 16294
- _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_end = 16445
- _globals["_PYTHONDATASOURCE"]._serialized_start = 16447
- _globals["_PYTHONDATASOURCE"]._serialized_end = 16522
- _globals["_COLLECTMETRICS"]._serialized_start = 16525
- _globals["_COLLECTMETRICS"]._serialized_end = 16661
- _globals["_PARSE"]._serialized_start = 16664
- _globals["_PARSE"]._serialized_end = 17052
- _globals["_PARSE_OPTIONSENTRY"]._serialized_start = 5748
- _globals["_PARSE_OPTIONSENTRY"]._serialized_end = 5806
- _globals["_PARSE_PARSEFORMAT"]._serialized_start = 16953
- _globals["_PARSE_PARSEFORMAT"]._serialized_end = 17041
- _globals["_ASOFJOIN"]._serialized_start = 17055
- _globals["_ASOFJOIN"]._serialized_end = 17530
- _globals["_LATERALJOIN"]._serialized_start = 17533
- _globals["_LATERALJOIN"]._serialized_end = 17763
+ _globals["_MLRELATION"]._serialized_end = 4451
+ _globals["_MLRELATION_TRANSFORM"]._serialized_start = 4179
+ _globals["_MLRELATION_TRANSFORM"]._serialized_end = 4414
+ _globals["_FETCH"]._serialized_start = 4454
+ _globals["_FETCH"]._serialized_end = 4785
+ _globals["_FETCH_METHOD"]._serialized_start = 4570
+ _globals["_FETCH_METHOD"]._serialized_end = 4785
+ _globals["_FETCH_METHOD_ARGS"]._serialized_start = 4658
+ _globals["_FETCH_METHOD_ARGS"]._serialized_end = 4785
+ _globals["_UNKNOWN"]._serialized_start = 4787
+ _globals["_UNKNOWN"]._serialized_end = 4796
+ _globals["_RELATIONCOMMON"]._serialized_start = 4799
+ _globals["_RELATIONCOMMON"]._serialized_end = 4941
+ _globals["_SQL"]._serialized_start = 4944
+ _globals["_SQL"]._serialized_end = 5422
+ _globals["_SQL_ARGSENTRY"]._serialized_start = 5238
+ _globals["_SQL_ARGSENTRY"]._serialized_end = 5328
+ _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_start = 5330
+ _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_end = 5422
+ _globals["_WITHRELATIONS"]._serialized_start = 5424
+ _globals["_WITHRELATIONS"]._serialized_end = 5541
+ _globals["_READ"]._serialized_start = 5544
+ _globals["_READ"]._serialized_end = 6207
+ _globals["_READ_NAMEDTABLE"]._serialized_start = 5722
+ _globals["_READ_NAMEDTABLE"]._serialized_end = 5914
+ _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_start = 5856
+ _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_end = 5914
+ _globals["_READ_DATASOURCE"]._serialized_start = 5917
+ _globals["_READ_DATASOURCE"]._serialized_end = 6194
+ _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_start = 5856
+ _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_end = 5914
+ _globals["_PROJECT"]._serialized_start = 6209
+ _globals["_PROJECT"]._serialized_end = 6326
+ _globals["_FILTER"]._serialized_start = 6328
+ _globals["_FILTER"]._serialized_end = 6440
+ _globals["_JOIN"]._serialized_start = 6443
+ _globals["_JOIN"]._serialized_end = 7104
+ _globals["_JOIN_JOINDATATYPE"]._serialized_start = 6782
+ _globals["_JOIN_JOINDATATYPE"]._serialized_end = 6874
+ _globals["_JOIN_JOINTYPE"]._serialized_start = 6877
+ _globals["_JOIN_JOINTYPE"]._serialized_end = 7085
+ _globals["_SETOPERATION"]._serialized_start = 7107
+ _globals["_SETOPERATION"]._serialized_end = 7586
+ _globals["_SETOPERATION_SETOPTYPE"]._serialized_start = 7423
+ _globals["_SETOPERATION_SETOPTYPE"]._serialized_end = 7537
+ _globals["_LIMIT"]._serialized_start = 7588
+ _globals["_LIMIT"]._serialized_end = 7664
+ _globals["_OFFSET"]._serialized_start = 7666
+ _globals["_OFFSET"]._serialized_end = 7745
+ _globals["_TAIL"]._serialized_start = 7747
+ _globals["_TAIL"]._serialized_end = 7822
+ _globals["_AGGREGATE"]._serialized_start = 7825
+ _globals["_AGGREGATE"]._serialized_end = 8591
+ _globals["_AGGREGATE_PIVOT"]._serialized_start = 8240
+ _globals["_AGGREGATE_PIVOT"]._serialized_end = 8351
+ _globals["_AGGREGATE_GROUPINGSETS"]._serialized_start = 8353
+ _globals["_AGGREGATE_GROUPINGSETS"]._serialized_end = 8429
+ _globals["_AGGREGATE_GROUPTYPE"]._serialized_start = 8432
+ _globals["_AGGREGATE_GROUPTYPE"]._serialized_end = 8591
+ _globals["_SORT"]._serialized_start = 8594
+ _globals["_SORT"]._serialized_end = 8754
+ _globals["_DROP"]._serialized_start = 8757
+ _globals["_DROP"]._serialized_end = 8898
+ _globals["_DEDUPLICATE"]._serialized_start = 8901
+ _globals["_DEDUPLICATE"]._serialized_end = 9141
+ _globals["_LOCALRELATION"]._serialized_start = 9143
+ _globals["_LOCALRELATION"]._serialized_end = 9232
+ _globals["_CACHEDLOCALRELATION"]._serialized_start = 9234
+ _globals["_CACHEDLOCALRELATION"]._serialized_end = 9306
+ _globals["_CACHEDREMOTERELATION"]._serialized_start = 9308
+ _globals["_CACHEDREMOTERELATION"]._serialized_end = 9363
+ _globals["_SAMPLE"]._serialized_start = 9366
+ _globals["_SAMPLE"]._serialized_end = 9639
+ _globals["_RANGE"]._serialized_start = 9642
+ _globals["_RANGE"]._serialized_end = 9787
+ _globals["_SUBQUERYALIAS"]._serialized_start = 9789
+ _globals["_SUBQUERYALIAS"]._serialized_end = 9903
+ _globals["_REPARTITION"]._serialized_start = 9906
+ _globals["_REPARTITION"]._serialized_end = 10048
+ _globals["_SHOWSTRING"]._serialized_start = 10051
+ _globals["_SHOWSTRING"]._serialized_end = 10193
+ _globals["_HTMLSTRING"]._serialized_start = 10195
+ _globals["_HTMLSTRING"]._serialized_end = 10309
+ _globals["_STATSUMMARY"]._serialized_start = 10311
+ _globals["_STATSUMMARY"]._serialized_end = 10403
+ _globals["_STATDESCRIBE"]._serialized_start = 10405
+ _globals["_STATDESCRIBE"]._serialized_end = 10486
+ _globals["_STATCROSSTAB"]._serialized_start = 10488
+ _globals["_STATCROSSTAB"]._serialized_end = 10589
+ _globals["_STATCOV"]._serialized_start = 10591
+ _globals["_STATCOV"]._serialized_end = 10687
+ _globals["_STATCORR"]._serialized_start = 10690
+ _globals["_STATCORR"]._serialized_end = 10827
+ _globals["_STATAPPROXQUANTILE"]._serialized_start = 10830
+ _globals["_STATAPPROXQUANTILE"]._serialized_end = 10994
+ _globals["_STATFREQITEMS"]._serialized_start = 10996
+ _globals["_STATFREQITEMS"]._serialized_end = 11121
+ _globals["_STATSAMPLEBY"]._serialized_start = 11124
+ _globals["_STATSAMPLEBY"]._serialized_end = 11433
+ _globals["_STATSAMPLEBY_FRACTION"]._serialized_start = 11325
+ _globals["_STATSAMPLEBY_FRACTION"]._serialized_end = 11424
+ _globals["_NAFILL"]._serialized_start = 11436
+ _globals["_NAFILL"]._serialized_end = 11570
+ _globals["_NADROP"]._serialized_start = 11573
+ _globals["_NADROP"]._serialized_end = 11707
+ _globals["_NAREPLACE"]._serialized_start = 11710
+ _globals["_NAREPLACE"]._serialized_end = 12006
+ _globals["_NAREPLACE_REPLACEMENT"]._serialized_start = 11865
+ _globals["_NAREPLACE_REPLACEMENT"]._serialized_end = 12006
+ _globals["_TODF"]._serialized_start = 12008
+ _globals["_TODF"]._serialized_end = 12096
+ _globals["_WITHCOLUMNSRENAMED"]._serialized_start = 12099
+ _globals["_WITHCOLUMNSRENAMED"]._serialized_end = 12481
+ _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_start = 12343
+ _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_end = 12410
+ _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_start = 12412
+ _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_end = 12481
+ _globals["_WITHCOLUMNS"]._serialized_start = 12483
+ _globals["_WITHCOLUMNS"]._serialized_end = 12602
+ _globals["_WITHWATERMARK"]._serialized_start = 12605
+ _globals["_WITHWATERMARK"]._serialized_end = 12739
+ _globals["_HINT"]._serialized_start = 12742
+ _globals["_HINT"]._serialized_end = 12874
+ _globals["_UNPIVOT"]._serialized_start = 12877
+ _globals["_UNPIVOT"]._serialized_end = 13204
+ _globals["_UNPIVOT_VALUES"]._serialized_start = 13134
+ _globals["_UNPIVOT_VALUES"]._serialized_end = 13193
+ _globals["_TRANSPOSE"]._serialized_start = 13206
+ _globals["_TRANSPOSE"]._serialized_end = 13328
+ _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_start = 13330
+ _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_end = 13455
+ _globals["_TOSCHEMA"]._serialized_start = 13457
+ _globals["_TOSCHEMA"]._serialized_end = 13563
+ _globals["_REPARTITIONBYEXPRESSION"]._serialized_start = 13566
+ _globals["_REPARTITIONBYEXPRESSION"]._serialized_end = 13769
+ _globals["_MAPPARTITIONS"]._serialized_start = 13772
+ _globals["_MAPPARTITIONS"]._serialized_end = 14004
+ _globals["_GROUPMAP"]._serialized_start = 14007
+ _globals["_GROUPMAP"]._serialized_end = 14857
+ _globals["_TRANSFORMWITHSTATEINFO"]._serialized_start = 14860
+ _globals["_TRANSFORMWITHSTATEINFO"]._serialized_end = 15083
+ _globals["_COGROUPMAP"]._serialized_start = 15086
+ _globals["_COGROUPMAP"]._serialized_end = 15612
+ _globals["_APPLYINPANDASWITHSTATE"]._serialized_start = 15615
+ _globals["_APPLYINPANDASWITHSTATE"]._serialized_end = 15972
+ _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_start = 15975
+ _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_end = 16219
+ _globals["_PYTHONUDTF"]._serialized_start = 16222
+ _globals["_PYTHONUDTF"]._serialized_end = 16399
+ _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_start = 16402
+ _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_end = 16553
+ _globals["_PYTHONDATASOURCE"]._serialized_start = 16555
+ _globals["_PYTHONDATASOURCE"]._serialized_end = 16630
+ _globals["_COLLECTMETRICS"]._serialized_start = 16633
+ _globals["_COLLECTMETRICS"]._serialized_end = 16769
+ _globals["_PARSE"]._serialized_start = 16772
+ _globals["_PARSE"]._serialized_end = 17160
+ _globals["_PARSE_OPTIONSENTRY"]._serialized_start = 5856
+ _globals["_PARSE_OPTIONSENTRY"]._serialized_end = 5914
+ _globals["_PARSE_PARSEFORMAT"]._serialized_start = 17061
+ _globals["_PARSE_PARSEFORMAT"]._serialized_end = 17149
+ _globals["_ASOFJOIN"]._serialized_start = 17163
+ _globals["_ASOFJOIN"]._serialized_end = 17638
+ _globals["_LATERALJOIN"]._serialized_start = 17641
+ _globals["_LATERALJOIN"]._serialized_end = 17871
# @@protoc_insertion_point(module_scope)
diff --git a/python/pyspark/sql/connect/proto/relations_pb2.pyi b/python/pyspark/sql/connect/proto/relations_pb2.pyi
index beeeb712da762..e1eb7945c19f0 100644
--- a/python/pyspark/sql/connect/proto/relations_pb2.pyi
+++ b/python/pyspark/sql/connect/proto/relations_pb2.pyi
@@ -707,28 +707,57 @@ class MlRelation(google.protobuf.message.Message):
TRANSFORM_FIELD_NUMBER: builtins.int
FETCH_FIELD_NUMBER: builtins.int
+ MODEL_SUMMARY_DATASET_FIELD_NUMBER: builtins.int
@property
def transform(self) -> global___MlRelation.Transform: ...
@property
def fetch(self) -> global___Fetch: ...
+ @property
+ def model_summary_dataset(self) -> global___Relation:
+ """(Optional) the dataset for restoring the model summary"""
def __init__(
self,
*,
transform: global___MlRelation.Transform | None = ...,
fetch: global___Fetch | None = ...,
+ model_summary_dataset: global___Relation | None = ...,
) -> None: ...
def HasField(
self,
field_name: typing_extensions.Literal[
- "fetch", b"fetch", "ml_type", b"ml_type", "transform", b"transform"
+ "_model_summary_dataset",
+ b"_model_summary_dataset",
+ "fetch",
+ b"fetch",
+ "ml_type",
+ b"ml_type",
+ "model_summary_dataset",
+ b"model_summary_dataset",
+ "transform",
+ b"transform",
],
) -> builtins.bool: ...
def ClearField(
self,
field_name: typing_extensions.Literal[
- "fetch", b"fetch", "ml_type", b"ml_type", "transform", b"transform"
+ "_model_summary_dataset",
+ b"_model_summary_dataset",
+ "fetch",
+ b"fetch",
+ "ml_type",
+ b"ml_type",
+ "model_summary_dataset",
+ b"model_summary_dataset",
+ "transform",
+ b"transform",
],
) -> None: ...
+ @typing.overload
+ def WhichOneof(
+ self,
+ oneof_group: typing_extensions.Literal["_model_summary_dataset", b"_model_summary_dataset"],
+ ) -> typing_extensions.Literal["model_summary_dataset"] | None: ...
+ @typing.overload
def WhichOneof(
self, oneof_group: typing_extensions.Literal["ml_type", b"ml_type"]
) -> typing_extensions.Literal["transform", "fetch"] | None: ...
diff --git a/python/pyspark/sql/connect/proto/types_pb2.py b/python/pyspark/sql/connect/proto/types_pb2.py
index 2e0ef70484666..3f43bf35af40f 100644
--- a/python/pyspark/sql/connect/proto/types_pb2.py
+++ b/python/pyspark/sql/connect/proto/types_pb2.py
@@ -35,7 +35,7 @@
DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(
- b"\n\x19spark/connect/types.proto\x12\rspark.connect\"\xf9!\n\x08\x44\x61taType\x12\x32\n\x04null\x18\x01 \x01(\x0b\x32\x1c.spark.connect.DataType.NULLH\x00R\x04null\x12\x38\n\x06\x62inary\x18\x02 \x01(\x0b\x32\x1e.spark.connect.DataType.BinaryH\x00R\x06\x62inary\x12;\n\x07\x62oolean\x18\x03 \x01(\x0b\x32\x1f.spark.connect.DataType.BooleanH\x00R\x07\x62oolean\x12\x32\n\x04\x62yte\x18\x04 \x01(\x0b\x32\x1c.spark.connect.DataType.ByteH\x00R\x04\x62yte\x12\x35\n\x05short\x18\x05 \x01(\x0b\x32\x1d.spark.connect.DataType.ShortH\x00R\x05short\x12;\n\x07integer\x18\x06 \x01(\x0b\x32\x1f.spark.connect.DataType.IntegerH\x00R\x07integer\x12\x32\n\x04long\x18\x07 \x01(\x0b\x32\x1c.spark.connect.DataType.LongH\x00R\x04long\x12\x35\n\x05\x66loat\x18\x08 \x01(\x0b\x32\x1d.spark.connect.DataType.FloatH\x00R\x05\x66loat\x12\x38\n\x06\x64ouble\x18\t \x01(\x0b\x32\x1e.spark.connect.DataType.DoubleH\x00R\x06\x64ouble\x12;\n\x07\x64\x65\x63imal\x18\n \x01(\x0b\x32\x1f.spark.connect.DataType.DecimalH\x00R\x07\x64\x65\x63imal\x12\x38\n\x06string\x18\x0b \x01(\x0b\x32\x1e.spark.connect.DataType.StringH\x00R\x06string\x12\x32\n\x04\x63har\x18\x0c \x01(\x0b\x32\x1c.spark.connect.DataType.CharH\x00R\x04\x63har\x12<\n\x08var_char\x18\r \x01(\x0b\x32\x1f.spark.connect.DataType.VarCharH\x00R\x07varChar\x12\x32\n\x04\x64\x61te\x18\x0e \x01(\x0b\x32\x1c.spark.connect.DataType.DateH\x00R\x04\x64\x61te\x12\x41\n\ttimestamp\x18\x0f \x01(\x0b\x32!.spark.connect.DataType.TimestampH\x00R\ttimestamp\x12K\n\rtimestamp_ntz\x18\x10 \x01(\x0b\x32$.spark.connect.DataType.TimestampNTZH\x00R\x0ctimestampNtz\x12W\n\x11\x63\x61lendar_interval\x18\x11 \x01(\x0b\x32(.spark.connect.DataType.CalendarIntervalH\x00R\x10\x63\x61lendarInterval\x12[\n\x13year_month_interval\x18\x12 \x01(\x0b\x32).spark.connect.DataType.YearMonthIntervalH\x00R\x11yearMonthInterval\x12U\n\x11\x64\x61y_time_interval\x18\x13 \x01(\x0b\x32'.spark.connect.DataType.DayTimeIntervalH\x00R\x0f\x64\x61yTimeInterval\x12\x35\n\x05\x61rray\x18\x14 \x01(\x0b\x32\x1d.spark.connect.DataType.ArrayH\x00R\x05\x61rray\x12\x38\n\x06struct\x18\x15 \x01(\x0b\x32\x1e.spark.connect.DataType.StructH\x00R\x06struct\x12/\n\x03map\x18\x16 \x01(\x0b\x32\x1b.spark.connect.DataType.MapH\x00R\x03map\x12;\n\x07variant\x18\x19 \x01(\x0b\x32\x1f.spark.connect.DataType.VariantH\x00R\x07variant\x12/\n\x03udt\x18\x17 \x01(\x0b\x32\x1b.spark.connect.DataType.UDTH\x00R\x03udt\x12>\n\x08unparsed\x18\x18 \x01(\x0b\x32 .spark.connect.DataType.UnparsedH\x00R\x08unparsed\x1a\x43\n\x07\x42oolean\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a@\n\x04\x42yte\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\x41\n\x05Short\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\x43\n\x07Integer\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a@\n\x04Long\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\x41\n\x05\x46loat\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\x42\n\x06\x44ouble\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a`\n\x06String\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x12\x1c\n\tcollation\x18\x02 \x01(\tR\tcollation\x1a\x42\n\x06\x42inary\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a@\n\x04NULL\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\x45\n\tTimestamp\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a@\n\x04\x44\x61te\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1aH\n\x0cTimestampNTZ\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1aL\n\x10\x43\x61lendarInterval\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\xb3\x01\n\x11YearMonthInterval\x12$\n\x0bstart_field\x18\x01 \x01(\x05H\x00R\nstartField\x88\x01\x01\x12 \n\tend_field\x18\x02 \x01(\x05H\x01R\x08\x65ndField\x88\x01\x01\x12\x38\n\x18type_variation_reference\x18\x03 \x01(\rR\x16typeVariationReferenceB\x0e\n\x0c_start_fieldB\x0c\n\n_end_field\x1a\xb1\x01\n\x0f\x44\x61yTimeInterval\x12$\n\x0bstart_field\x18\x01 \x01(\x05H\x00R\nstartField\x88\x01\x01\x12 \n\tend_field\x18\x02 \x01(\x05H\x01R\x08\x65ndField\x88\x01\x01\x12\x38\n\x18type_variation_reference\x18\x03 \x01(\rR\x16typeVariationReferenceB\x0e\n\x0c_start_fieldB\x0c\n\n_end_field\x1aX\n\x04\x43har\x12\x16\n\x06length\x18\x01 \x01(\x05R\x06length\x12\x38\n\x18type_variation_reference\x18\x02 \x01(\rR\x16typeVariationReference\x1a[\n\x07VarChar\x12\x16\n\x06length\x18\x01 \x01(\x05R\x06length\x12\x38\n\x18type_variation_reference\x18\x02 \x01(\rR\x16typeVariationReference\x1a\x99\x01\n\x07\x44\x65\x63imal\x12\x19\n\x05scale\x18\x01 \x01(\x05H\x00R\x05scale\x88\x01\x01\x12!\n\tprecision\x18\x02 \x01(\x05H\x01R\tprecision\x88\x01\x01\x12\x38\n\x18type_variation_reference\x18\x03 \x01(\rR\x16typeVariationReferenceB\x08\n\x06_scaleB\x0c\n\n_precision\x1a\xa1\x01\n\x0bStructField\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x34\n\tdata_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x08\x64\x61taType\x12\x1a\n\x08nullable\x18\x03 \x01(\x08R\x08nullable\x12\x1f\n\x08metadata\x18\x04 \x01(\tH\x00R\x08metadata\x88\x01\x01\x42\x0b\n\t_metadata\x1a\x7f\n\x06Struct\x12;\n\x06\x66ields\x18\x01 \x03(\x0b\x32#.spark.connect.DataType.StructFieldR\x06\x66ields\x12\x38\n\x18type_variation_reference\x18\x02 \x01(\rR\x16typeVariationReference\x1a\xa2\x01\n\x05\x41rray\x12:\n\x0c\x65lement_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x0b\x65lementType\x12#\n\rcontains_null\x18\x02 \x01(\x08R\x0c\x63ontainsNull\x12\x38\n\x18type_variation_reference\x18\x03 \x01(\rR\x16typeVariationReference\x1a\xdb\x01\n\x03Map\x12\x32\n\x08key_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x07keyType\x12\x36\n\nvalue_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\tvalueType\x12.\n\x13value_contains_null\x18\x03 \x01(\x08R\x11valueContainsNull\x12\x38\n\x18type_variation_reference\x18\x04 \x01(\rR\x16typeVariationReference\x1a\x43\n\x07Variant\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\xa1\x02\n\x03UDT\x12\x12\n\x04type\x18\x01 \x01(\tR\x04type\x12 \n\tjvm_class\x18\x02 \x01(\tH\x00R\x08jvmClass\x88\x01\x01\x12&\n\x0cpython_class\x18\x03 \x01(\tH\x01R\x0bpythonClass\x88\x01\x01\x12;\n\x17serialized_python_class\x18\x04 \x01(\tH\x02R\x15serializedPythonClass\x88\x01\x01\x12\x37\n\x08sql_type\x18\x05 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x03R\x07sqlType\x88\x01\x01\x42\x0c\n\n_jvm_classB\x0f\n\r_python_classB\x1a\n\x18_serialized_python_classB\x0b\n\t_sql_type\x1a\x34\n\x08Unparsed\x12(\n\x10\x64\x61ta_type_string\x18\x01 \x01(\tR\x0e\x64\x61taTypeStringB\x06\n\x04kindB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3"
+ b"\n\x19spark/connect/types.proto\x12\rspark.connect\"\xac#\n\x08\x44\x61taType\x12\x32\n\x04null\x18\x01 \x01(\x0b\x32\x1c.spark.connect.DataType.NULLH\x00R\x04null\x12\x38\n\x06\x62inary\x18\x02 \x01(\x0b\x32\x1e.spark.connect.DataType.BinaryH\x00R\x06\x62inary\x12;\n\x07\x62oolean\x18\x03 \x01(\x0b\x32\x1f.spark.connect.DataType.BooleanH\x00R\x07\x62oolean\x12\x32\n\x04\x62yte\x18\x04 \x01(\x0b\x32\x1c.spark.connect.DataType.ByteH\x00R\x04\x62yte\x12\x35\n\x05short\x18\x05 \x01(\x0b\x32\x1d.spark.connect.DataType.ShortH\x00R\x05short\x12;\n\x07integer\x18\x06 \x01(\x0b\x32\x1f.spark.connect.DataType.IntegerH\x00R\x07integer\x12\x32\n\x04long\x18\x07 \x01(\x0b\x32\x1c.spark.connect.DataType.LongH\x00R\x04long\x12\x35\n\x05\x66loat\x18\x08 \x01(\x0b\x32\x1d.spark.connect.DataType.FloatH\x00R\x05\x66loat\x12\x38\n\x06\x64ouble\x18\t \x01(\x0b\x32\x1e.spark.connect.DataType.DoubleH\x00R\x06\x64ouble\x12;\n\x07\x64\x65\x63imal\x18\n \x01(\x0b\x32\x1f.spark.connect.DataType.DecimalH\x00R\x07\x64\x65\x63imal\x12\x38\n\x06string\x18\x0b \x01(\x0b\x32\x1e.spark.connect.DataType.StringH\x00R\x06string\x12\x32\n\x04\x63har\x18\x0c \x01(\x0b\x32\x1c.spark.connect.DataType.CharH\x00R\x04\x63har\x12<\n\x08var_char\x18\r \x01(\x0b\x32\x1f.spark.connect.DataType.VarCharH\x00R\x07varChar\x12\x32\n\x04\x64\x61te\x18\x0e \x01(\x0b\x32\x1c.spark.connect.DataType.DateH\x00R\x04\x64\x61te\x12\x41\n\ttimestamp\x18\x0f \x01(\x0b\x32!.spark.connect.DataType.TimestampH\x00R\ttimestamp\x12K\n\rtimestamp_ntz\x18\x10 \x01(\x0b\x32$.spark.connect.DataType.TimestampNTZH\x00R\x0ctimestampNtz\x12W\n\x11\x63\x61lendar_interval\x18\x11 \x01(\x0b\x32(.spark.connect.DataType.CalendarIntervalH\x00R\x10\x63\x61lendarInterval\x12[\n\x13year_month_interval\x18\x12 \x01(\x0b\x32).spark.connect.DataType.YearMonthIntervalH\x00R\x11yearMonthInterval\x12U\n\x11\x64\x61y_time_interval\x18\x13 \x01(\x0b\x32'.spark.connect.DataType.DayTimeIntervalH\x00R\x0f\x64\x61yTimeInterval\x12\x35\n\x05\x61rray\x18\x14 \x01(\x0b\x32\x1d.spark.connect.DataType.ArrayH\x00R\x05\x61rray\x12\x38\n\x06struct\x18\x15 \x01(\x0b\x32\x1e.spark.connect.DataType.StructH\x00R\x06struct\x12/\n\x03map\x18\x16 \x01(\x0b\x32\x1b.spark.connect.DataType.MapH\x00R\x03map\x12;\n\x07variant\x18\x19 \x01(\x0b\x32\x1f.spark.connect.DataType.VariantH\x00R\x07variant\x12/\n\x03udt\x18\x17 \x01(\x0b\x32\x1b.spark.connect.DataType.UDTH\x00R\x03udt\x12>\n\x08unparsed\x18\x18 \x01(\x0b\x32 .spark.connect.DataType.UnparsedH\x00R\x08unparsed\x12\x32\n\x04time\x18\x1c \x01(\x0b\x32\x1c.spark.connect.DataType.TimeH\x00R\x04time\x1a\x43\n\x07\x42oolean\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a@\n\x04\x42yte\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\x41\n\x05Short\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\x43\n\x07Integer\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a@\n\x04Long\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\x41\n\x05\x46loat\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\x42\n\x06\x44ouble\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a`\n\x06String\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x12\x1c\n\tcollation\x18\x02 \x01(\tR\tcollation\x1a\x42\n\x06\x42inary\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a@\n\x04NULL\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\x45\n\tTimestamp\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a@\n\x04\x44\x61te\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1aH\n\x0cTimestampNTZ\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1aq\n\x04Time\x12!\n\tprecision\x18\x01 \x01(\x05H\x00R\tprecision\x88\x01\x01\x12\x38\n\x18type_variation_reference\x18\x02 \x01(\rR\x16typeVariationReferenceB\x0c\n\n_precision\x1aL\n\x10\x43\x61lendarInterval\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\xb3\x01\n\x11YearMonthInterval\x12$\n\x0bstart_field\x18\x01 \x01(\x05H\x00R\nstartField\x88\x01\x01\x12 \n\tend_field\x18\x02 \x01(\x05H\x01R\x08\x65ndField\x88\x01\x01\x12\x38\n\x18type_variation_reference\x18\x03 \x01(\rR\x16typeVariationReferenceB\x0e\n\x0c_start_fieldB\x0c\n\n_end_field\x1a\xb1\x01\n\x0f\x44\x61yTimeInterval\x12$\n\x0bstart_field\x18\x01 \x01(\x05H\x00R\nstartField\x88\x01\x01\x12 \n\tend_field\x18\x02 \x01(\x05H\x01R\x08\x65ndField\x88\x01\x01\x12\x38\n\x18type_variation_reference\x18\x03 \x01(\rR\x16typeVariationReferenceB\x0e\n\x0c_start_fieldB\x0c\n\n_end_field\x1aX\n\x04\x43har\x12\x16\n\x06length\x18\x01 \x01(\x05R\x06length\x12\x38\n\x18type_variation_reference\x18\x02 \x01(\rR\x16typeVariationReference\x1a[\n\x07VarChar\x12\x16\n\x06length\x18\x01 \x01(\x05R\x06length\x12\x38\n\x18type_variation_reference\x18\x02 \x01(\rR\x16typeVariationReference\x1a\x99\x01\n\x07\x44\x65\x63imal\x12\x19\n\x05scale\x18\x01 \x01(\x05H\x00R\x05scale\x88\x01\x01\x12!\n\tprecision\x18\x02 \x01(\x05H\x01R\tprecision\x88\x01\x01\x12\x38\n\x18type_variation_reference\x18\x03 \x01(\rR\x16typeVariationReferenceB\x08\n\x06_scaleB\x0c\n\n_precision\x1a\xa1\x01\n\x0bStructField\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x34\n\tdata_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x08\x64\x61taType\x12\x1a\n\x08nullable\x18\x03 \x01(\x08R\x08nullable\x12\x1f\n\x08metadata\x18\x04 \x01(\tH\x00R\x08metadata\x88\x01\x01\x42\x0b\n\t_metadata\x1a\x7f\n\x06Struct\x12;\n\x06\x66ields\x18\x01 \x03(\x0b\x32#.spark.connect.DataType.StructFieldR\x06\x66ields\x12\x38\n\x18type_variation_reference\x18\x02 \x01(\rR\x16typeVariationReference\x1a\xa2\x01\n\x05\x41rray\x12:\n\x0c\x65lement_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x0b\x65lementType\x12#\n\rcontains_null\x18\x02 \x01(\x08R\x0c\x63ontainsNull\x12\x38\n\x18type_variation_reference\x18\x03 \x01(\rR\x16typeVariationReference\x1a\xdb\x01\n\x03Map\x12\x32\n\x08key_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x07keyType\x12\x36\n\nvalue_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\tvalueType\x12.\n\x13value_contains_null\x18\x03 \x01(\x08R\x11valueContainsNull\x12\x38\n\x18type_variation_reference\x18\x04 \x01(\rR\x16typeVariationReference\x1a\x43\n\x07Variant\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\xa1\x02\n\x03UDT\x12\x12\n\x04type\x18\x01 \x01(\tR\x04type\x12 \n\tjvm_class\x18\x02 \x01(\tH\x00R\x08jvmClass\x88\x01\x01\x12&\n\x0cpython_class\x18\x03 \x01(\tH\x01R\x0bpythonClass\x88\x01\x01\x12;\n\x17serialized_python_class\x18\x04 \x01(\tH\x02R\x15serializedPythonClass\x88\x01\x01\x12\x37\n\x08sql_type\x18\x05 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x03R\x07sqlType\x88\x01\x01\x42\x0c\n\n_jvm_classB\x0f\n\r_python_classB\x1a\n\x18_serialized_python_classB\x0b\n\t_sql_type\x1a\x34\n\x08Unparsed\x12(\n\x10\x64\x61ta_type_string\x18\x01 \x01(\tR\x0e\x64\x61taTypeStringB\x06\n\x04kindJ\x04\x08\x1a\x10\x1bJ\x04\x08\x1b\x10\x1c\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3"
)
_globals = globals()
@@ -47,57 +47,59 @@
"DESCRIPTOR"
]._serialized_options = b"\n\036org.apache.spark.connect.protoP\001Z\022internal/generated"
_globals["_DATATYPE"]._serialized_start = 45
- _globals["_DATATYPE"]._serialized_end = 4390
- _globals["_DATATYPE_BOOLEAN"]._serialized_start = 1595
- _globals["_DATATYPE_BOOLEAN"]._serialized_end = 1662
- _globals["_DATATYPE_BYTE"]._serialized_start = 1664
- _globals["_DATATYPE_BYTE"]._serialized_end = 1728
- _globals["_DATATYPE_SHORT"]._serialized_start = 1730
- _globals["_DATATYPE_SHORT"]._serialized_end = 1795
- _globals["_DATATYPE_INTEGER"]._serialized_start = 1797
- _globals["_DATATYPE_INTEGER"]._serialized_end = 1864
- _globals["_DATATYPE_LONG"]._serialized_start = 1866
- _globals["_DATATYPE_LONG"]._serialized_end = 1930
- _globals["_DATATYPE_FLOAT"]._serialized_start = 1932
- _globals["_DATATYPE_FLOAT"]._serialized_end = 1997
- _globals["_DATATYPE_DOUBLE"]._serialized_start = 1999
- _globals["_DATATYPE_DOUBLE"]._serialized_end = 2065
- _globals["_DATATYPE_STRING"]._serialized_start = 2067
- _globals["_DATATYPE_STRING"]._serialized_end = 2163
- _globals["_DATATYPE_BINARY"]._serialized_start = 2165
- _globals["_DATATYPE_BINARY"]._serialized_end = 2231
- _globals["_DATATYPE_NULL"]._serialized_start = 2233
- _globals["_DATATYPE_NULL"]._serialized_end = 2297
- _globals["_DATATYPE_TIMESTAMP"]._serialized_start = 2299
- _globals["_DATATYPE_TIMESTAMP"]._serialized_end = 2368
- _globals["_DATATYPE_DATE"]._serialized_start = 2370
- _globals["_DATATYPE_DATE"]._serialized_end = 2434
- _globals["_DATATYPE_TIMESTAMPNTZ"]._serialized_start = 2436
- _globals["_DATATYPE_TIMESTAMPNTZ"]._serialized_end = 2508
- _globals["_DATATYPE_CALENDARINTERVAL"]._serialized_start = 2510
- _globals["_DATATYPE_CALENDARINTERVAL"]._serialized_end = 2586
- _globals["_DATATYPE_YEARMONTHINTERVAL"]._serialized_start = 2589
- _globals["_DATATYPE_YEARMONTHINTERVAL"]._serialized_end = 2768
- _globals["_DATATYPE_DAYTIMEINTERVAL"]._serialized_start = 2771
- _globals["_DATATYPE_DAYTIMEINTERVAL"]._serialized_end = 2948
- _globals["_DATATYPE_CHAR"]._serialized_start = 2950
- _globals["_DATATYPE_CHAR"]._serialized_end = 3038
- _globals["_DATATYPE_VARCHAR"]._serialized_start = 3040
- _globals["_DATATYPE_VARCHAR"]._serialized_end = 3131
- _globals["_DATATYPE_DECIMAL"]._serialized_start = 3134
- _globals["_DATATYPE_DECIMAL"]._serialized_end = 3287
- _globals["_DATATYPE_STRUCTFIELD"]._serialized_start = 3290
- _globals["_DATATYPE_STRUCTFIELD"]._serialized_end = 3451
- _globals["_DATATYPE_STRUCT"]._serialized_start = 3453
- _globals["_DATATYPE_STRUCT"]._serialized_end = 3580
- _globals["_DATATYPE_ARRAY"]._serialized_start = 3583
- _globals["_DATATYPE_ARRAY"]._serialized_end = 3745
- _globals["_DATATYPE_MAP"]._serialized_start = 3748
- _globals["_DATATYPE_MAP"]._serialized_end = 3967
- _globals["_DATATYPE_VARIANT"]._serialized_start = 3969
- _globals["_DATATYPE_VARIANT"]._serialized_end = 4036
- _globals["_DATATYPE_UDT"]._serialized_start = 4039
- _globals["_DATATYPE_UDT"]._serialized_end = 4328
- _globals["_DATATYPE_UNPARSED"]._serialized_start = 4330
- _globals["_DATATYPE_UNPARSED"]._serialized_end = 4382
+ _globals["_DATATYPE"]._serialized_end = 4569
+ _globals["_DATATYPE_BOOLEAN"]._serialized_start = 1647
+ _globals["_DATATYPE_BOOLEAN"]._serialized_end = 1714
+ _globals["_DATATYPE_BYTE"]._serialized_start = 1716
+ _globals["_DATATYPE_BYTE"]._serialized_end = 1780
+ _globals["_DATATYPE_SHORT"]._serialized_start = 1782
+ _globals["_DATATYPE_SHORT"]._serialized_end = 1847
+ _globals["_DATATYPE_INTEGER"]._serialized_start = 1849
+ _globals["_DATATYPE_INTEGER"]._serialized_end = 1916
+ _globals["_DATATYPE_LONG"]._serialized_start = 1918
+ _globals["_DATATYPE_LONG"]._serialized_end = 1982
+ _globals["_DATATYPE_FLOAT"]._serialized_start = 1984
+ _globals["_DATATYPE_FLOAT"]._serialized_end = 2049
+ _globals["_DATATYPE_DOUBLE"]._serialized_start = 2051
+ _globals["_DATATYPE_DOUBLE"]._serialized_end = 2117
+ _globals["_DATATYPE_STRING"]._serialized_start = 2119
+ _globals["_DATATYPE_STRING"]._serialized_end = 2215
+ _globals["_DATATYPE_BINARY"]._serialized_start = 2217
+ _globals["_DATATYPE_BINARY"]._serialized_end = 2283
+ _globals["_DATATYPE_NULL"]._serialized_start = 2285
+ _globals["_DATATYPE_NULL"]._serialized_end = 2349
+ _globals["_DATATYPE_TIMESTAMP"]._serialized_start = 2351
+ _globals["_DATATYPE_TIMESTAMP"]._serialized_end = 2420
+ _globals["_DATATYPE_DATE"]._serialized_start = 2422
+ _globals["_DATATYPE_DATE"]._serialized_end = 2486
+ _globals["_DATATYPE_TIMESTAMPNTZ"]._serialized_start = 2488
+ _globals["_DATATYPE_TIMESTAMPNTZ"]._serialized_end = 2560
+ _globals["_DATATYPE_TIME"]._serialized_start = 2562
+ _globals["_DATATYPE_TIME"]._serialized_end = 2675
+ _globals["_DATATYPE_CALENDARINTERVAL"]._serialized_start = 2677
+ _globals["_DATATYPE_CALENDARINTERVAL"]._serialized_end = 2753
+ _globals["_DATATYPE_YEARMONTHINTERVAL"]._serialized_start = 2756
+ _globals["_DATATYPE_YEARMONTHINTERVAL"]._serialized_end = 2935
+ _globals["_DATATYPE_DAYTIMEINTERVAL"]._serialized_start = 2938
+ _globals["_DATATYPE_DAYTIMEINTERVAL"]._serialized_end = 3115
+ _globals["_DATATYPE_CHAR"]._serialized_start = 3117
+ _globals["_DATATYPE_CHAR"]._serialized_end = 3205
+ _globals["_DATATYPE_VARCHAR"]._serialized_start = 3207
+ _globals["_DATATYPE_VARCHAR"]._serialized_end = 3298
+ _globals["_DATATYPE_DECIMAL"]._serialized_start = 3301
+ _globals["_DATATYPE_DECIMAL"]._serialized_end = 3454
+ _globals["_DATATYPE_STRUCTFIELD"]._serialized_start = 3457
+ _globals["_DATATYPE_STRUCTFIELD"]._serialized_end = 3618
+ _globals["_DATATYPE_STRUCT"]._serialized_start = 3620
+ _globals["_DATATYPE_STRUCT"]._serialized_end = 3747
+ _globals["_DATATYPE_ARRAY"]._serialized_start = 3750
+ _globals["_DATATYPE_ARRAY"]._serialized_end = 3912
+ _globals["_DATATYPE_MAP"]._serialized_start = 3915
+ _globals["_DATATYPE_MAP"]._serialized_end = 4134
+ _globals["_DATATYPE_VARIANT"]._serialized_start = 4136
+ _globals["_DATATYPE_VARIANT"]._serialized_end = 4203
+ _globals["_DATATYPE_UDT"]._serialized_start = 4206
+ _globals["_DATATYPE_UDT"]._serialized_end = 4495
+ _globals["_DATATYPE_UNPARSED"]._serialized_start = 4497
+ _globals["_DATATYPE_UNPARSED"]._serialized_end = 4549
# @@protoc_insertion_point(module_scope)
diff --git a/python/pyspark/sql/connect/proto/types_pb2.pyi b/python/pyspark/sql/connect/proto/types_pb2.pyi
index fcf35b8c1f196..d46770c4f888e 100644
--- a/python/pyspark/sql/connect/proto/types_pb2.pyi
+++ b/python/pyspark/sql/connect/proto/types_pb2.pyi
@@ -279,6 +279,40 @@ class DataType(google.protobuf.message.Message):
],
) -> None: ...
+ class Time(google.protobuf.message.Message):
+ DESCRIPTOR: google.protobuf.descriptor.Descriptor
+
+ PRECISION_FIELD_NUMBER: builtins.int
+ TYPE_VARIATION_REFERENCE_FIELD_NUMBER: builtins.int
+ precision: builtins.int
+ type_variation_reference: builtins.int
+ def __init__(
+ self,
+ *,
+ precision: builtins.int | None = ...,
+ type_variation_reference: builtins.int = ...,
+ ) -> None: ...
+ def HasField(
+ self,
+ field_name: typing_extensions.Literal[
+ "_precision", b"_precision", "precision", b"precision"
+ ],
+ ) -> builtins.bool: ...
+ def ClearField(
+ self,
+ field_name: typing_extensions.Literal[
+ "_precision",
+ b"_precision",
+ "precision",
+ b"precision",
+ "type_variation_reference",
+ b"type_variation_reference",
+ ],
+ ) -> None: ...
+ def WhichOneof(
+ self, oneof_group: typing_extensions.Literal["_precision", b"_precision"]
+ ) -> typing_extensions.Literal["precision"] | None: ...
+
class CalendarInterval(google.protobuf.message.Message):
DESCRIPTOR: google.protobuf.descriptor.Descriptor
@@ -788,6 +822,7 @@ class DataType(google.protobuf.message.Message):
VARIANT_FIELD_NUMBER: builtins.int
UDT_FIELD_NUMBER: builtins.int
UNPARSED_FIELD_NUMBER: builtins.int
+ TIME_FIELD_NUMBER: builtins.int
@property
def null(self) -> global___DataType.NULL: ...
@property
@@ -845,6 +880,8 @@ class DataType(google.protobuf.message.Message):
@property
def unparsed(self) -> global___DataType.Unparsed:
"""UnparsedDataType"""
+ @property
+ def time(self) -> global___DataType.Time: ...
def __init__(
self,
*,
@@ -873,6 +910,7 @@ class DataType(google.protobuf.message.Message):
variant: global___DataType.Variant | None = ...,
udt: global___DataType.UDT | None = ...,
unparsed: global___DataType.Unparsed | None = ...,
+ time: global___DataType.Time | None = ...,
) -> None: ...
def HasField(
self,
@@ -915,6 +953,8 @@ class DataType(google.protobuf.message.Message):
b"string",
"struct",
b"struct",
+ "time",
+ b"time",
"timestamp",
b"timestamp",
"timestamp_ntz",
@@ -972,6 +1012,8 @@ class DataType(google.protobuf.message.Message):
b"string",
"struct",
b"struct",
+ "time",
+ b"time",
"timestamp",
b"timestamp",
"timestamp_ntz",
@@ -1017,6 +1059,7 @@ class DataType(google.protobuf.message.Message):
"variant",
"udt",
"unparsed",
+ "time",
]
| None
): ...
diff --git a/python/pyspark/sql/connect/session.py b/python/pyspark/sql/connect/session.py
index 303b9c9aac127..6ccffc718d064 100644
--- a/python/pyspark/sql/connect/session.py
+++ b/python/pyspark/sql/connect/session.py
@@ -23,7 +23,7 @@
import threading
import os
import warnings
-from collections.abc import Sized
+from collections.abc import Callable, Sized
import functools
from threading import RLock
from typing import (
@@ -106,6 +106,7 @@
)
if TYPE_CHECKING:
+ import pyspark.sql.connect.proto as pb2
from pyspark.sql.connect._typing import OptionalPrimitiveType
from pyspark.sql.connect.catalog import Catalog
from pyspark.sql.connect.udf import UDFRegistration
@@ -130,6 +131,7 @@ class Builder:
def __init__(self) -> None:
self._options: Dict[str, Any] = {}
self._channel_builder: Optional[DefaultChannelBuilder] = None
+ self._hook_factories: list["Callable[[SparkSession], SparkSession.Hook]"] = []
@overload
def config(self, key: str, value: Any) -> "SparkSession.Builder":
@@ -191,6 +193,13 @@ class CustomChannelBuilder(ChannelBuilder):
self._channel_builder = channelBuilder
return self
+ def _registerHook(
+ self, hook_factory: "Callable[[SparkSession], SparkSession.Hook]"
+ ) -> "SparkSession.Builder":
+ with self._lock:
+ self._hook_factories.append(hook_factory)
+ return self
+
def enableHiveSupport(self) -> "SparkSession.Builder":
raise PySparkNotImplementedError(
errorClass="NOT_IMPLEMENTED", messageParameters={"feature": "enableHiveSupport"}
@@ -235,11 +244,13 @@ def create(self) -> "SparkSession":
if has_channel_builder:
assert self._channel_builder is not None
- session = SparkSession(connection=self._channel_builder)
+ session = SparkSession(
+ connection=self._channel_builder, hook_factories=self._hook_factories
+ )
else:
spark_remote = to_str(self._options.get("spark.remote"))
assert spark_remote is not None
- session = SparkSession(connection=spark_remote)
+ session = SparkSession(connection=spark_remote, hook_factories=self._hook_factories)
SparkSession._set_default_and_active_session(session)
self._apply_options(session)
@@ -255,6 +266,16 @@ def getOrCreate(self) -> "SparkSession":
self._apply_options(session)
return session
+ class Hook:
+ """A Hook can be used to inject behavior into the session."""
+
+ def on_execute_plan(self, request: "pb2.ExecutePlanRequest") -> "pb2.ExecutePlanRequest":
+ """Called before sending an ExecutePlanRequest.
+
+ The request is replaced with the one returned by this method.
+ """
+ return request
+
_client: SparkConnectClient
# SPARK-47544: Explicitly declaring this as an identifier instead of a method.
@@ -262,7 +283,12 @@ def getOrCreate(self) -> "SparkSession":
builder: Builder = classproperty(lambda cls: cls.Builder()) # type: ignore
builder.__doc__ = PySparkSession.builder.__doc__
- def __init__(self, connection: Union[str, DefaultChannelBuilder], userId: Optional[str] = None):
+ def __init__(
+ self,
+ connection: Union[str, DefaultChannelBuilder],
+ userId: Optional[str] = None,
+ hook_factories: Optional[list["Callable[[SparkSession], Hook]"]] = None,
+ ) -> None:
"""
Creates a new SparkSession for the Spark Connect interface.
@@ -277,8 +303,15 @@ def __init__(self, connection: Union[str, DefaultChannelBuilder], userId: Option
isolate their Spark Sessions. If the `user_id` is not set, will default to
the $USER environment. Defining the user ID as part of the connection string
takes precedence.
+ hook_factories: list[Callable[[SparkSession], Hook]], optional
+ Optional list of hook factories for hooks that should be registered for this session.
"""
- self._client = SparkConnectClient(connection=connection, user_id=userId)
+ hook_factories = hook_factories or []
+ self._client = SparkConnectClient(
+ connection=connection,
+ user_id=userId,
+ session_hooks=[factory(self) for factory in hook_factories],
+ )
self._session_id = self._client._session_id
# Set to false to prevent client.release_session on close() (testing only)
@@ -586,7 +619,7 @@ def createDataFrame(
safecheck = configs["spark.sql.execution.pandas.convertToArrowArraySafely"]
- ser = ArrowStreamPandasSerializer(cast(str, timezone), safecheck == "true")
+ ser = ArrowStreamPandasSerializer(cast(str, timezone), safecheck == "true", False)
_table = pa.Table.from_batches(
[
diff --git a/python/pyspark/sql/connect/tvf.py b/python/pyspark/sql/connect/tvf.py
index cf94fdb64915f..59a4e4fbe344e 100644
--- a/python/pyspark/sql/connect/tvf.py
+++ b/python/pyspark/sql/connect/tvf.py
@@ -127,7 +127,7 @@ def _test() -> None:
print("Not supported in no-GIL mode", file=sys.stderr)
sys.exit(0)
- from pyspark.testing import should_test_connect
+ from pyspark.testing.utils import should_test_connect
if not should_test_connect:
print("Skipping pyspark.sql.connect.tvf doctests", file=sys.stderr)
diff --git a/python/pyspark/sql/connect/types.py b/python/pyspark/sql/connect/types.py
index c2eb5f4e017f0..8f9e7c0561cc0 100644
--- a/python/pyspark/sql/connect/types.py
+++ b/python/pyspark/sql/connect/types.py
@@ -29,6 +29,7 @@
IntegerType,
FloatType,
DateType,
+ TimeType,
TimestampType,
TimestampNTZType,
DayTimeIntervalType,
@@ -128,6 +129,10 @@ def pyspark_types_to_proto_types(data_type: DataType) -> pb2.DataType:
ret = pb2.DataType()
if isinstance(data_type, NullType):
ret.null.CopyFrom(pb2.DataType.NULL())
+ elif isinstance(data_type, CharType):
+ ret.char.length = data_type.length
+ elif isinstance(data_type, VarcharType):
+ ret.var_char.length = data_type.length
elif isinstance(data_type, StringType):
ret.string.collation = data_type.collation
elif isinstance(data_type, BooleanType):
@@ -151,6 +156,8 @@ def pyspark_types_to_proto_types(data_type: DataType) -> pb2.DataType:
ret.decimal.precision = data_type.precision
elif isinstance(data_type, DateType):
ret.date.CopyFrom(pb2.DataType.Date())
+ elif isinstance(data_type, TimeType):
+ ret.time.precision = data_type.precision
elif isinstance(data_type, TimestampType):
ret.timestamp.CopyFrom(pb2.DataType.Timestamp())
elif isinstance(data_type, TimestampNTZType):
@@ -237,6 +244,8 @@ def proto_schema_to_pyspark_data_type(schema: pb2.DataType) -> DataType:
return VarcharType(schema.var_char.length)
elif schema.HasField("date"):
return DateType()
+ elif schema.HasField("time"):
+ return TimeType(schema.time.precision) if schema.time.HasField("precision") else TimeType()
elif schema.HasField("timestamp"):
return TimestampType()
elif schema.HasField("timestamp_ntz"):
diff --git a/python/pyspark/sql/connect/udf.py b/python/pyspark/sql/connect/udf.py
index 7009a51695c5d..0f45690a9db33 100644
--- a/python/pyspark/sql/connect/udf.py
+++ b/python/pyspark/sql/connect/udf.py
@@ -162,7 +162,6 @@ def __init__(
def returnType(self) -> DataType:
# Make sure this is called after Connect Session is initialized.
# ``_parse_datatype_string`` accesses to Connect Server for parsing a DDL formatted string.
- # TODO: PythonEvalType.SQL_BATCHED_UDF
if self._returnType_placeholder is None:
if isinstance(self._returnType, DataType):
self._returnType_placeholder = self._returnType
@@ -280,13 +279,15 @@ def register(
PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF,
PythonEvalType.SQL_SCALAR_ARROW_ITER_UDF,
PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF,
+ PythonEvalType.SQL_GROUPED_AGG_ARROW_UDF,
]:
raise PySparkTypeError(
errorClass="INVALID_UDF_EVAL_TYPE",
messageParameters={
"eval_type": "SQL_BATCHED_UDF, SQL_ARROW_BATCHED_UDF, "
- "SQL_SCALAR_PANDAS_UDF, SQL_SCALAR_PANDAS_ITER_UDF or "
- "SQL_GROUPED_AGG_PANDAS_UDF"
+ "SQL_SCALAR_PANDAS_UDF, SQL_SCALAR_ARROW_UDF, "
+ "SQL_SCALAR_PANDAS_ITER_UDF, SQL_SCALAR_ARROW_ITER_UDF, "
+ "SQL_GROUPED_AGG_PANDAS_UDF or SQL_GROUPED_AGG_ARROW_UDF"
},
)
self.sparkSession._client.register_udf(
diff --git a/python/pyspark/sql/connect/udtf.py b/python/pyspark/sql/connect/udtf.py
index ed9ab26788f77..f049932071674 100644
--- a/python/pyspark/sql/connect/udtf.py
+++ b/python/pyspark/sql/connect/udtf.py
@@ -22,7 +22,7 @@
check_dependencies(__name__)
import warnings
-from typing import List, Type, TYPE_CHECKING, Optional, Union
+from typing import List, Type, TYPE_CHECKING, Optional, Union, Any
from pyspark.util import PythonEvalType
from pyspark.sql.connect.column import Column
@@ -34,10 +34,11 @@
from pyspark.sql.connect.table_arg import TableArg
from pyspark.sql.connect.types import UnparsedDataType
from pyspark.sql.connect.utils import get_python_ver
+from pyspark.sql.pandas.utils import require_minimum_pyarrow_version, require_minimum_pandas_version
from pyspark.sql.udtf import AnalyzeArgument, AnalyzeResult # noqa: F401
from pyspark.sql.udtf import UDTFRegistration as PySparkUDTFRegistration, _validate_udtf_handler
from pyspark.sql.types import DataType, StructType
-from pyspark.errors import PySparkRuntimeError, PySparkTypeError
+from pyspark.errors import PySparkRuntimeError, PySparkTypeError, PySparkAttributeError
if TYPE_CHECKING:
@@ -87,11 +88,6 @@ def _create_py_udtf(
eval_type: int = PythonEvalType.SQL_TABLE_UDF
if arrow_enabled:
- from pyspark.sql.pandas.utils import (
- require_minimum_pandas_version,
- require_minimum_pyarrow_version,
- )
-
try:
require_minimum_pandas_version()
require_minimum_pyarrow_version()
@@ -106,6 +102,43 @@ def _create_py_udtf(
return _create_udtf(cls, returnType, name, eval_type, deterministic)
+def _create_pyarrow_udtf(
+ cls: Type,
+ returnType: Optional[Union[StructType, str]],
+ name: Optional[str] = None,
+ deterministic: bool = False,
+) -> "UserDefinedTableFunction":
+ """Create a PyArrow-native Python UDTF."""
+ # Validate PyArrow dependencies
+ require_minimum_pyarrow_version()
+
+ # Validate the handler class with PyArrow-specific checks
+ _validate_arrow_udtf_handler(cls, returnType)
+
+ return _create_udtf(
+ cls=cls,
+ returnType=returnType,
+ name=name,
+ evalType=PythonEvalType.SQL_ARROW_UDTF,
+ deterministic=deterministic,
+ )
+
+
+def _validate_arrow_udtf_handler(cls: Any, returnType: Optional[Union[StructType, str]]) -> None:
+ """Validate the handler class of a PyArrow UDTF."""
+ # First run standard UDTF validation
+ _validate_udtf_handler(cls, returnType)
+
+ # Block analyze method usage in arrow UDTFs
+ # TODO(SPARK-53286): Support analyze method for Arrow UDTFs to enable dynamic return types
+ has_analyze = hasattr(cls, "analyze")
+ if has_analyze:
+ raise PySparkAttributeError(
+ errorClass="INVALID_ARROW_UDTF_WITH_ANALYZE",
+ messageParameters={"name": cls.__name__},
+ )
+
+
class UserDefinedTableFunction:
"""
User defined function in Python
@@ -203,12 +236,16 @@ def register(
},
)
- if f.evalType not in [PythonEvalType.SQL_TABLE_UDF, PythonEvalType.SQL_ARROW_TABLE_UDF]:
+ if f.evalType not in [
+ PythonEvalType.SQL_TABLE_UDF,
+ PythonEvalType.SQL_ARROW_TABLE_UDF,
+ PythonEvalType.SQL_ARROW_UDTF,
+ ]:
raise PySparkTypeError(
errorClass="INVALID_UDTF_EVAL_TYPE",
messageParameters={
"name": name,
- "eval_type": "SQL_TABLE_UDF, SQL_ARROW_TABLE_UDF",
+ "eval_type": "SQL_TABLE_UDF, SQL_ARROW_TABLE_UDF, SQL_ARROW_UDTF",
},
)
diff --git a/python/pyspark/sql/conversion.py b/python/pyspark/sql/conversion.py
index a0cd89da38224..f1aa55c2039ac 100644
--- a/python/pyspark/sql/conversion.py
+++ b/python/pyspark/sql/conversion.py
@@ -18,7 +18,7 @@
import array
import datetime
import decimal
-from typing import TYPE_CHECKING, Any, Callable, List, Sequence
+from typing import TYPE_CHECKING, Any, Callable, List, Optional, Sequence, Union, overload
from pyspark.errors import PySparkValueError
from pyspark.sql.pandas.types import _dedup_names, _deduplicate_field_names, to_arrow_schema
@@ -91,16 +91,40 @@ def _need_converter(
else:
return False
+ @overload
+ @staticmethod
+ def _create_converter(
+ dataType: DataType, nullable: bool = True, *, int_to_decimal_coercion_enabled: bool = False
+ ) -> Callable:
+ pass
+
+ @overload
@staticmethod
def _create_converter(
dataType: DataType,
nullable: bool = True,
- ) -> Callable:
+ *,
+ none_on_identity: bool = True,
+ int_to_decimal_coercion_enabled: bool = False,
+ ) -> Optional[Callable]:
+ pass
+
+ @staticmethod
+ def _create_converter(
+ dataType: DataType,
+ nullable: bool = True,
+ *,
+ none_on_identity: bool = False,
+ int_to_decimal_coercion_enabled: bool = False,
+ ) -> Optional[Callable]:
assert dataType is not None and isinstance(dataType, DataType)
assert isinstance(nullable, bool)
if not LocalDataToArrowConversion._need_converter(dataType, nullable):
- return lambda value: value
+ if none_on_identity:
+ return None
+ else:
+ return lambda value: value
if isinstance(dataType, NullType):
@@ -113,10 +137,16 @@ def convert_null(value: Any) -> Any:
elif isinstance(dataType, StructType):
field_names = dataType.fieldNames()
+ len_field_names = len(field_names)
dedup_field_names = _dedup_names(dataType.names)
field_convs = [
- LocalDataToArrowConversion._create_converter(field.dataType, field.nullable)
+ LocalDataToArrowConversion._create_converter(
+ field.dataType,
+ field.nullable,
+ none_on_identity=True,
+ int_to_decimal_coercion_enabled=int_to_decimal_coercion_enabled,
+ )
for field in dataType.fields
]
@@ -126,71 +156,113 @@ def convert_struct(value: Any) -> Any:
raise PySparkValueError(f"input for {dataType} must not be None")
return None
else:
- assert isinstance(value, (tuple, dict)) or hasattr(
- value, "__dict__"
- ), f"{type(value)} {value}"
-
- _dict = {}
- if (
- not isinstance(value, Row)
- and not isinstance(value, tuple) # inherited namedtuple
- and hasattr(value, "__dict__")
- ):
- value = value.__dict__
- if isinstance(value, dict):
- for i, field in enumerate(field_names):
- _dict[dedup_field_names[i]] = field_convs[i](value.get(field))
- else:
- if len(value) != len(field_names):
+ # The `value` should be tuple, dict, or have `__dict__`.
+ if isinstance(value, tuple): # `Row` inherits `tuple`
+ if len(value) != len_field_names:
raise PySparkValueError(
errorClass="AXIS_LENGTH_MISMATCH",
messageParameters={
- "expected_length": str(len(field_names)),
+ "expected_length": str(len_field_names),
"actual_length": str(len(value)),
},
)
- for i in range(len(field_names)):
- _dict[dedup_field_names[i]] = field_convs[i](value[i])
-
- return _dict
+ return {
+ dedup_field_names[i]: (
+ field_convs[i](value[i]) # type: ignore[misc]
+ if field_convs[i] is not None
+ else value[i]
+ )
+ for i in range(len_field_names)
+ }
+ elif isinstance(value, dict):
+ return {
+ dedup_field_names[i]: (
+ field_convs[i](value.get(field)) # type: ignore[misc]
+ if field_convs[i] is not None
+ else value.get(field)
+ )
+ for i, field in enumerate(field_names)
+ }
+ else:
+ assert hasattr(value, "__dict__"), f"{type(value)} {value}"
+ value = value.__dict__
+ return {
+ dedup_field_names[i]: (
+ field_convs[i](value.get(field)) # type: ignore[misc]
+ if field_convs[i] is not None
+ else value.get(field)
+ )
+ for i, field in enumerate(field_names)
+ }
return convert_struct
elif isinstance(dataType, ArrayType):
element_conv = LocalDataToArrowConversion._create_converter(
- dataType.elementType, dataType.containsNull
+ dataType.elementType,
+ dataType.containsNull,
+ none_on_identity=True,
+ int_to_decimal_coercion_enabled=int_to_decimal_coercion_enabled,
)
- def convert_array(value: Any) -> Any:
- if value is None:
- if not nullable:
- raise PySparkValueError(f"input for {dataType} must not be None")
- return None
- else:
- assert isinstance(value, (list, array.array))
- return [element_conv(v) for v in value]
+ if element_conv is None:
+
+ def convert_array(value: Any) -> Any:
+ if value is None:
+ if not nullable:
+ raise PySparkValueError(f"input for {dataType} must not be None")
+ return None
+ else:
+ assert isinstance(value, (list, array.array))
+ return list(value)
+
+ else:
+
+ def convert_array(value: Any) -> Any:
+ if value is None:
+ if not nullable:
+ raise PySparkValueError(f"input for {dataType} must not be None")
+ return None
+ else:
+ assert isinstance(value, (list, array.array))
+ return [element_conv(v) for v in value]
return convert_array
elif isinstance(dataType, MapType):
- key_conv = LocalDataToArrowConversion._create_converter(dataType.keyType)
+ key_conv = LocalDataToArrowConversion._create_converter(
+ dataType.keyType,
+ nullable=False,
+ int_to_decimal_coercion_enabled=int_to_decimal_coercion_enabled,
+ )
value_conv = LocalDataToArrowConversion._create_converter(
- dataType.valueType, dataType.valueContainsNull
+ dataType.valueType,
+ dataType.valueContainsNull,
+ none_on_identity=True,
+ int_to_decimal_coercion_enabled=int_to_decimal_coercion_enabled,
)
- def convert_map(value: Any) -> Any:
- if value is None:
- if not nullable:
- raise PySparkValueError(f"input for {dataType} must not be None")
- return None
- else:
- assert isinstance(value, dict)
+ if value_conv is None:
- _tuples = []
- for k, v in value.items():
- _tuples.append((key_conv(k), value_conv(v)))
+ def convert_map(value: Any) -> Any:
+ if value is None:
+ if not nullable:
+ raise PySparkValueError(f"input for {dataType} must not be None")
+ return None
+ else:
+ assert isinstance(value, dict)
+ return [(key_conv(k), v) for k, v in value.items()]
+
+ else:
- return _tuples
+ def convert_map(value: Any) -> Any:
+ if value is None:
+ if not nullable:
+ raise PySparkValueError(f"input for {dataType} must not be None")
+ return None
+ else:
+ assert isinstance(value, dict)
+ return [(key_conv(k), value_conv(v)) for k, v in value.items()]
return convert_map
@@ -241,8 +313,15 @@ def convert_decimal(value: Any) -> Any:
raise PySparkValueError(f"input for {dataType} must not be None")
return None
else:
+ if int_to_decimal_coercion_enabled and isinstance(value, int):
+ value = decimal.Decimal(value)
+
assert isinstance(value, decimal.Decimal)
- return None if value.is_nan() else value
+ if value.is_nan():
+ if not nullable:
+ raise PySparkValueError(f"input for {dataType} must not be None")
+ return None
+ return value
return convert_decimal
@@ -255,7 +334,7 @@ def convert_string(value: Any) -> Any:
return None
else:
if isinstance(value, bool):
- # To match the PySpark which convert bool to string in
+ # To match the PySpark Classic which convert bool to string in
# the JVM side (python.EvaluatePython.makeFromJava)
return str(value).lower()
else:
@@ -266,15 +345,32 @@ def convert_string(value: Any) -> Any:
elif isinstance(dataType, UserDefinedType):
udt: UserDefinedType = dataType
- conv = LocalDataToArrowConversion._create_converter(udt.sqlType())
+ conv = LocalDataToArrowConversion._create_converter(
+ udt.sqlType(),
+ nullable=nullable,
+ none_on_identity=True,
+ int_to_decimal_coercion_enabled=int_to_decimal_coercion_enabled,
+ )
- def convert_udt(value: Any) -> Any:
- if value is None:
- if not nullable:
- raise PySparkValueError(f"input for {dataType} must not be None")
- return None
- else:
- return conv(udt.serialize(value))
+ if conv is None:
+
+ def convert_udt(value: Any) -> Any:
+ if value is None:
+ if not nullable:
+ raise PySparkValueError(f"input for {dataType} must not be None")
+ return None
+ else:
+ return udt.serialize(value)
+
+ else:
+
+ def convert_udt(value: Any) -> Any:
+ if value is None:
+ if not nullable:
+ raise PySparkValueError(f"input for {dataType} must not be None")
+ return None
+ else:
+ return conv(udt.serialize(value))
return convert_udt
@@ -301,7 +397,10 @@ def convert_other(value: Any) -> Any:
return convert_other
else:
- return lambda value: value
+ if none_on_identity:
+ return None
+ else:
+ return lambda value: value
@staticmethod
def convert(data: Sequence[Any], schema: StructType, use_large_var_types: bool) -> "pa.Table":
@@ -313,55 +412,73 @@ def convert(data: Sequence[Any], schema: StructType, use_large_var_types: bool)
assert schema is not None and isinstance(schema, StructType)
column_names = schema.fieldNames()
+ len_column_names = len(column_names)
- column_convs = [
- LocalDataToArrowConversion._create_converter(field.dataType, field.nullable)
- for field in schema.fields
- ]
-
- pylist: List[List] = [[] for _ in range(len(column_names))]
-
- for item in data:
- if isinstance(item, VariantVal):
+ def to_row(item: Any) -> tuple:
+ if item is None:
+ return tuple([None] * len_column_names)
+ elif isinstance(item, tuple): # `Row` inherits `tuple`
+ if len(item) != len_column_names:
+ raise PySparkValueError(
+ errorClass="AXIS_LENGTH_MISMATCH",
+ messageParameters={
+ "expected_length": str(len_column_names),
+ "actual_length": str(len(item)),
+ },
+ )
+ return tuple(item)
+ elif isinstance(item, dict):
+ return tuple([item.get(col) for col in column_names])
+ elif isinstance(item, VariantVal):
raise PySparkValueError("Rows cannot be of type VariantVal")
- if (
- not isinstance(item, Row)
- and not isinstance(item, tuple) # inherited namedtuple
- and hasattr(item, "__dict__")
- ):
+ elif hasattr(item, "__dict__"):
item = item.__dict__
- if isinstance(item, dict):
- for i, col in enumerate(column_names):
- pylist[i].append(column_convs[i](item.get(col)))
- elif item is None:
- for i, col in enumerate(column_names):
- pylist[i].append(None)
+ return tuple([item.get(col) for col in column_names])
else:
- if len(item) != len(column_names):
+ if len(item) != len_column_names:
raise PySparkValueError(
errorClass="AXIS_LENGTH_MISMATCH",
messageParameters={
- "expected_length": str(len(column_names)),
+ "expected_length": str(len_column_names),
"actual_length": str(len(item)),
},
)
+ return tuple(item)
+
+ rows = [to_row(item) for item in data]
+
+ if len_column_names > 0:
+ column_convs = [
+ LocalDataToArrowConversion._create_converter(
+ field.dataType,
+ field.nullable,
+ none_on_identity=True,
+ # Default to False for general data conversion
+ int_to_decimal_coercion_enabled=False,
+ )
+ for field in schema.fields
+ ]
- for i in range(len(column_names)):
- pylist[i].append(column_convs[i](item[i]))
+ pylist = [
+ [conv(row[i]) for row in rows] if conv is not None else [row[i] for row in rows]
+ for i, conv in enumerate(column_convs)
+ ]
- pa_schema = to_arrow_schema(
- StructType(
- [
- StructField(
- field.name, _deduplicate_field_names(field.dataType), field.nullable
- )
- for field in schema.fields
- ]
- ),
- prefers_large_types=use_large_var_types,
- )
+ pa_schema = to_arrow_schema(
+ StructType(
+ [
+ StructField(
+ field.name, _deduplicate_field_names(field.dataType), field.nullable
+ )
+ for field in schema.fields
+ ]
+ ),
+ prefers_large_types=use_large_var_types,
+ )
- return pa.Table.from_arrays(pylist, schema=pa_schema)
+ return pa.Table.from_arrays(pylist, schema=pa_schema)
+ else:
+ return pa.Table.from_struct_array(pa.array([{}] * len(rows)))
class ArrowTableToRowsConversion:
@@ -393,12 +510,29 @@ def _need_converter(dataType: DataType) -> bool:
else:
return False
+ @overload
@staticmethod
def _create_converter(dataType: DataType) -> Callable:
+ pass
+
+ @overload
+ @staticmethod
+ def _create_converter(
+ dataType: DataType, *, none_on_identity: bool = True
+ ) -> Optional[Callable]:
+ pass
+
+ @staticmethod
+ def _create_converter(
+ dataType: DataType, *, none_on_identity: bool = False
+ ) -> Optional[Callable]:
assert dataType is not None and isinstance(dataType, DataType)
if not ArrowTableToRowsConversion._need_converter(dataType):
- return lambda value: value
+ if none_on_identity:
+ return None
+ else:
+ return lambda value: value
if isinstance(dataType, NullType):
return lambda value: None
@@ -408,7 +542,8 @@ def _create_converter(dataType: DataType) -> Callable:
dedup_field_names = _dedup_names(field_names)
field_convs = [
- ArrowTableToRowsConversion._create_converter(f.dataType) for f in dataType.fields
+ ArrowTableToRowsConversion._create_converter(f.dataType, none_on_identity=True)
+ for f in dataType.fields
]
def convert_struct(value: Any) -> Any:
@@ -418,7 +553,9 @@ def convert_struct(value: Any) -> Any:
assert isinstance(value, dict)
_values = [
- field_convs[i](value.get(name, None))
+ field_convs[i](value.get(name, None)) # type: ignore[misc]
+ if field_convs[i] is not None
+ else value.get(name, None)
for i, name in enumerate(dedup_field_names)
]
return _create_row(field_names, _values)
@@ -426,28 +563,79 @@ def convert_struct(value: Any) -> Any:
return convert_struct
elif isinstance(dataType, ArrayType):
- element_conv = ArrowTableToRowsConversion._create_converter(dataType.elementType)
+ element_conv = ArrowTableToRowsConversion._create_converter(
+ dataType.elementType, none_on_identity=True
+ )
- def convert_array(value: Any) -> Any:
- if value is None:
- return None
- else:
- assert isinstance(value, list)
- return [element_conv(v) for v in value]
+ if element_conv is None:
+
+ def convert_array(value: Any) -> Any:
+ if value is None:
+ return None
+ else:
+ assert isinstance(value, list)
+ return value
+
+ else:
+
+ def convert_array(value: Any) -> Any:
+ if value is None:
+ return None
+ else:
+ assert isinstance(value, list)
+ return [element_conv(v) for v in value]
return convert_array
elif isinstance(dataType, MapType):
- key_conv = ArrowTableToRowsConversion._create_converter(dataType.keyType)
- value_conv = ArrowTableToRowsConversion._create_converter(dataType.valueType)
+ key_conv = ArrowTableToRowsConversion._create_converter(
+ dataType.keyType, none_on_identity=True
+ )
+ value_conv = ArrowTableToRowsConversion._create_converter(
+ dataType.valueType, none_on_identity=True
+ )
+
+ if key_conv is None:
+ if value_conv is None:
+
+ def convert_map(value: Any) -> Any:
+ if value is None:
+ return None
+ else:
+ assert isinstance(value, list)
+ assert all(isinstance(t, tuple) and len(t) == 2 for t in value)
+ return dict(value)
+
+ else:
+
+ def convert_map(value: Any) -> Any:
+ if value is None:
+ return None
+ else:
+ assert isinstance(value, list)
+ assert all(isinstance(t, tuple) and len(t) == 2 for t in value)
+ return dict((t[0], value_conv(t[1])) for t in value)
+
+ else:
+ if value_conv is None:
+
+ def convert_map(value: Any) -> Any:
+ if value is None:
+ return None
+ else:
+ assert isinstance(value, list)
+ assert all(isinstance(t, tuple) and len(t) == 2 for t in value)
+ return dict((key_conv(t[0]), t[1]) for t in value)
- def convert_map(value: Any) -> Any:
- if value is None:
- return None
else:
- assert isinstance(value, list)
- assert all(isinstance(t, tuple) and len(t) == 2 for t in value)
- return dict((key_conv(t[0]), value_conv(t[1])) for t in value)
+
+ def convert_map(value: Any) -> Any:
+ if value is None:
+ return None
+ else:
+ assert isinstance(value, list)
+ assert all(isinstance(t, tuple) and len(t) == 2 for t in value)
+ return dict((key_conv(t[0]), value_conv(t[1])) for t in value)
return convert_map
@@ -487,13 +675,25 @@ def convert_timestample_ntz(value: Any) -> Any:
elif isinstance(dataType, UserDefinedType):
udt: UserDefinedType = dataType
- conv = ArrowTableToRowsConversion._create_converter(udt.sqlType())
+ conv = ArrowTableToRowsConversion._create_converter(
+ udt.sqlType(), none_on_identity=True
+ )
- def convert_udt(value: Any) -> Any:
- if value is None:
- return None
- else:
- return udt.deserialize(conv(value))
+ if conv is None:
+
+ def convert_udt(value: Any) -> Any:
+ if value is None:
+ return None
+ else:
+ return udt.deserialize(value)
+
+ else:
+
+ def convert_udt(value: Any) -> Any:
+ if value is None:
+ return None
+ else:
+ return udt.deserialize(conv(value))
return convert_udt
@@ -514,10 +714,29 @@ def convert_variant(value: Any) -> Any:
return convert_variant
else:
- return lambda value: value
+ if none_on_identity:
+ return None
+ else:
+ return lambda value: value
+ @overload
@staticmethod
- def convert(table: "pa.Table", schema: StructType) -> List[Row]:
+ def convert( # type: ignore[overload-overlap]
+ table: "pa.Table", schema: StructType
+ ) -> List[Row]:
+ pass
+
+ @overload
+ @staticmethod
+ def convert(
+ table: "pa.Table", schema: StructType, *, return_as_tuples: bool = True
+ ) -> List[tuple]:
+ pass
+
+ @staticmethod # type: ignore[misc]
+ def convert(
+ table: "pa.Table", schema: StructType, *, return_as_tuples: bool = False
+ ) -> List[Union[Row, tuple]]:
require_minimum_pyarrow_version()
import pyarrow as pa
@@ -525,14 +744,27 @@ def convert(table: "pa.Table", schema: StructType) -> List[Row]:
assert schema is not None and isinstance(schema, StructType)
- field_converters = [
- ArrowTableToRowsConversion._create_converter(f.dataType) for f in schema.fields
- ]
+ fields = schema.fieldNames()
+
+ if len(fields) > 0:
+ field_converters = [
+ ArrowTableToRowsConversion._create_converter(f.dataType, none_on_identity=True)
+ for f in schema.fields
+ ]
- columnar_data = [column.to_pylist() for column in table.columns]
+ columnar_data = [
+ [conv(v) for v in column.to_pylist()] if conv is not None else column.to_pylist()
+ for column, conv in zip(table.columns, field_converters)
+ ]
- rows: List[Row] = []
- for i in range(0, table.num_rows):
- values = [field_converters[j](columnar_data[j][i]) for j in range(table.num_columns)]
- rows.append(_create_row(fields=schema.fieldNames(), values=values))
- return rows
+ if return_as_tuples:
+ rows = [tuple(cols) for cols in zip(*columnar_data)]
+ else:
+ rows = [_create_row(fields, tuple(cols)) for cols in zip(*columnar_data)]
+ assert len(rows) == table.num_rows, f"{len(rows)}, {table.num_rows}"
+ return rows
+ else:
+ if return_as_tuples:
+ return [tuple()] * table.num_rows
+ else:
+ return [_create_row(fields, tuple())] * table.num_rows
diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py
index c00c3f484232b..675d972e3ef51 100644
--- a/python/pyspark/sql/dataframe.py
+++ b/python/pyspark/sql/dataframe.py
@@ -4182,7 +4182,10 @@ def unpivot(
| 2| 12| 1.2|
+---+---+------+
- >>> df.unpivot("id", ["int", "double"], "var", "val").show()
+ >>> from pyspark.sql import functions as sf
+ >>> df.unpivot(
+ ... "id", ["int", "double"], "var", "val"
+ ... ).sort("id", sf.desc("var")).show()
+---+------+----+
| id| var| val|
+---+------+----+
@@ -5234,7 +5237,7 @@ def replace(
|NULL| NULL|NULL|
+----+------+----+
- Example 4: Replace 10 to 20 in the 'name' column.
+ Example 4: Replace 10 to 18 in the 'age' column.
>>> df.na.replace(10, 18, 'age').show()
+----+------+-----+
diff --git a/python/pyspark/sql/datasource.py b/python/pyspark/sql/datasource.py
index c9704ff9f2599..f1908180a3ba5 100644
--- a/python/pyspark/sql/datasource.py
+++ b/python/pyspark/sql/datasource.py
@@ -43,13 +43,13 @@
"DataSource",
"DataSourceReader",
"DataSourceStreamReader",
- "SimpleDataSourceStreamReader",
"DataSourceWriter",
"DataSourceArrowWriter",
"DataSourceStreamWriter",
+ "DataSourceStreamArrowWriter",
+ "SimpleDataSourceStreamReader",
"DataSourceRegistration",
"InputPartition",
- "SimpleDataSourceStreamReader",
"WriterCommitMessage",
"Filter",
"EqualTo",
@@ -1098,6 +1098,59 @@ def abort(self, messages: List[Optional["WriterCommitMessage"]], batchId: int) -
...
+class DataSourceStreamArrowWriter(DataSourceStreamWriter):
+ """
+ A base class for data stream writers that process data using PyArrow's `RecordBatch`.
+
+ Unlike :class:`DataSourceStreamWriter`, which works with an iterator of Spark Rows, this class
+ is optimized for using the Arrow format when writing streaming data. It can offer better
+ performance when interfacing with systems or libraries that natively support Arrow for
+ streaming use cases.
+
+ .. versionadded: 4.1.0
+ """
+
+ @abstractmethod
+ def write(self, iterator: Iterator["RecordBatch"]) -> "WriterCommitMessage":
+ """
+ Writes an iterator of PyArrow `RecordBatch` objects to the streaming sink.
+
+ This method is called on executors to write data to the streaming data sink in
+ each microbatch. It accepts an iterator of PyArrow `RecordBatch` objects and
+ returns a single row representing a commit message, or None if there is no commit message.
+
+ The driver collects commit messages, if any, from all executors and passes them
+ to the :class:`DataSourceStreamArrowWriter.commit` method if all tasks run
+ successfully. If any task fails, the :class:`DataSourceStreamArrowWriter.abort` method
+ will be called with the collected commit messages.
+
+ Parameters
+ ----------
+ iterator : iterator of :class:`RecordBatch`\\s
+ An iterator of PyArrow `RecordBatch` objects representing the input data.
+
+ Returns
+ -------
+ :class:`WriterCommitMessage`
+ a serializable commit message
+
+ Examples
+ --------
+ >>> from dataclasses import dataclass
+ >>> @dataclass
+ ... class MyCommitMessage(WriterCommitMessage):
+ ... num_rows: int
+ ... batch_id: int
+ ...
+ >>> def write(self, iterator: Iterator["RecordBatch"]) -> "WriterCommitMessage":
+ ... total_rows = 0
+ ... for batch in iterator:
+ ... total_rows += len(batch)
+ ... return MyCommitMessage(num_rows=total_rows, batch_id=self.current_batch_id)
+ """
+ ...
+
+
class WriterCommitMessage:
"""
A commit message returned by the :meth:`DataSourceWriter.write` and will be
diff --git a/python/pyspark/sql/functions/__init__.py b/python/pyspark/sql/functions/__init__.py
index 8ab2ac377c2a8..0d1df20d13d6f 100644
--- a/python/pyspark/sql/functions/__init__.py
+++ b/python/pyspark/sql/functions/__init__.py
@@ -202,6 +202,7 @@
"convert_timezone",
"curdate",
"current_date",
+ "current_time",
"current_timestamp",
"current_timezone",
"date_add",
@@ -228,6 +229,7 @@
"make_date",
"make_dt_interval",
"make_interval",
+ "make_time",
"make_timestamp",
"make_timestamp_ltz",
"make_timestamp_ntz",
@@ -246,7 +248,9 @@
"timestamp_micros",
"timestamp_millis",
"timestamp_seconds",
+ "time_trunc",
"to_date",
+ "to_time",
"to_timestamp",
"to_timestamp_ltz",
"to_timestamp_ntz",
@@ -257,7 +261,9 @@
"try_make_timestamp",
"try_make_timestamp_ltz",
"try_make_timestamp_ntz",
+ "try_to_time",
"try_to_timestamp",
+ "try_to_date",
"unix_date",
"unix_micros",
"unix_millis",
@@ -498,6 +504,7 @@
# UDF, UDTF and UDT
"AnalyzeArgument",
"AnalyzeResult",
+ "ArrowUDFType",
"OrderingColumn",
"PandasUDFType",
"PartitioningColumn",
@@ -505,9 +512,11 @@
"SkipRestOfInputTableException",
"UserDefinedFunction",
"UserDefinedTableFunction",
+ "arrow_udf",
"call_udf",
"pandas_udf",
"udf",
"udtf",
+ "arrow_udtf",
"unwrap_udt",
]
diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py
index 37b65c3203da8..b09713e0c289e 100644
--- a/python/pyspark/sql/functions/builtin.py
+++ b/python/pyspark/sql/functions/builtin.py
@@ -57,10 +57,15 @@
from pyspark.sql.udtf import AnalyzeArgument, AnalyzeResult # noqa: F401
from pyspark.sql.udtf import OrderingColumn, PartitioningColumn, SelectedColumn # noqa: F401
from pyspark.sql.udtf import SkipRestOfInputTableException # noqa: F401
-from pyspark.sql.udtf import UserDefinedTableFunction, _create_py_udtf
+from pyspark.sql.udtf import UserDefinedTableFunction, _create_py_udtf, _create_pyarrow_udtf
# Keep pandas_udf and PandasUDFType import for backwards compatible import; moved in SPARK-28264
-from pyspark.sql.pandas.functions import pandas_udf, PandasUDFType # noqa: F401
+from pyspark.sql.pandas.functions import ( # noqa: F401
+ arrow_udf, # noqa: F401
+ pandas_udf, # noqa: F401
+ ArrowUDFType, # noqa: F401
+ PandasUDFType, # noqa: F401
+) # noqa: F401
from pyspark.sql.utils import (
to_str as _to_str,
@@ -9288,6 +9293,68 @@ def current_timezone() -> Column:
return _invoke_function("current_timezone")
+@overload
+def current_time() -> Column:
+ ...
+
+
+@overload
+def current_time(precision: int) -> Column:
+ ...
+
+
+@_try_remote_functions
+def current_time(precision: Optional[int] = None) -> Column:
+ """
+ Returns the current time at the start of query evaluation as a :class:`TimeType` column. All
+ calls of current_time within the same query return the same value.
+
+ .. versionadded:: 4.1.0
+
+ Parameters
+ ----------
+ precision: literal int, optional
+ number in the range [0..6], indicating how many fractional digits of seconds to include.
+ If omitted, the default is 6.
+
+ Returns
+ -------
+ :class:`~pyspark.sql.Column`
+ current time.
+
+ See Also
+ --------
+ :meth:`pyspark.sql.functions.current_date`
+ :meth:`pyspark.sql.functions.current_timestamp`
+
+ Examples
+ --------
+ Example 1: Current time with default precision
+
+ >>> from pyspark.sql import functions as sf
+ >>> spark.range(1).select(sf.current_time().alias("time")).show() # doctest: +SKIP
+ +---------------+
+ | time|
+ +---------------+
+ |16:57:04.304361|
+ +---------------+
+
+ Example 2: Current time with specified precision
+
+ >>> from pyspark.sql import functions as sf
+ >>> spark.range(1).select(sf.current_time(3).alias("time")).show() # doctest: +SKIP
+ +------------+
+ | time|
+ +------------+
+ |16:57:04.304|
+ +------------+
+ """
+ if precision is None:
+ return _invoke_function("current_time")
+ else:
+ return _invoke_function("current_time", _enum_to_value(precision))
+
+
@_try_remote_functions
def current_timestamp() -> Column:
"""
@@ -10127,10 +10194,13 @@ def hour(col: "ColumnOrName") -> Column:
.. versionchanged:: 3.4.0
Supports Spark Connect.
+ .. versionchanged:: 4.1.0
+ Added support for time type.
+
Parameters
----------
col : :class:`~pyspark.sql.Column` or column name
- target date/timestamp column to work on.
+ target date/time/timestamp column to work on.
Returns
-------
@@ -10177,6 +10247,21 @@ def hour(col: "ColumnOrName") -> Column:
|2015-04-08 13:08:15| timestamp| 13|
|2024-10-31 10:09:16| timestamp| 10|
+-------------------+----------+--------+
+
+ Example 3: Extract the hours from a time column
+
+ >>> import datetime
+ >>> from pyspark.sql import functions as sf
+ >>> df = spark.createDataFrame([
+ ... ("13:08:15",),
+ ... ("10:09:16",)], ['t']).withColumn("t", sf.col("t").cast("time"))
+ >>> df.select("*", sf.typeof('t'), sf.hour('t')).show()
+ +--------+---------+-------+
+ | t|typeof(t)|hour(t)|
+ +--------+---------+-------+
+ |13:08:15| time(6)| 13|
+ |10:09:16| time(6)| 10|
+ +--------+---------+-------+
"""
return _invoke_function_over_columns("hour", col)
@@ -10191,10 +10276,13 @@ def minute(col: "ColumnOrName") -> Column:
.. versionchanged:: 3.4.0
Supports Spark Connect.
+ .. versionchanged:: 4.1.0
+ Added support for time type.
+
Parameters
----------
col : :class:`~pyspark.sql.Column` or column name
- target date/timestamp column to work on.
+ target date/time/timestamp column to work on.
See Also
--------
@@ -10241,6 +10329,21 @@ def minute(col: "ColumnOrName") -> Column:
|2015-04-08 13:08:15| timestamp| 8|
|2024-10-31 10:09:16| timestamp| 9|
+-------------------+----------+----------+
+
+ Example 3: Extract the minutes from a time column
+
+ >>> import datetime
+ >>> from pyspark.sql import functions as sf
+ >>> df = spark.createDataFrame([
+ ... ("13:08:15",),
+ ... ("10:09:16",)], ['t']).withColumn("t", sf.col("t").cast("time"))
+ >>> df.select("*", sf.typeof('t'), sf.minute('t')).show()
+ +--------+---------+---------+
+ | t|typeof(t)|minute(t)|
+ +--------+---------+---------+
+ |13:08:15| time(6)| 8|
+ |10:09:16| time(6)| 9|
+ +--------+---------+---------+
"""
return _invoke_function_over_columns("minute", col)
@@ -10255,10 +10358,13 @@ def second(col: "ColumnOrName") -> Column:
.. versionchanged:: 3.4.0
Supports Spark Connect.
+ .. versionchanged:: 4.1.0
+ Added support for time type.
+
Parameters
----------
col : :class:`~pyspark.sql.Column` or column name
- target date/timestamp column to work on.
+ target date/time/timestamp column to work on.
Returns
-------
@@ -10305,6 +10411,21 @@ def second(col: "ColumnOrName") -> Column:
|2015-04-08 13:08:15| timestamp| 15|
|2024-10-31 10:09:16| timestamp| 16|
+-------------------+----------+----------+
+
+ Example 3: Extract the seconds from a time column
+
+ >>> import datetime
+ >>> from pyspark.sql import functions as sf
+ >>> df = spark.createDataFrame([
+ ... ("13:08:15",),
+ ... ("10:09:16",)], ['t']).withColumn("t", sf.col("t").cast("time"))
+ >>> df.select("*", sf.typeof('t'), sf.second('t')).show()
+ +--------+---------+---------+
+ | t|typeof(t)|second(t)|
+ +--------+---------+---------+
+ |13:08:15| time(6)| 15|
+ |10:09:16| time(6)| 16|
+ +--------+---------+---------+
"""
return _invoke_function_over_columns("second", col)
@@ -11344,6 +11465,70 @@ def to_date(col: "ColumnOrName", format: Optional[str] = None) -> Column:
return _invoke_function("to_date", _to_java_column(col), _enum_to_value(format))
+@_try_remote_functions
+def try_to_date(col: "ColumnOrName", format: Optional[str] = None) -> Column:
+ """This is a special version of `try_to_date` that performs the same operation, but returns a
+ NULL value instead of raising an error if date cannot be created.
+
+ .. _datetime pattern: https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+
+ .. versionadded:: 4.0.0
+
+ Parameters
+ ----------
+ col : :class:`~pyspark.sql.Column` or column name
+ input column of values to convert.
+ format: literal string, optional
+ format to use to convert date values.
+
+ Returns
+ -------
+ :class:`~pyspark.sql.Column`
+ date value as :class:`pyspark.sql.types.DateType` type.
+
+ See Also
+ --------
+ :meth:`pyspark.sql.functions.to_timestamp`
+ :meth:`pyspark.sql.functions.to_timestamp_ltz`
+ :meth:`pyspark.sql.functions.to_timestamp_ntz`
+ :meth:`pyspark.sql.functions.to_utc_timestamp`
+ :meth:`pyspark.sql.functions.try_to_timestamp`
+ :meth:`pyspark.sql.functions.date_format`
+
+ Examples
+ --------
+ >>> import pyspark.sql.functions as sf
+ >>> df = spark.createDataFrame([('1997-02-28',)], ['ts'])
+ >>> df.select('*', sf.try_to_date(df.ts)).show()
+ +----------+---------------+
+ | ts|try_to_date(ts)|
+ +----------+---------------+
+ |1997-02-28| 1997-02-28|
+ +----------+---------------+
+
+ >>> df.select('*', sf.try_to_date('ts', 'yyyy-MM-dd')).show()
+ +----------+---------------------------+
+ | ts|try_to_date(ts, yyyy-MM-dd)|
+ +----------+---------------------------+
+ |1997-02-28| 1997-02-28|
+ +----------+---------------------------+
+
+ >>> df = spark.createDataFrame([('foo',)], ['ts'])
+ >>> df.select(sf.try_to_date(df.ts)).show()
+ +---------------+
+ |try_to_date(ts)|
+ +---------------+
+ | NULL|
+ +---------------+
+ """
+ from pyspark.sql.classic.column import _to_java_column
+
+ if format is None:
+ return _invoke_function_over_columns("try_to_date", col)
+ else:
+ return _invoke_function("try_to_date", _to_java_column(col), _enum_to_value(format))
+
+
@_try_remote_functions
def unix_date(col: "ColumnOrName") -> Column:
"""Returns the number of days since 1970-01-01.
@@ -11515,6 +11700,74 @@ def unix_seconds(col: "ColumnOrName") -> Column:
return _invoke_function_over_columns("unix_seconds", col)
+@overload
+def to_time(str: "ColumnOrName") -> Column:
+ ...
+
+
+@overload
+def to_time(str: "ColumnOrName", format: "ColumnOrName") -> Column:
+ ...
+
+
+@_try_remote_functions
+def to_time(str: "ColumnOrName", format: Optional["ColumnOrName"] = None) -> Column:
+ """Converts a :class:`~pyspark.sql.Column` into :class:`pyspark.sql.types.TimeType` using the
+ optionally specified format. Specify formats according to `datetime pattern`_. By default, it
+ follows casting rules to :class:`pyspark.sql.types.TimeType` if the format is omitted.
+ Equivalent to ``col.cast("time")``.
+
+ .. _datetime pattern: https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+
+ .. versionadded:: 4.1.0
+
+ Parameters
+ ----------
+ str : :class:`~pyspark.sql.Column` or column name
+ string to be parsed to time.
+ format: :class:`~pyspark.sql.Column` or column name, optional
+ time format pattern to follow.
+
+ Returns
+ -------
+ :class:`~pyspark.sql.Column`
+ time value as :class:`pyspark.sql.types.TimeType` type.
+
+ See Also
+ --------
+ :meth:`pyspark.sql.functions.to_timestamp`
+ :meth:`pyspark.sql.functions.try_to_time`
+
+ Examples
+ --------
+ Example 1: Convert string to a time
+
+ >>> import pyspark.sql.functions as sf
+ >>> df = spark.createDataFrame([("10:30:00",)], ["str"])
+ >>> df.select(sf.to_time(df.str).alias("time")).show()
+ +--------+
+ | time|
+ +--------+
+ |10:30:00|
+ +--------+
+
+ Example 2: Convert string to a time with a format
+
+ >>> import pyspark.sql.functions as sf
+ >>> df = spark.createDataFrame([("10:30:00", "HH:mm:ss")], ["str", "format"])
+ >>> df.select(sf.to_time(df.str, df.format).alias("time")).show()
+ +--------+
+ | time|
+ +--------+
+ |10:30:00|
+ +--------+
+ """
+ if format is None:
+ return _invoke_function_over_columns("to_time", str)
+ else:
+ return _invoke_function_over_columns("to_time", str, format)
+
+
@overload
def to_timestamp(col: "ColumnOrName") -> Column:
...
@@ -11593,6 +11846,85 @@ def to_timestamp(col: "ColumnOrName", format: Optional[str] = None) -> Column:
return _invoke_function("to_timestamp", _to_java_column(col), _enum_to_value(format))
+@overload
+def try_to_time(str: "ColumnOrName") -> Column:
+ ...
+
+
+@overload
+def try_to_time(str: "ColumnOrName", format: "ColumnOrName") -> Column:
+ ...
+
+
+@_try_remote_functions
+def try_to_time(str: "ColumnOrName", format: Optional["ColumnOrName"] = None) -> Column:
+ """Converts a :class:`~pyspark.sql.Column` into :class:`pyspark.sql.types.TimeType` using the
+ optionally specified format. Specify formats according to `datetime pattern`_. By default, it
+ follows casting rules to :class:`pyspark.sql.types.TimeType` if the format is omitted.
+ Equivalent to ``col.cast("time")``. The function always returns null on an invalid input.
+
+ .. _datetime pattern: https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+
+ .. versionadded:: 4.1.0
+
+ Parameters
+ ----------
+ str : :class:`~pyspark.sql.Column` or column name
+ string to be parsed to time.
+ format: :class:`~pyspark.sql.Column` or column name, optional
+ time format pattern to follow.
+
+ Returns
+ -------
+ :class:`~pyspark.sql.Column`
+ time value as :class:`pyspark.sql.types.TimeType` type.
+
+ See Also
+ --------
+ :meth:`pyspark.sql.functions.to_time`
+ :meth:`pyspark.sql.functions.try_to_timestamp`
+
+ Examples
+ --------
+ Example 1: Convert string to a time
+
+ >>> import pyspark.sql.functions as sf
+ >>> df = spark.createDataFrame([("10:30:00",)], ["str"])
+ >>> df.select(sf.try_to_time(df.str).alias("time")).show()
+ +--------+
+ | time|
+ +--------+
+ |10:30:00|
+ +--------+
+
+ Example 2: Convert string to a time with a format
+
+ >>> import pyspark.sql.functions as sf
+ >>> df = spark.createDataFrame([("10:30:00", "HH:mm:ss")], ["str", "format"])
+ >>> df.select(sf.try_to_time(df.str, df.format).alias("time")).show()
+ +--------+
+ | time|
+ +--------+
+ |10:30:00|
+ +--------+
+
+ Example 3: Converion failure results in NULL
+
+ >>> import pyspark.sql.functions as sf
+ >>> df = spark.createDataFrame([("malformed",)], ["str"])
+ >>> df.select(sf.try_to_time(df.str).alias("time")).show()
+ +----+
+ |time|
+ +----+
+ |NULL|
+ +----+
+ """
+ if format is None:
+ return _invoke_function_over_columns("try_to_time", str)
+ else:
+ return _invoke_function_over_columns("try_to_time", str, format)
+
+
@_try_remote_functions
def try_to_timestamp(col: "ColumnOrName", format: Optional["ColumnOrName"] = None) -> Column:
"""
@@ -12370,6 +12702,46 @@ def timestamp_seconds(col: "ColumnOrName") -> Column:
return _invoke_function_over_columns("timestamp_seconds", col)
+@_try_remote_functions
+def time_trunc(unit: "ColumnOrName", time: "ColumnOrName") -> Column:
+ """
+ Returns `time` truncated to the `unit`.
+
+ .. versionadded:: 4.1.0
+
+ Parameters
+ ----------
+ unit : :class:`~pyspark.sql.Column` or column name
+ The unit to truncate the time to. Supported units are: "HOUR", "MINUTE", "SECOND",
+ "MILLISECOND", and "MICROSECOND". The unit is case-insensitive.
+ time : :class:`~pyspark.sql.Column` or column name
+ A time to truncate.
+
+ Returns
+ -------
+ :class:`~pyspark.sql.Column`
+ A time truncated to the specified unit.
+
+ See Also
+ --------
+ :meth:`pyspark.sql.functions.date_trunc`
+
+ Examples
+ --------
+ >>> from pyspark.sql import functions as sf
+ >>> df = spark.createDataFrame(
+ ... [("HOUR", "13:08:15")],
+ ... ['unit', 'time']).withColumn("time", sf.col("time").cast("time"))
+ >>> df.select('*', sf.time_trunc('unit', 'time')).show()
+ +----+--------+----------------------+
+ |unit| time|time_trunc(unit, time)|
+ +----+--------+----------------------+
+ |HOUR|13:08:15| 13:00:00|
+ +----+--------+----------------------+
+ """
+ return _invoke_function_over_columns("time_trunc", unit, time)
+
+
@_try_remote_functions
def timestamp_millis(col: "ColumnOrName") -> Column:
"""
@@ -24305,6 +24677,41 @@ def make_interval(
)
+@_try_remote_functions
+def make_time(hour: "ColumnOrName", minute: "ColumnOrName", second: "ColumnOrName") -> Column:
+ """
+ Create time from hour, minute and second fields. For invalid inputs it will throw an error.
+
+ .. versionadded:: 4.1.0
+
+ Parameters
+ ----------
+ hour : :class:`~pyspark.sql.Column` or column name
+ The hour to represent, from 0 to 23.
+ minute : :class:`~pyspark.sql.Column` or column name
+ The minute to represent, from 0 to 59.
+ second : :class:`~pyspark.sql.Column` or column name
+ The second to represent, from 0 to 59.999999.
+
+ Returns
+ -------
+ :class:`~pyspark.sql.Column`
+ A column representing the created time.
+
+ Examples
+ --------
+ >>> from pyspark.sql import functions as sf
+ >>> df = spark.createDataFrame([(6, 30, 45.887)], ["hour", "minute", "second"])
+ >>> df.select(sf.make_time("hour", "minute", "second").alias("time")).show()
+ +------------+
+ | time|
+ +------------+
+ |06:30:45.887|
+ +------------+
+ """
+ return _invoke_function_over_columns("make_time", hour, minute, second)
+
+
@_try_remote_functions
def make_timestamp(
years: "ColumnOrName",
@@ -26640,32 +27047,7 @@ def udf(
# The following table shows most of Python data and SQL type conversions in normal UDFs that
# are not yet visible to the user. Some of behaviors are buggy and might be changed in the near
# future. The table might have to be eventually documented externally.
- # Please see SPARK-28131's PR to see the codes in order to generate the table below.
- #
- # +-----------------------------+--------------+----------+------+---------------+--------------------+-----------------------------+----------+----------------------+---------+--------------------+----------------------------+------------+--------------+------------------+----------------------+ # noqa
- # |SQL Type \ Python Value(Type)|None(NoneType)|True(bool)|1(int)| a(str)| 1970-01-01(date)|1970-01-01 00:00:00(datetime)|1.0(float)|array('i', [1])(array)|[1](list)| (1,)(tuple)|bytearray(b'ABC')(bytearray)| 1(Decimal)|{'a': 1}(dict)|Row(kwargs=1)(Row)|Row(namedtuple=1)(Row)| # noqa
- # +-----------------------------+--------------+----------+------+---------------+--------------------+-----------------------------+----------+----------------------+---------+--------------------+----------------------------+------------+--------------+------------------+----------------------+ # noqa
- # | boolean| None| True| None| None| None| None| None| None| None| None| None| None| None| X| X| # noqa
- # | tinyint| None| None| 1| None| None| None| None| None| None| None| None| None| None| X| X| # noqa
- # | smallint| None| None| 1| None| None| None| None| None| None| None| None| None| None| X| X| # noqa
- # | int| None| None| 1| None| None| None| None| None| None| None| None| None| None| X| X| # noqa
- # | bigint| None| None| 1| None| None| None| None| None| None| None| None| None| None| X| X| # noqa
- # | string| None| 'true'| '1'| 'a'|'java.util.Gregor...| 'java.util.Gregor...| '1.0'| '[I@66cbb73a'| '[1]'|'[Ljava.lang.Obje...| '[B@5a51eb1a'| '1'| '{a=1}'| X| X| # noqa
- # | date| None| X| X| X|datetime.date(197...| datetime.date(197...| X| X| X| X| X| X| X| X| X| # noqa
- # | timestamp| None| X| X| X| X| datetime.datetime...| X| X| X| X| X| X| X| X| X| # noqa
- # | float| None| None| None| None| None| None| 1.0| None| None| None| None| None| None| X| X| # noqa
- # | double| None| None| None| None| None| None| 1.0| None| None| None| None| None| None| X| X| # noqa
- # | array| None| None| None| None| None| None| None| [1]| [1]| [1]| [65, 66, 67]| None| None| X| X| # noqa
- # | binary| None| None| None|bytearray(b'a')| None| None| None| None| None| None| bytearray(b'ABC')| None| None| X| X| # noqa
- # | decimal(10,0)| None| None| None| None| None| None| None| None| None| None| None|Decimal('1')| None| X| X| # noqa
- # | map| None| None| None| None| None| None| None| None| None| None| None| None| {'a': 1}| X| X| # noqa
- # | struct<_1:int>| None| X| X| X| X| X| X| X|Row(_1=1)| Row(_1=1)| X| X| Row(_1=None)| Row(_1=1)| Row(_1=1)| # noqa
- # +-----------------------------+--------------+----------+------+---------------+--------------------+-----------------------------+----------+----------------------+---------+--------------------+----------------------------+------------+--------------+------------------+----------------------+ # noqa
- #
- # Note: DDL formatted string is used for 'SQL Type' for simplicity. This string can be
- # used in `returnType`.
- # Note: The values inside of the table are generated by `repr`.
- # Note: 'X' means it throws an exception during the conversion.
+ # Please see python/pyspark/sql/tests/udf_type_tests for type tests and golden files
# decorator @udf, @udf(), @udf(dataType())
if f is None or isinstance(f, (str, DataType)):
@@ -26850,6 +27232,75 @@ def udtf(
return _create_py_udtf(cls=cls, returnType=returnType, useArrow=useArrow)
+@_try_remote_functions
+def arrow_udtf(
+ cls: Optional[Type] = None,
+ *,
+ returnType: Optional[Union[StructType, str]] = None,
+) -> Union["UserDefinedTableFunction", Callable[[Type], "UserDefinedTableFunction"]]:
+ """Creates a PyArrow-native user defined table function (UDTF).
+
+ This function provides a PyArrow-native interface for UDTFs, where the eval method
+ receives PyArrow RecordBatches or Arrays and returns an Iterator of PyArrow Tables
+ or RecordBatches.
+ This enables true vectorized computation without row-by-row processing overhead.
+
+ .. versionadded:: 4.1.0
+
+ Parameters
+ ----------
+ cls : class, optional
+ the Python user-defined table function handler class.
+ returnType : :class:`pyspark.sql.types.StructType` or str, optional
+ the return type of the user-defined table function. The value can be either a
+ :class:`pyspark.sql.types.StructType` object or a DDL-formatted struct type string.
+
+ Examples
+ --------
+ UDTF with PyArrow RecordBatch input:
+
+ >>> import pyarrow as pa
+ >>> from pyspark.sql.functions import arrow_udtf
+ >>> @arrow_udtf(returnType="x int, y int")
+ ... class MyUDTF:
+ ... def eval(self, batch: pa.RecordBatch):
+ ... # Process the entire batch vectorized
+ ... x_array = batch.column('x')
+ ... y_array = batch.column('y')
+ ... result_table = pa.table({
+ ... 'x': x_array,
+ ... 'y': y_array
+ ... })
+ ... yield result_table
+ ...
+ >>> df = spark.range(10).selectExpr("id as x", "id as y")
+ >>> MyUDTF(df.asTable()).show() # doctest: +SKIP
+
+ UDTF with PyArrow Array inputs:
+
+ >>> @arrow_udtf(returnType="x int, y int")
+ ... class MyUDTF2:
+ ... def eval(self, x: pa.Array, y: pa.Array):
+ ... # Process arrays vectorized
+ ... result_table = pa.table({
+ ... 'x': x,
+ ... 'y': y
+ ... })
+ ... yield result_table
+ ...
+ >>> MyUDTF2(lit(1), lit(2)).show() # doctest: +SKIP
+
+ Notes
+ -----
+ - The eval method must accept PyArrow RecordBatches or Arrays as input
+ - The eval method must yield PyArrow Tables or RecordBatches as output
+ """
+ if cls is None:
+ return functools.partial(_create_pyarrow_udtf, returnType=returnType)
+ else:
+ return _create_pyarrow_udtf(cls=cls, returnType=returnType)
+
+
def _test() -> None:
import doctest
from pyspark.sql import SparkSession
diff --git a/python/pyspark/sql/pandas/_typing/__init__.pyi b/python/pyspark/sql/pandas/_typing/__init__.pyi
index 8b617ee02eea1..d1e2b7aae6f86 100644
--- a/python/pyspark/sql/pandas/_typing/__init__.pyi
+++ b/python/pyspark/sql/pandas/_typing/__init__.pyi
@@ -63,6 +63,8 @@ GroupedMapUDFTransformWithStateInitStateType = Literal[214]
# Arrow UDFs
ArrowScalarUDFType = Literal[250]
ArrowScalarIterUDFType = Literal[251]
+ArrowGroupedAggUDFType = Literal[252]
+ArrowWindowAggUDFType = Literal[253]
class ArrowVariadicScalarToScalarFunction(Protocol):
def __call__(self, *_: pyarrow.Array) -> pyarrow.Array: ...
diff --git a/python/pyspark/sql/pandas/conversion.py b/python/pyspark/sql/pandas/conversion.py
index 18360fd813921..cb7dfa5552600 100644
--- a/python/pyspark/sql/pandas/conversion.py
+++ b/python/pyspark/sql/pandas/conversion.py
@@ -28,7 +28,6 @@
from warnings import warn
from pyspark.errors.exceptions.captured import unwrap_spark_exception
-from pyspark.loose_version import LooseVersion
from pyspark.util import _load_from_socket
from pyspark.sql.pandas.serializers import ArrowCollectSerializer
from pyspark.sql.pandas.types import _dedup_names
@@ -121,18 +120,10 @@ def toPandas(self) -> "PandasDataFrameLike":
# Pandas DataFrame created from PyArrow uses datetime64[ns] for date type
# values, but we should use datetime.date to match the behavior with when
# Arrow optimization is disabled.
- pandas_options = {"date_as_object": True}
-
- if LooseVersion(pa.__version__) >= LooseVersion("13.0.0"):
- # A legacy option to coerce date32, date64, duration, and timestamp
- # time units to nanoseconds when converting to pandas.
- # This option can only be added since 13.0.0.
- pandas_options.update(
- {
- "coerce_temporal_nanoseconds": True,
- }
- )
-
+ pandas_options = {
+ "date_as_object": True,
+ "coerce_temporal_nanoseconds": True,
+ }
if self_destruct:
# Configure PyArrow to use as little memory as possible:
# self_destruct - free columns as they are converted
@@ -739,7 +730,7 @@ def _create_from_pandas_with_arrow(
jsparkSession = self._jsparkSession
safecheck = self._jconf.arrowSafeTypeConversion()
- ser = ArrowStreamPandasSerializer(timezone, safecheck)
+ ser = ArrowStreamPandasSerializer(timezone, safecheck, False)
@no_type_check
def reader_func(temp_filename):
diff --git a/python/pyspark/sql/pandas/functions.py b/python/pyspark/sql/pandas/functions.py
index 6fc15c03a6bdc..79f3f75bd573c 100644
--- a/python/pyspark/sql/pandas/functions.py
+++ b/python/pyspark/sql/pandas/functions.py
@@ -48,26 +48,292 @@ class ArrowUDFType:
SCALAR_ITER = PythonEvalType.SQL_SCALAR_ARROW_ITER_UDF
+ GROUPED_AGG = PythonEvalType.SQL_GROUPED_AGG_ARROW_UDF
+
def arrow_udf(f=None, returnType=None, functionType=None):
- return vectorized_udf(f, returnType, functionType, "arrow")
+ """
+ Creates an arrow user defined function.
+ Arrow UDFs are user defined functions that are executed by Spark using Arrow to transfer
+ and work with the data, which allows `pyarrow.Array` operations. An Arrow UDF is defined
+ using the `arrow_udf` as a decorator or to wrap the function, and no additional configuration
+ is required. An Arrow UDF behaves as a regular PySpark function API in general.
-def pandas_udf(f=None, returnType=None, functionType=None):
- return vectorized_udf(f, returnType, functionType, "pandas")
+ .. versionadded:: 4.1.0
+ Parameters
+ ----------
+ f : function, optional
+ user-defined function. A python function if used as a standalone function
+ returnType : :class:`pyspark.sql.types.DataType` or str, optional
+ the return type of the user-defined function. The value can be either a
+ :class:`pyspark.sql.types.DataType` object or a DDL-formatted type string.
+ functionType : int, optional
+ an enum value in :class:`pyspark.sql.functions.ArrowUDFType`.
+ Default: SCALAR. This parameter exists for compatibility.
+ Using Python type hints is encouraged.
-def vectorized_udf(
- f=None,
- returnType=None,
- functionType=None,
- kind: str = "pandas",
-):
+ Examples
+ --------
+ In order to use this API, customarily the below are imported:
+
+ >>> import pyarrow as pa
+ >>> from pyspark.sql.functions import arrow_udf
+
+ `Python type hints `_
+ detect the function types as below:
+
+ >>> from pyspark.sql.functions import ArrowUDFType
+ >>> from pyspark.sql.types import IntegerType
+ >>> @arrow_udf(IntegerType(), ArrowUDFType.SCALAR)
+ ... def slen(v: pa.Array) -> pa.Array:
+ ... return pa.compute.utf8_length(v)
+
+ Note that the type hint should use `pyarrow.Array` in all cases.
+
+ * Arrays to Arrays
+ `pyarrow.Array`, ... -> `pyarrow.Array`
+
+ The function takes one or more `pyarrow.Array` and outputs one `pyarrow.Array`.
+ The output of the function should always be of the same length as the input.
+
+ >>> @arrow_udf("string")
+ ... def to_upper(s: pa.Array) -> pa.Array:
+ ... return pa.compute.ascii_upper(s)
+ ...
+ >>> df = spark.createDataFrame([("John Doe",)], ("name",))
+ >>> df.select(to_upper("name")).show()
+ +--------------+
+ |to_upper(name)|
+ +--------------+
+ | JOHN DOE|
+ +--------------+
+
+ >>> @arrow_udf("first string, last string")
+ ... def split_expand(v: pa.Array) -> pa.Array:
+ ... b = pa.compute.ascii_split_whitespace(v)
+ ... s0 = pa.array([t[0] for t in b])
+ ... s1 = pa.array([t[1] for t in b])
+ ... return pa.StructArray.from_arrays([s0, s1], names=["first", "last"])
+ ...
+ >>> df = spark.createDataFrame([("John Doe",)], ("name",))
+ >>> df.select(split_expand("name")).show()
+ +------------------+
+ |split_expand(name)|
+ +------------------+
+ | {John, Doe}|
+ +------------------+
+
+ This type of Pandas UDF can use keyword arguments:
+
+ >>> from pyspark.sql import functions as sf
+ >>> @arrow_udf(returnType=IntegerType())
+ ... def calc(a: pa.Array, b: pa.Array) -> pa.Array:
+ ... return pa.compute.add(a, pa.compute.multiply(b, 10))
+ ...
+ >>> spark.range(2).select(calc(b=sf.col("id") * 10, a=sf.col("id"))).show()
+ +-----------------------------+
+ |calc(b => (id * 10), a => id)|
+ +-----------------------------+
+ | 0|
+ | 101|
+ +-----------------------------+
+
+ .. note:: The length of the input is not that of the whole input column, but is the
+ length of an internal batch used for each call to the function.
+
+ * Iterator of Arrays to Iterator of Arrays
+ `Iterator[pyarrow.Array]` -> `Iterator[pyarrow.Array]`
+
+ The function takes an iterator of `pyarrow.Array` and outputs an iterator of
+ `pyarrow.Array`. In this case, the created arrow UDF instance requires one input
+ column when this is called as a PySpark column. The length of the entire output from
+ the function should be the same length of the entire input; therefore, it can
+ prefetch the data from the input iterator as long as the lengths are the same.
+
+ It is also useful when the UDF execution
+ requires initializing some states, although internally it works identically as
+ Arrays to Arrays case. The pseudocode below illustrates the example.
+
+ .. highlight:: python
+ .. code-block:: python
+
+ @arrow_udf("long")
+ def calculate(iterator: Iterator[pa.Array]) -> Iterator[pa.Array]:
+ # Do some expensive initialization with a state
+ state = very_expensive_initialization()
+ for x in iterator:
+ # Use that state for whole iterator.
+ yield calculate_with_state(x, state)
+
+ df.select(calculate("value")).show()
+
+ >>> import pandas as pd
+ >>> from typing import Iterator
+ >>> @arrow_udf("long")
+ ... def plus_one(iterator: Iterator[pa.Array]) -> Iterator[pa.Array]:
+ ... for v in iterator:
+ ... yield pa.compute.add(v, 1)
+ ...
+ >>> df = spark.createDataFrame(pd.DataFrame([1, 2, 3], columns=["v"]))
+ >>> df.select(plus_one(df.v)).show()
+ +-----------+
+ |plus_one(v)|
+ +-----------+
+ | 2|
+ | 3|
+ | 4|
+ +-----------+
+
+ .. note:: The length of each series is the length of a batch internally used.
+
+ * Iterator of Multiple Arrays to Iterator of Arrays
+ `Iterator[Tuple[pyarrow.Array, ...]]` -> `Iterator[pyarrow.Array]`
+
+ The function takes an iterator of a tuple of multiple `pyarrow.Array` and outputs an
+ iterator of `pyarrow.Array`. In this case, the created arrow UDF instance requires
+ input columns as many as the series when this is called as a PySpark column.
+ Otherwise, it has the same characteristics and restrictions as Iterator of Arrays
+ to Iterator of Arrays case.
+
+ >>> from typing import Iterator, Tuple
+ >>> from pyspark.sql import functions as sf
+ >>> @arrow_udf("long")
+ ... def multiply(iterator: Iterator[Tuple[pa.Array, pa.Array]]) -> Iterator[pa.Array]:
+ ... for v1, v2 in iterator:
+ ... yield pa.compute.multiply(v1, v2.field("v"))
+ ...
+ >>> df = spark.createDataFrame(pd.DataFrame([1, 2, 3], columns=["v"]))
+ >>> df.withColumn('output', multiply(sf.col("v"), sf.struct(sf.col("v")))).show()
+ +---+------+
+ | v|output|
+ +---+------+
+ | 1| 1|
+ | 2| 4|
+ | 3| 9|
+ +---+------+
+
+ .. note:: The length of each series is the length of a batch internally used.
+
+ * Arrays to Scalar
+ `pyarrow.Array`, ... -> `Any`
+
+ The function takes `pyarrow.Array` and returns a scalar value. The returned scalar
+ can be a python primitive type, (e.g., int or float), a numpy data type (e.g.,
+ numpy.int64 or numpy.float64), or a `pyarrow.Scalar` instance which supports complex
+ return types.
+ `Any` should ideally be a specific scalar type accordingly.
+
+ >>> @arrow_udf("double")
+ ... def mean_udf(v: pa.Array) -> float:
+ ... return pa.compute.mean(v).as_py()
+ ...
+ >>> df = spark.createDataFrame(
+ ... [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)], ("id", "v"))
+ >>> df.groupby("id").agg(mean_udf(df['v'])).show()
+ +---+-----------+
+ | id|mean_udf(v)|
+ +---+-----------+
+ | 1| 1.5|
+ | 2| 6.0|
+ +---+-----------+
+
+ The retun type can also be a complex type such as struct, list, or map.
+
+ >>> @arrow_udf("struct")
+ ... def min_max_udf(v: pa.Array) -> pa.Scalar:
+ ... m1 = pa.compute.min(v)
+ ... m2 = pa.compute.max(v)
+ ... t = pa.struct([pa.field("m1", pa.float64()), pa.field("m2", pa.float64())])
+ ... return pa.scalar(value={"m1": m1.as_py(), "m2": m2.as_py()}, type=t)
+ ...
+ >>> df = spark.createDataFrame(
+ ... [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)], ("id", "v"))
+ >>> df.groupby("id").agg(min_max_udf(df['v'])).show()
+ +---+--------------+
+ | id|min_max_udf(v)|
+ +---+--------------+
+ | 1| {1.0, 2.0}|
+ | 2| {3.0, 10.0}|
+ +---+--------------+
+
+ This type of Pandas UDF can use keyword arguments:
+
+ >>> @arrow_udf("double")
+ ... def weighted_mean_udf(v: pa.Array, w: pa.Array) -> float:
+ ... import numpy as np
+ ... return np.average(v.to_numpy(), weights=w)
+ ...
+ >>> df = spark.createDataFrame(
+ ... [(1, 1.0, 1.0), (1, 2.0, 2.0), (2, 3.0, 1.0), (2, 5.0, 2.0), (2, 10.0, 3.0)],
+ ... ("id", "v", "w"))
+ >>> df.groupby("id").agg(weighted_mean_udf(w=df["w"], v=df["v"])).show()
+ +---+---------------------------------+
+ | id|weighted_mean_udf(w => w, v => v)|
+ +---+---------------------------------+
+ | 1| 1.6666666666666667|
+ | 2| 7.166666666666667|
+ +---+---------------------------------+
+
+ This UDF can also be used as window functions as below:
+
+ >>> from pyspark.sql import Window
+ >>> @arrow_udf("double")
+ ... def mean_udf(v: pa.Array) -> float:
+ ... return pa.compute.mean(v).as_py()
+ ...
+ >>> df = spark.createDataFrame(
+ ... [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)], ("id", "v"))
+ >>> w = Window.partitionBy('id').orderBy('v').rowsBetween(-1, 0)
+ >>> df.withColumn('mean_v', mean_udf("v").over(w)).show()
+ +---+----+------+
+ | id| v|mean_v|
+ +---+----+------+
+ | 1| 1.0| 1.0|
+ | 1| 2.0| 1.5|
+ | 2| 3.0| 3.0|
+ | 2| 5.0| 4.0|
+ | 2|10.0| 7.5|
+ +---+----+------+
+
+ .. note:: For performance reasons, the input series to window functions are not copied.
+ Therefore, mutating the input arrays is not allowed and will cause incorrect results.
+ For the same reason, users should also not rely on the index of the input arrays.
+
+ Notes
+ -----
+ The user-defined functions do not support conditional expressions or short circuiting
+ in boolean expressions and it ends up with being executed all internally. If the functions
+ can fail on special rows, the workaround is to incorporate the condition into the functions.
+
+ The user-defined functions do not take keyword arguments on the calling side.
+
+ The data type of returned `pyarrow.Array` from the user-defined functions should be
+ matched with defined `returnType` (see :meth:`types.to_arrow_type` and
+ :meth:`types.from_arrow_type`). When there is mismatch between them, Spark might do
+ conversion on returned data. The conversion is not guaranteed to be correct and results
+ should be checked for accuracy by users.
+
+ See Also
+ --------
+ pyspark.sql.GroupedData.agg
+ pyspark.sql.DataFrame.mapInArrow
+ pyspark.sql.GroupedData.applyInArrow
+ pyspark.sql.PandasCogroupedOps.applyInArrow
+ pyspark.sql.UDFRegistration.register
"""
- Creates a vectorized user defined function.
+ require_minimum_pyarrow_version()
+
+ return vectorized_udf(f, returnType, functionType, "arrow")
+
+
+def pandas_udf(f=None, returnType=None, functionType=None):
+ """
+ Creates a pandas user defined function.
Pandas UDFs are user defined functions that are executed by Spark using Arrow to transfer
- data and Pandas to work with the data, which allows vectorized operations. A Pandas UDF
+ data and Pandas to work with the data, which allows pandas operations. A Pandas UDF
is defined using the `pandas_udf` as a decorator or to wrap the function, and no
additional configuration is required. A Pandas UDF behaves as a regular PySpark function
API in general.
@@ -102,6 +368,7 @@ def vectorized_udf(
From Spark 3.0 with Python 3.6+, `Python type hints `_
detect the function types as below:
+ >>> from pyspark.sql.types import IntegerType
>>> @pandas_udf(IntegerType())
... def slen(s: pd.Series) -> pd.Series:
... return s.str.len()
@@ -125,21 +392,26 @@ def vectorized_udf(
`pandas.DataFrame` as below:
>>> @pandas_udf("col1 string, col2 long")
- >>> def func(s1: pd.Series, s2: pd.Series, s3: pd.DataFrame) -> pd.DataFrame:
+ ... def func(s1: pd.Series, s2: pd.Series, s3: pd.DataFrame) -> pd.DataFrame:
... s3['col2'] = s1 + s2.str.len()
... return s3
- ...
- >>> # Create a Spark DataFrame that has three columns including a struct column.
- ... df = spark.createDataFrame(
+
+
+ Create a Spark DataFrame that has three columns including a struct column.
+
+ >>> df = spark.createDataFrame(
... [[1, "a string", ("a nested string",)]],
... "long_col long, string_col string, struct_col struct")
+
>>> df.printSchema()
root
- |-- long_column: long (nullable = true)
- |-- string_column: string (nullable = true)
- |-- struct_column: struct (nullable = true)
+ |-- long_col: long (nullable = true)
+ |-- string_col: string (nullable = true)
+ |-- struct_col: struct (nullable = true)
| |-- col1: string (nullable = true)
+
>>> df.select(func("long_col", "string_col", "struct_col")).printSchema()
+ root
|-- func(long_col, string_col, struct_col): struct (nullable = true)
| |-- col1: string (nullable = true)
| |-- col2: long (nullable = true)
@@ -179,11 +451,12 @@ def vectorized_udf(
This type of Pandas UDF can use keyword arguments:
+ >>> from pyspark.sql import functions as sf
>>> @pandas_udf(returnType=IntegerType())
... def calc(a: pd.Series, b: pd.Series) -> pd.Series:
... return a + 10 * b
...
- >>> spark.range(2).select(calc(b=col("id") * 10, a=col("id"))).show()
+ >>> spark.range(2).select(calc(b=sf.col("id") * 10, a=sf.col("id"))).show()
+-----------------------------+
|calc(b => (id * 10), a => id)|
+-----------------------------+
@@ -248,14 +521,14 @@ def calculate(iterator: Iterator[pd.Series]) -> Iterator[pd.Series]:
to Iterator of Series case.
>>> from typing import Iterator, Tuple
- >>> from pyspark.sql.functions import struct, col
+ >>> from pyspark.sql import functions as sf
>>> @pandas_udf("long")
... def multiply(iterator: Iterator[Tuple[pd.Series, pd.DataFrame]]) -> Iterator[pd.Series]:
... for s1, df in iterator:
... yield s1 * df.v
...
>>> df = spark.createDataFrame(pd.DataFrame([1, 2, 3], columns=["v"]))
- >>> df.withColumn('output', multiply(col("v"), struct(col("v")))).show()
+ >>> df.withColumn('output', multiply(sf.col("v"), sf.struct(sf.col("v")))).show()
+---+------+
| v|output|
+---+------+
@@ -359,40 +632,26 @@ def calculate(iterator: Iterator[pd.Series]) -> Iterator[pd.Series]:
pyspark.sql.UDFRegistration.register
"""
- # The following table shows most of Pandas data and SQL type conversions in Pandas UDFs that
- # are not yet visible to the user. Some of behaviors are buggy and might be changed in the near
- # future. The table might have to be eventually documented externally.
- # Please see SPARK-28132's PR to see the codes in order to generate the table below.
- #
- # +-----------------------------+----------------------+------------------+------------------+------------------+--------------------+--------------------+------------------+------------------+------------------+------------------+--------------+--------------+--------------+-----------------------------------+-----------------------------------------------------+-----------------+--------------------+-----------------------------+--------------+-----------------+------------------+---------------+--------------------------------+ # noqa
- # |SQL Type \ Pandas Value(Type)|None(object(NoneType))| True(bool)| 1(int8)| 1(int16)| 1(int32)| 1(int64)| 1(uint8)| 1(uint16)| 1(uint32)| 1(uint64)| 1.0(float16)| 1.0(float32)| 1.0(float64)|1970-01-01 00:00:00(datetime64[ns])|1970-01-01 00:00:00-05:00(datetime64[ns, US/Eastern])|a(object(string))| 1(object(Decimal))|[1 2 3](object(array[int32]))| 1.0(float128)|(1+0j)(complex64)|(1+0j)(complex128)| A(category)|1 days 00:00:00(timedelta64[ns])| # noqa
- # +-----------------------------+----------------------+------------------+------------------+------------------+--------------------+--------------------+------------------+------------------+------------------+------------------+--------------+--------------+--------------+-----------------------------------+-----------------------------------------------------+-----------------+--------------------+-----------------------------+--------------+-----------------+------------------+---------------+--------------------------------+ # noqa
- # | boolean| None| True| True| True| True| True| True| True| True| True| True| True| True| X| X| X| X| X| X| X| X| X| X| # noqa
- # | tinyint| None| 1| 1| 1| 1| 1| 1| 1| 1| 1| 1| 1| 1| X| X| X| 1| X| X| X| X| X| X| # noqa
- # | smallint| None| 1| 1| 1| 1| 1| 1| 1| 1| 1| 1| 1| 1| X| X| X| 1| X| X| X| X| X| X| # noqa
- # | int| None| 1| 1| 1| 1| 1| 1| 1| 1| 1| 1| 1| 1| X| X| X| 1| X| X| X| X| X| X| # noqa
- # | bigint| None| 1| 1| 1| 1| 1| 1| 1| 1| 1| 1| 1| 1| 0| 18000000000000| X| 1| X| X| X| X| X| 86400000000000| # noqa
- # | float| None| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| X| X| X| X| X| X| X| X| X| X| # noqa
- # | double| None| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| X| X| X| X| X| X| X| X| X| X| # noqa
- # | date| None| X| X| X|datetime.date(197...| X| X| X| X| X| X| X| X| datetime.date(197...| datetime.date(197...| X|datetime.date(197...| X| X| X| X| X| X| # noqa
- # | timestamp| None| X| X| X| X|datetime.datetime...| X| X| X| X| X| X| X| datetime.datetime...| datetime.datetime...| X|datetime.datetime...| X| X| X| X| X| X| # noqa
- # | string| None| X| X| X| X| X| X| X| X| X| X| X| X| X| X| 'a'| X| X| X| X| X| 'A'| X| # noqa
- # | decimal(10,0)| None| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| Decimal('1')| X| X| X| X| X| X| # noqa
- # | array| None| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| [1, 2, 3]| X| X| X| X| X| # noqa
- # | map| None| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| # noqa
- # | struct<_1:int>| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| # noqa
- # | binary| None|bytearray(b'\x01')|bytearray(b'\x01')|bytearray(b'\x01')| bytearray(b'\x01')| bytearray(b'\x01')|bytearray(b'\x01')|bytearray(b'\x01')|bytearray(b'\x01')|bytearray(b'\x01')|bytearray(b'')|bytearray(b'')|bytearray(b'')| bytearray(b'')| bytearray(b'')| bytearray(b'a')| X| X|bytearray(b'')| bytearray(b'')| bytearray(b'')|bytearray(b'A')| bytearray(b'')| # noqa
- # +-----------------------------+----------------------+------------------+------------------+------------------+--------------------+--------------------+------------------+------------------+------------------+------------------+--------------+--------------+--------------+-----------------------------------+-----------------------------------------------------+-----------------+--------------------+-----------------------------+--------------+-----------------+------------------+---------------+--------------------------------+ # noqa
- #
- # Note: DDL formatted string is used for 'SQL Type' for simplicity. This string can be
- # used in `returnType`.
- # Note: The values inside of the table are generated by `repr`.
- # Note: Python 3.11.9, Pandas 2.2.3 and PyArrow 17.0.0 are used.
- # Note: Timezone is KST.
- # Note: 'X' means it throws an exception during the conversion.
+ # The return type and input type behavior of pandas_udfs is documented in
+ # python/pyspark/sql/tests/udf_type_tests.
+ # It shows most of Pandas data and SQL type conversions in Pandas UDFs that are not
+ # yet visible to the user.
+ # Some of behaviors are buggy and might be changed in the near future. The table might
+ # have to be eventually documented externally.
+ # The folder python/pyspark/sql/tests/udf_type_tests contains type tests and golden
+ # files, as well as the code to regenerate the tables.
require_minimum_pandas_version()
require_minimum_pyarrow_version()
+ return vectorized_udf(f, returnType, functionType, "pandas")
+
+
+def vectorized_udf(
+ f=None,
+ returnType=None,
+ functionType=None,
+ kind: str = "pandas",
+):
assert kind in ["pandas", "arrow"], "kind should be either 'pandas' or 'arrow'"
# decorator @pandas_udf(returnType, functionType)
@@ -454,6 +713,7 @@ def calculate(iterator: Iterator[pd.Series]) -> Iterator[pd.Series]:
if kind == "arrow" and eval_type not in [
PythonEvalType.SQL_SCALAR_ARROW_UDF,
PythonEvalType.SQL_SCALAR_ARROW_ITER_UDF,
+ PythonEvalType.SQL_GROUPED_AGG_ARROW_UDF,
None,
]: # None means it should infer the type from type hints.
raise PySparkTypeError(
@@ -524,7 +784,7 @@ def _validate_vectorized_udf(f, evalType, kind: str = "pandas") -> int:
type_hints = get_type_hints(f)
except NameError:
type_hints = {}
- evalType = infer_eval_type(signature(f), type_hints)
+ evalType = infer_eval_type(signature(f), type_hints, kind)
assert evalType is not None
if evalType is None:
@@ -541,6 +801,7 @@ def _validate_vectorized_udf(f, evalType, kind: str = "pandas") -> int:
evalType == PythonEvalType.SQL_SCALAR_PANDAS_UDF
or evalType == PythonEvalType.SQL_SCALAR_ARROW_UDF
or evalType == PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF
+ or evalType == PythonEvalType.SQL_SCALAR_ARROW_ITER_UDF
)
and len(argspec.args) == 0
and argspec.varargs is None
@@ -549,7 +810,7 @@ def _validate_vectorized_udf(f, evalType, kind: str = "pandas") -> int:
errorClass="INVALID_PANDAS_UDF",
messageParameters={
"detail": f"0-arg {kind_str} are not supported. "
- "Instead, create a 1-arg pandas_udf and ignore the arg in your function.",
+ f"Instead, create a 1-arg {kind_str} and ignore the arg in your function.",
},
)
@@ -602,3 +863,31 @@ def _create_vectorized_udf(f, returnType, evalType, kind):
return _create_connect_udf(f, returnType, evalType)
else:
return _create_udf(f, returnType, evalType)
+
+
+def _test() -> None:
+ import sys
+ import doctest
+ from pyspark.sql import SparkSession
+ import pyspark.sql.pandas.functions
+
+ globs = pyspark.sql.column.__dict__.copy()
+ spark = (
+ SparkSession.builder.master("local[4]")
+ .appName("pyspark.sql.pandas.functions tests")
+ .getOrCreate()
+ )
+ globs["spark"] = spark
+
+ (failure_count, test_count) = doctest.testmod(
+ pyspark.sql.pandas.functions,
+ globs=globs,
+ optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | doctest.REPORT_NDIFF,
+ )
+ spark.stop()
+ if failure_count:
+ sys.exit(-1)
+
+
+if __name__ == "__main__":
+ _test()
diff --git a/python/pyspark/sql/pandas/functions.pyi b/python/pyspark/sql/pandas/functions.pyi
index 925871bd0e61e..70ff08679b6b0 100644
--- a/python/pyspark/sql/pandas/functions.pyi
+++ b/python/pyspark/sql/pandas/functions.pyi
@@ -39,6 +39,7 @@ from pyspark.sql.pandas._typing import (
ArrowScalarUDFType,
ArrowScalarIterFunction,
ArrowScalarIterUDFType,
+ ArrowGroupedAggUDFType,
)
from pyspark import since as since # noqa: F401
@@ -54,6 +55,7 @@ class PandasUDFType:
class ArrowUDFType:
SCALAR: ArrowScalarUDFType
SCALAR_ITER: ArrowScalarIterUDFType
+ GROUPED_AGG: ArrowGroupedAggUDFType
@overload
def arrow_udf(
diff --git a/python/pyspark/sql/pandas/serializers.py b/python/pyspark/sql/pandas/serializers.py
index b154318dc430c..437ceede78a69 100644
--- a/python/pyspark/sql/pandas/serializers.py
+++ b/python/pyspark/sql/pandas/serializers.py
@@ -19,12 +19,12 @@
Serializers for PyArrow and pandas conversions. See `pyspark.serializers` for more details.
"""
+from decimal import Decimal
from itertools import groupby
from typing import TYPE_CHECKING, Optional
import pyspark
from pyspark.errors import PySparkRuntimeError, PySparkTypeError, PySparkValueError
-from pyspark.loose_version import LooseVersion
from pyspark.serializers import (
Serializer,
read_int,
@@ -33,6 +33,7 @@
CPickleSerializer,
)
from pyspark.sql import Row
+from pyspark.sql.conversion import LocalDataToArrowConversion, ArrowTableToRowsConversion
from pyspark.sql.pandas.types import (
from_arrow_type,
is_variant,
@@ -195,6 +196,38 @@ def load_stream(self, stream):
return ArrowStreamSerializer.load_stream(self, stream)
+class ArrowStreamArrowUDTFSerializer(ArrowStreamUDTFSerializer):
+ """
+ Serializer for PyArrow-native UDTFs that work directly with PyArrow RecordBatches and Arrays.
+ """
+
+ def __init__(self, table_arg_offsets=None):
+ super().__init__()
+ self.table_arg_offsets = table_arg_offsets if table_arg_offsets else []
+
+ def load_stream(self, stream):
+ """
+ Flatten the struct into Arrow's record batches.
+ """
+ import pyarrow as pa
+
+ batches = super().load_stream(stream)
+ for batch in batches:
+ result_batches = []
+ for i in range(batch.num_columns):
+ if i in self.table_arg_offsets:
+ struct = batch.column(i)
+ # Flatten the struct and create a RecordBatch from it
+ flattened_batch = pa.RecordBatch.from_arrays(
+ struct.flatten(), schema=pa.schema(struct.type)
+ )
+ result_batches.append(flattened_batch)
+ else:
+ # Keep the column as it is for non-table columns
+ result_batches.append(batch.column(i))
+ yield result_batches
+
+
class ArrowStreamGroupUDFSerializer(ArrowStreamUDFSerializer):
"""
Serializes pyarrow.RecordBatch data with Arrow streaming format.
@@ -250,12 +283,50 @@ class ArrowStreamPandasSerializer(ArrowStreamSerializer):
If True, conversion from Arrow to Pandas checks for overflow/truncation
assign_cols_by_name : bool
If True, then Pandas DataFrames will get columns by name
+ int_to_decimal_coercion_enabled : bool
+ If True, applies additional coercions in Python before converting to Arrow
+ This has performance penalties.
"""
- def __init__(self, timezone, safecheck):
+ def __init__(self, timezone, safecheck, int_to_decimal_coercion_enabled):
super(ArrowStreamPandasSerializer, self).__init__()
self._timezone = timezone
self._safecheck = safecheck
+ self._int_to_decimal_coercion_enabled = int_to_decimal_coercion_enabled
+
+ @staticmethod
+ def _apply_python_coercions(series, arrow_type):
+ """
+ Apply additional coercions to the series in Python before converting to Arrow:
+ - Convert integer series to decimal type.
+ When we have a pandas series of integers that needs to be converted to
+ pyarrow.decimal128 (with precision < 20), PyArrow fails with precision errors.
+ Explicitly cast to Decimal first.
+
+ Parameters
+ ----------
+ series : pandas.Series
+ The series to potentially convert
+ arrow_type : pyarrow.DataType
+ The target arrow type
+
+ Returns
+ -------
+ pandas.Series
+ The potentially converted pandas series
+ """
+ import pyarrow.types as types
+ import pandas as pd
+
+ # Convert integer series to Decimal objects
+ if (
+ types.is_decimal(arrow_type)
+ and series.dtype.kind in ["i", "u"] # integer types (signed/unsigned)
+ and not series.empty
+ ):
+ series = series.apply(lambda x: Decimal(x) if pd.notna(x) else None)
+
+ return series
def arrow_to_pandas(
self, arrow_column, idx, struct_in_pandas="dict", ndarray_as_list=False, spark_type=None
@@ -264,20 +335,10 @@ def arrow_to_pandas(
# instead of creating datetime64[ns] as intermediate data to avoid overflow caused by
# datetime64[ns] type handling.
# Cast dates to objects instead of datetime64[ns] dtype to avoid overflow.
- pandas_options = {"date_as_object": True}
-
- import pyarrow as pa
-
- if LooseVersion(pa.__version__) >= LooseVersion("13.0.0"):
- # A legacy option to coerce date32, date64, duration, and timestamp
- # time units to nanoseconds when converting to pandas.
- # This option can only be added since 13.0.0.
- pandas_options.update(
- {
- "coerce_temporal_nanoseconds": True,
- }
- )
-
+ pandas_options = {
+ "date_as_object": True,
+ "coerce_temporal_nanoseconds": True,
+ }
s = arrow_column.to_pandas(**pandas_options)
# TODO(SPARK-43579): cache the converter for reuse
@@ -325,6 +386,9 @@ def _create_array(self, series, arrow_type, spark_type=None, arrow_cast=False):
)
series = conv(series)
+ if self._int_to_decimal_coercion_enabled:
+ series = self._apply_python_coercions(series, arrow_type)
+
if hasattr(series.array, "__arrow_array__"):
mask = None
else:
@@ -443,8 +507,11 @@ def __init__(
ndarray_as_list=False,
arrow_cast=False,
input_types=None,
+ int_to_decimal_coercion_enabled=False,
):
- super(ArrowStreamPandasUDFSerializer, self).__init__(timezone, safecheck)
+ super(ArrowStreamPandasUDFSerializer, self).__init__(
+ timezone, safecheck, int_to_decimal_coercion_enabled
+ )
self._assign_cols_by_name = assign_cols_by_name
self._df_for_struct = df_for_struct
self._struct_in_pandas = struct_in_pandas
@@ -653,15 +720,15 @@ def _create_array(self, arr, arrow_type, arrow_cast):
assert isinstance(arr, pa.Array)
assert isinstance(arrow_type, pa.DataType)
- # TODO: should we handle timezone here?
-
- try:
+ if arr.type == arrow_type:
return arr
- except pa.lib.ArrowException:
- if arrow_cast:
- return arr.cast(target_type=arrow_type, safe=self._safecheck)
- else:
- raise
+ elif arrow_cast:
+ return arr.cast(target_type=arrow_type, safe=self._safecheck)
+ else:
+ raise PySparkTypeError(
+ "Arrow UDFs require the return type to match the expected Arrow type. "
+ f"Expected: {arrow_type}, but got: {arr.type}."
+ )
def dump_stream(self, iterator, stream):
"""
@@ -695,12 +762,119 @@ def __repr__(self):
return "ArrowStreamArrowUDFSerializer"
+class ArrowBatchUDFSerializer(ArrowStreamArrowUDFSerializer):
+ """
+ Serializer used by Python worker to evaluate Arrow Python UDFs
+ when the legacy pandas conversion is disabled
+ (instead of legacy ArrowStreamPandasUDFSerializer).
+
+ Parameters
+ ----------
+ timezone : str
+ A timezone to respect when handling timestamp values
+ safecheck : bool
+ If True, conversion from Arrow to Pandas checks for overflow/truncation
+ input_types : list
+ List of input data types for the UDF
+ int_to_decimal_coercion_enabled : bool
+ If True, applies additional coercions in Python before converting to Arrow
+ This has performance penalties.
+ """
+
+ def __init__(
+ self,
+ timezone,
+ safecheck,
+ input_types,
+ int_to_decimal_coercion_enabled=False,
+ ):
+ super().__init__(
+ timezone=timezone,
+ safecheck=safecheck,
+ assign_cols_by_name=False,
+ arrow_cast=True,
+ )
+ self._input_types = input_types
+ self._int_to_decimal_coercion_enabled = int_to_decimal_coercion_enabled
+
+ def load_stream(self, stream):
+ """
+ Loads a stream of Arrow record batches and converts them to Python values.
+
+ Parameters
+ ----------
+ stream : object
+ Input stream containing Arrow record batches
+
+ Yields
+ ------
+ list
+ List of columns containing list of Python values.
+ """
+ converters = [
+ ArrowTableToRowsConversion._create_converter(dt, none_on_identity=True)
+ for dt in self._input_types
+ ]
+
+ for batch in super().load_stream(stream):
+ columns = [
+ [conv(v) for v in column.to_pylist()] if conv is not None else column.to_pylist()
+ for column, conv in zip(batch.itercolumns(), converters)
+ ]
+ if len(columns) == 0:
+ yield [[pyspark._NoValue] * batch.num_rows]
+ else:
+ yield columns
+
+ def dump_stream(self, iterator, stream):
+ """
+ Dumps an iterator of Python values as a stream of Arrow record batches.
+
+ Parameters
+ ----------
+ iterator : iterator
+ Iterator yielding tuples of (data, arrow_type, spark_type) for single UDF
+ or list of tuples for multiple UDFs in a projection
+ stream : object
+ Output stream to write the Arrow record batches
+
+ Returns
+ -------
+ object
+ Result of writing the Arrow stream via ArrowStreamArrowUDFSerializer dump_stream
+ """
+ import pyarrow as pa
+
+ def create_array(results, arrow_type, spark_type):
+ conv = LocalDataToArrowConversion._create_converter(
+ spark_type,
+ none_on_identity=True,
+ int_to_decimal_coercion_enabled=self._int_to_decimal_coercion_enabled,
+ )
+ converted = [conv(res) for res in results] if conv is not None else results
+ try:
+ return pa.array(converted, type=arrow_type)
+ except pa.lib.ArrowInvalid:
+ return pa.array(converted).cast(target_type=arrow_type, safe=self._safecheck)
+
+ def py_to_batch():
+ for packed in iterator:
+ if len(packed) == 3 and isinstance(packed[1], pa.DataType):
+ # single array UDF in a projection
+ yield create_array(packed[0], packed[1], packed[2]), packed[1]
+ else:
+ # multiple array UDFs in a projection
+ yield [(create_array(*t), t[1]) for t in packed]
+
+ return super().dump_stream(py_to_batch(), stream)
+
+
class ArrowStreamPandasUDTFSerializer(ArrowStreamPandasUDFSerializer):
"""
Serializer used by Python worker to evaluate Arrow-optimized Python UDTFs.
"""
- def __init__(self, timezone, safecheck):
+ def __init__(self, timezone, safecheck, input_types, int_to_decimal_coercion_enabled):
super(ArrowStreamPandasUDTFSerializer, self).__init__(
timezone=timezone,
safecheck=safecheck,
@@ -720,6 +894,9 @@ def __init__(self, timezone, safecheck):
ndarray_as_list=True,
# Enables explicit casting for mismatched return types of Arrow Python UDTFs.
arrow_cast=True,
+ input_types=input_types,
+ # Enable additional coercions for UDTF serialization
+ int_to_decimal_coercion_enabled=int_to_decimal_coercion_enabled,
)
self._converter_map = dict()
@@ -742,35 +919,44 @@ def _create_batch(self, series):
import pandas as pd
import pyarrow as pa
- # Make input conform to [(series1, type1), (series2, type2), ...]
- if not isinstance(series, (list, tuple)) or (
- len(series) == 2 and isinstance(series[1], pa.DataType)
+ # Make input conform to
+ # [(series1, arrow_type1, spark_type1), (series2, arrow_type2, spark_type2), ...]
+ if (
+ not isinstance(series, (list, tuple))
+ or (len(series) == 2 and isinstance(series[1], pa.DataType))
+ or (
+ len(series) == 3
+ and isinstance(series[1], pa.DataType)
+ and isinstance(series[2], DataType)
+ )
):
series = [series]
series = ((s, None) if not isinstance(s, (list, tuple)) else s for s in series)
+ series = ((s[0], s[1], None) if len(s) == 2 else s for s in series)
arrs = []
- for s, t in series:
+ for s, arrow_type, spark_type in series:
if not isinstance(s, pd.DataFrame):
raise PySparkValueError(
"Output of an arrow-optimized Python UDTFs expects "
f"a pandas.DataFrame but got: {type(s)}"
)
- arrs.append(self._create_struct_array(s, t))
+ arrs.append(self._create_struct_array(s, arrow_type, spark_type))
return pa.RecordBatch.from_arrays(arrs, ["_%d" % i for i in range(len(arrs))])
def _get_or_create_converter_from_pandas(self, dt):
- if dt not in self._converter_map:
+ key = dt.json()
+ if key not in self._converter_map:
conv = _create_converter_from_pandas(
dt,
timezone=self._timezone,
error_on_duplicated_field_names=False,
ignore_unexpected_complex_type_values=True,
)
- self._converter_map[dt] = conv
- return self._converter_map[dt]
+ self._converter_map[key] = conv
+ return self._converter_map[key]
def _create_array(self, series, arrow_type, spark_type=None, arrow_cast=False):
"""
@@ -806,6 +992,9 @@ def _create_array(self, series, arrow_type, spark_type=None, arrow_cast=False):
conv = self._get_or_create_converter_from_pandas(dt)
series = conv(series)
+ if self._int_to_decimal_coercion_enabled:
+ series = self._apply_python_coercions(series, arrow_type)
+
if hasattr(series.array, "__arrow_array__"):
mask = None
else:
@@ -937,9 +1126,14 @@ def __init__(
state_object_schema,
arrow_max_records_per_batch,
prefers_large_var_types,
+ int_to_decimal_coercion_enabled,
):
super(ApplyInPandasWithStateSerializer, self).__init__(
- timezone, safecheck, assign_cols_by_name
+ timezone,
+ safecheck,
+ assign_cols_by_name,
+ int_to_decimal_coercion_enabled=int_to_decimal_coercion_enabled,
+ arrow_cast=True,
)
self.pickleSer = CPickleSerializer()
self.utf8_deserializer = UTF8Deserializer()
@@ -1307,9 +1501,20 @@ class TransformWithStateInPandasSerializer(ArrowStreamPandasUDFSerializer):
Limit of the number of records that can be written to a single ArrowRecordBatch in memory.
"""
- def __init__(self, timezone, safecheck, assign_cols_by_name, arrow_max_records_per_batch):
+ def __init__(
+ self,
+ timezone,
+ safecheck,
+ assign_cols_by_name,
+ arrow_max_records_per_batch,
+ int_to_decimal_coercion_enabled,
+ ):
super(TransformWithStateInPandasSerializer, self).__init__(
- timezone, safecheck, assign_cols_by_name
+ timezone,
+ safecheck,
+ assign_cols_by_name,
+ int_to_decimal_coercion_enabled=int_to_decimal_coercion_enabled,
+ arrow_cast=True,
)
self.arrow_max_records_per_batch = arrow_max_records_per_batch
self.key_offsets = None
@@ -1383,9 +1588,20 @@ class TransformWithStateInPandasInitStateSerializer(TransformWithStateInPandasSe
Same as input parameters in TransformWithStateInPandasSerializer.
"""
- def __init__(self, timezone, safecheck, assign_cols_by_name, arrow_max_records_per_batch):
+ def __init__(
+ self,
+ timezone,
+ safecheck,
+ assign_cols_by_name,
+ arrow_max_records_per_batch,
+ int_to_decimal_coercion_enabled,
+ ):
super(TransformWithStateInPandasInitStateSerializer, self).__init__(
- timezone, safecheck, assign_cols_by_name, arrow_max_records_per_batch
+ timezone,
+ safecheck,
+ assign_cols_by_name,
+ arrow_max_records_per_batch,
+ int_to_decimal_coercion_enabled,
)
self.init_key_offsets = None
@@ -1398,6 +1614,7 @@ def load_stream(self, stream):
def generate_data_batches(batches):
"""
Deserialize ArrowRecordBatches and return a generator of pandas.Series list.
+
The deserialization logic assumes that Arrow RecordBatches contain the data with the
ordering that data chunks for same grouping key will appear sequentially.
See `TransformWithStateInPandasPythonInitialStateRunner` for arrow batch schema sent
diff --git a/python/pyspark/sql/pandas/typehints.py b/python/pyspark/sql/pandas/typehints.py
index ba7801ceea559..f010489b95126 100644
--- a/python/pyspark/sql/pandas/typehints.py
+++ b/python/pyspark/sql/pandas/typehints.py
@@ -27,29 +27,23 @@
PandasGroupedAggUDFType,
ArrowScalarUDFType,
ArrowScalarIterUDFType,
+ ArrowGroupedAggUDFType,
)
-def infer_eval_type(
- sig: Signature, type_hints: Dict[str, Any]
-) -> Union[
- "PandasScalarUDFType",
- "PandasScalarIterUDFType",
- "PandasGroupedAggUDFType",
- "ArrowScalarUDFType",
- "ArrowScalarIterUDFType",
-]:
+def infer_pandas_eval_type(
+ sig: Signature,
+ type_hints: Dict[str, Any],
+) -> Optional[Union["PandasScalarUDFType", "PandasScalarIterUDFType", "PandasGroupedAggUDFType"]]:
"""
Infers the evaluation type in :class:`pyspark.util.PythonEvalType` from
:class:`inspect.Signature` instance and type hints.
"""
- from pyspark.sql.pandas.functions import PandasUDFType, ArrowUDFType
+ from pyspark.sql.pandas.functions import PandasUDFType
require_minimum_pandas_version()
- require_minimum_pyarrow_version()
import pandas as pd
- import pyarrow as pa
annotations = {}
for param in sig.parameters.values():
@@ -83,12 +77,8 @@ def infer_eval_type(
)
for a in parameters_sig
) and (return_annotation == pd.Series or return_annotation == pd.DataFrame)
-
- # pa.Array, ... -> pa.Array
- is_arrow_array = all(
- a == pa.Array or check_union_annotation(a, parameter_check_func=lambda na: na == pa.Array)
- for a in parameters_sig
- ) and (return_annotation == pa.Array)
+ if is_series_or_frame:
+ return PandasUDFType.SCALAR
# Iterator[Tuple[Series, Frame or Union[DataFrame, Series], ...] -> Iterator[Series or Frame]
is_iterator_tuple_series_or_frame = (
@@ -111,21 +101,8 @@ def infer_eval_type(
return_annotation, parameter_check_func=lambda a: a == pd.DataFrame or a == pd.Series
)
)
-
- # Iterator[Tuple[pa.Array, ...] -> Iterator[pa.Array]
- is_iterator_tuple_array = (
- len(parameters_sig) == 1
- and check_iterator_annotation( # Iterator
- parameters_sig[0],
- parameter_check_func=lambda a: check_tuple_annotation( # Tuple
- a,
- parameter_check_func=lambda ta: (ta == Ellipsis or ta == pa.Array),
- ),
- )
- and check_iterator_annotation(
- return_annotation, parameter_check_func=lambda a: a == pa.Array
- )
- )
+ if is_iterator_tuple_series_or_frame:
+ return PandasUDFType.SCALAR_ITER
# Iterator[Series, Frame or Union[DataFrame, Series]] -> Iterator[Series or Frame]
is_iterator_series_or_frame = (
@@ -144,18 +121,8 @@ def infer_eval_type(
return_annotation, parameter_check_func=lambda a: a == pd.DataFrame or a == pd.Series
)
)
-
- # Iterator[pa.Array] -> Iterator[pa.Array]
- is_iterator_array = (
- len(parameters_sig) == 1
- and check_iterator_annotation(
- parameters_sig[0],
- parameter_check_func=lambda a: (a == pd.Series or a == pa.Array),
- )
- and check_iterator_annotation(
- return_annotation, parameter_check_func=lambda a: a == pa.Array
- )
- )
+ if is_iterator_series_or_frame:
+ return PandasUDFType.SCALAR_ITER
# Series, Frame or Union[DataFrame, Series], ... -> Any
is_series_or_frame_agg = all(
@@ -174,23 +141,141 @@ def infer_eval_type(
and not check_iterator_annotation(return_annotation)
and not check_tuple_annotation(return_annotation)
)
+ if is_series_or_frame_agg:
+ return PandasUDFType.GROUPED_AGG
- if is_series_or_frame:
- return PandasUDFType.SCALAR
- elif is_arrow_array:
+ return None
+
+
+def infer_arrow_eval_type(
+ sig: Signature, type_hints: Dict[str, Any]
+) -> Optional[Union["ArrowScalarUDFType", "ArrowScalarIterUDFType", "ArrowGroupedAggUDFType"]]:
+ """
+ Infers the evaluation type in :class:`pyspark.util.PythonEvalType` from
+ :class:`inspect.Signature` instance and type hints.
+ """
+ from pyspark.sql.pandas.functions import ArrowUDFType
+
+ require_minimum_pyarrow_version()
+
+ import pyarrow as pa
+
+ annotations = {}
+ for param in sig.parameters.values():
+ if param.annotation is not param.empty:
+ annotations[param.name] = type_hints.get(param.name, param.annotation)
+
+ # Check if all arguments have type hints
+ parameters_sig = [
+ annotations[parameter] for parameter in sig.parameters if parameter in annotations
+ ]
+ if len(parameters_sig) != len(sig.parameters):
+ raise PySparkValueError(
+ errorClass="TYPE_HINT_SHOULD_BE_SPECIFIED",
+ messageParameters={"target": "all parameters", "sig": str(sig)},
+ )
+
+ # Check if the return has a type hint
+ return_annotation = type_hints.get("return", sig.return_annotation)
+ if sig.empty is return_annotation:
+ raise PySparkValueError(
+ errorClass="TYPE_HINT_SHOULD_BE_SPECIFIED",
+ messageParameters={"target": "the return type", "sig": str(sig)},
+ )
+
+ # pa.Array, ... -> pa.Array
+ is_arrow_array = all(a == pa.Array for a in parameters_sig) and (return_annotation == pa.Array)
+ if is_arrow_array:
return ArrowUDFType.SCALAR
- elif is_iterator_tuple_series_or_frame or is_iterator_series_or_frame:
- return PandasUDFType.SCALAR_ITER
- elif is_iterator_tuple_array or is_iterator_array:
+
+ # Iterator[Tuple[pa.Array, ...] -> Iterator[pa.Array]
+ is_iterator_tuple_array = (
+ len(parameters_sig) == 1
+ and check_iterator_annotation( # Iterator
+ parameters_sig[0],
+ parameter_check_func=lambda a: check_tuple_annotation( # Tuple
+ a,
+ parameter_check_func=lambda ta: (ta == Ellipsis or ta == pa.Array),
+ ),
+ )
+ and check_iterator_annotation(
+ return_annotation, parameter_check_func=lambda a: a == pa.Array
+ )
+ )
+ if is_iterator_tuple_array:
return ArrowUDFType.SCALAR_ITER
- elif is_series_or_frame_agg:
- return PandasUDFType.GROUPED_AGG
+
+ # Iterator[pa.Array] -> Iterator[pa.Array]
+ is_iterator_array = (
+ len(parameters_sig) == 1
+ and check_iterator_annotation(
+ parameters_sig[0],
+ parameter_check_func=lambda a: a == pa.Array,
+ )
+ and check_iterator_annotation(
+ return_annotation, parameter_check_func=lambda a: a == pa.Array
+ )
+ )
+ if is_iterator_array:
+ return ArrowUDFType.SCALAR_ITER
+
+ # pa.Array, ... -> Any
+ is_array_agg = all(a == pa.Array for a in parameters_sig) and (
+ return_annotation != pa.Array
+ and not check_iterator_annotation(return_annotation)
+ and not check_tuple_annotation(return_annotation)
+ )
+ if is_array_agg:
+ return ArrowUDFType.GROUPED_AGG
+
+ return None
+
+
+def infer_eval_type(
+ sig: Signature,
+ type_hints: Dict[str, Any],
+ kind: str = "all",
+) -> Union[
+ "PandasScalarUDFType",
+ "PandasScalarIterUDFType",
+ "PandasGroupedAggUDFType",
+ "ArrowScalarUDFType",
+ "ArrowScalarIterUDFType",
+ "ArrowGroupedAggUDFType",
+]:
+ """
+ Infers the evaluation type in :class:`pyspark.util.PythonEvalType` from
+ :class:`inspect.Signature` instance and type hints.
+ """
+ assert kind in ["pandas", "arrow", "all"], "kind should be either 'pandas', 'arrow' or 'all'"
+
+ eval_type: Optional[
+ Union[
+ "PandasScalarUDFType",
+ "PandasScalarIterUDFType",
+ "PandasGroupedAggUDFType",
+ "ArrowScalarUDFType",
+ "ArrowScalarIterUDFType",
+ "ArrowGroupedAggUDFType",
+ ]
+ ] = None
+ if kind == "pandas":
+ eval_type = infer_pandas_eval_type(sig, type_hints)
+ elif kind == "arrow":
+ eval_type = infer_arrow_eval_type(sig, type_hints)
else:
+ eval_type = infer_pandas_eval_type(sig, type_hints) or infer_arrow_eval_type(
+ sig, type_hints
+ )
+
+ if eval_type is None:
raise PySparkNotImplementedError(
errorClass="UNSUPPORTED_SIGNATURE",
messageParameters={"signature": str(sig)},
)
+ return eval_type
+
def check_tuple_annotation(
annotation: Any, parameter_check_func: Optional[Callable[[Any], bool]] = None
diff --git a/python/pyspark/sql/pandas/types.py b/python/pyspark/sql/pandas/types.py
index 2f82609c84292..586d86aa6a535 100644
--- a/python/pyspark/sql/pandas/types.py
+++ b/python/pyspark/sql/pandas/types.py
@@ -37,6 +37,7 @@
StringType,
BinaryType,
DateType,
+ TimeType,
TimestampType,
TimestampNTZType,
DayTimeIntervalType,
@@ -127,6 +128,8 @@ def to_arrow_type(
arrow_type = pa.timestamp("us", tz=None)
elif type(dt) == DayTimeIntervalType:
arrow_type = pa.duration("us")
+ elif type(dt) == TimeType:
+ arrow_type = pa.time64("ns")
elif type(dt) == ArrayType:
field = pa.field(
"element",
@@ -302,6 +305,8 @@ def from_arrow_type(at: "pa.DataType", prefer_timestamp_ntz: bool = False) -> Da
spark_type = BinaryType()
elif types.is_date32(at):
spark_type = DateType()
+ elif types.is_time64(at):
+ spark_type = TimeType()
elif types.is_timestamp(at) and prefer_timestamp_ntz and at.tz is None:
spark_type = TimestampNTZType()
elif types.is_timestamp(at):
@@ -311,14 +316,6 @@ def from_arrow_type(at: "pa.DataType", prefer_timestamp_ntz: bool = False) -> Da
elif types.is_list(at):
spark_type = ArrayType(from_arrow_type(at.value_type, prefer_timestamp_ntz))
elif types.is_fixed_size_list(at):
- import pyarrow as pa
-
- if LooseVersion(pa.__version__) < LooseVersion("14.0.0"):
- # PyArrow versions before 14.0.0 do not support casting FixedSizeListArray to ListArray
- raise PySparkTypeError(
- errorClass="UNSUPPORTED_DATA_TYPE_FOR_ARROW_CONVERSION",
- messageParameters={"data_type": str(at)},
- )
spark_type = ArrayType(from_arrow_type(at.value_type, prefer_timestamp_ntz))
elif types.is_large_list(at):
spark_type = ArrayType(from_arrow_type(at.value_type, prefer_timestamp_ntz))
diff --git a/python/pyspark/sql/pandas/utils.py b/python/pyspark/sql/pandas/utils.py
index a351c13ff0a08..c37665c719c88 100644
--- a/python/pyspark/sql/pandas/utils.py
+++ b/python/pyspark/sql/pandas/utils.py
@@ -22,7 +22,7 @@
def require_minimum_pandas_version() -> None:
"""Raise ImportError if minimum version of Pandas is not installed"""
# TODO(HyukjinKwon): Relocate and deduplicate the version specification.
- minimum_pandas_version = "2.0.0"
+ minimum_pandas_version = "2.2.0"
try:
import pandas
@@ -61,7 +61,7 @@ def require_minimum_pandas_version() -> None:
def require_minimum_pyarrow_version() -> None:
"""Raise ImportError if minimum version of pyarrow is not installed"""
# TODO(HyukjinKwon): Relocate and deduplicate the version specification.
- minimum_pyarrow_version = "11.0.0"
+ minimum_pyarrow_version = "15.0.0"
import os
@@ -98,7 +98,7 @@ def require_minimum_pyarrow_version() -> None:
def require_minimum_numpy_version() -> None:
"""Raise ImportError if minimum version of NumPy is not installed"""
- minimum_numpy_version = "1.21"
+ minimum_numpy_version = "1.22"
try:
import numpy
diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py
index 241424c954b2d..2cb0f2f59b1b9 100644
--- a/python/pyspark/sql/session.py
+++ b/python/pyspark/sql/session.py
@@ -54,6 +54,7 @@
DataType,
StructField,
StructType,
+ VariantVal,
_make_type_verifier,
_infer_schema,
_has_nulltype,
@@ -1194,10 +1195,14 @@ def _createFromLocal(
if not isinstance(data, list):
data = list(data)
+ if any(isinstance(d, VariantVal) for d in data):
+ raise PySparkValueError("Rows cannot be of type VariantVal")
+
+ tupled_data: Iterable[Tuple]
if schema is None or isinstance(schema, (list, tuple)):
struct = self._inferSchemaFromList(data, names=schema)
converter = _create_converter(struct)
- tupled_data: Iterable[Tuple] = map(converter, data)
+ tupled_data = map(converter, data)
if isinstance(schema, (list, tuple)):
for i, name in enumerate(schema):
struct.fields[i].name = name
@@ -1205,7 +1210,8 @@ def _createFromLocal(
elif isinstance(schema, StructType):
struct = schema
- tupled_data = data
+ converter = _create_converter(struct)
+ tupled_data = map(converter, data)
else:
raise PySparkTypeError(
diff --git a/python/pyspark/sql/streaming/readwriter.py b/python/pyspark/sql/streaming/readwriter.py
index 34af8cd9b070e..8121dd609950e 100644
--- a/python/pyspark/sql/streaming/readwriter.py
+++ b/python/pyspark/sql/streaming/readwriter.py
@@ -1549,7 +1549,7 @@ def foreach(self, f: Union[Callable[[Row], None], "SupportsProcess"]) -> "DataSt
... def close(self, error):
... print("Closed with error: %s" % str(error))
...
- >>> q = df.writeStream.foreach(print_row).start()
+ >>> q = df.writeStream.foreach(RowPrinter()).start()
>>> time.sleep(3)
>>> q.stop()
"""
diff --git a/python/pyspark/sql/streaming/stateful_processor_api_client.py b/python/pyspark/sql/streaming/stateful_processor_api_client.py
index 145e24ea2cd19..65e58e025b171 100644
--- a/python/pyspark/sql/streaming/stateful_processor_api_client.py
+++ b/python/pyspark/sql/streaming/stateful_processor_api_client.py
@@ -494,26 +494,34 @@ def _receive_str(self) -> str:
def _serialize_to_bytes(self, schema: StructType, data: Tuple) -> bytes:
from pyspark.testing.utils import have_numpy
- converted = []
-
if have_numpy:
import numpy as np
- # In order to convert NumPy types to Python primitive types.
- for v in data:
+ def normalize_value(v: Any) -> Any:
+ # Convert NumPy types to Python primitive types.
if isinstance(v, np.generic):
- converted.append(v.tolist())
+ return v.tolist()
+ # List / tuple: recursively normalize each element
+ if isinstance(v, (list, tuple)):
+ return type(v)(normalize_value(e) for e in v)
+ # Dict: normalize both keys and values
+ if isinstance(v, dict):
+ return {normalize_value(k): normalize_value(val) for k, val in v.items()}
# Address a couple of pandas dtypes too.
elif hasattr(v, "to_pytimedelta"):
- converted.append(v.to_pytimedelta())
+ return v.to_pytimedelta()
elif hasattr(v, "to_pydatetime"):
- converted.append(v.to_pydatetime())
+ return v.to_pydatetime()
else:
- converted.append(v)
+ return v
+
+ converted = [normalize_value(v) for v in data]
else:
converted = list(data)
- row_value = Row(*converted)
+ field_names = [f.name for f in schema.fields]
+ row_value = Row(**dict(zip(field_names, converted)))
+
return self.pickleSer.dumps(schema.toInternal(row_value))
def _deserialize_from_bytes(self, value: bytes) -> Any:
diff --git a/python/pyspark/sql/tests/arrow/test_arrow.py b/python/pyspark/sql/tests/arrow/test_arrow.py
index 5a770a947889b..c74ca121f26d3 100644
--- a/python/pyspark/sql/tests/arrow/test_arrow.py
+++ b/python/pyspark/sql/tests/arrow/test_arrow.py
@@ -707,8 +707,8 @@ def check_createDataFrame_with_single_data_type(self):
def test_createDataFrame_does_not_modify_input(self):
# Some series get converted for Spark to consume, this makes sure input is unchanged
pdf = self.create_pandas_data_frame()
- # Use a nanosecond value to make sure it is not truncated
- pdf.iloc[0, 7] = pd.Timestamp(1)
+ # Use a nanosecond value that converts to microseconds without precision loss
+ pdf.iloc[0, 7] = pd.Timestamp(1000)
# Integers with nulls will get NaNs filled with 0 and will be casted
pdf.iloc[1, 1] = None
pdf_copy = pdf.copy(deep=True)
@@ -1713,13 +1713,8 @@ def test_createDataFrame_arrow_fixed_size_binary(self):
def test_createDataFrame_arrow_fixed_size_list(self):
a = pa.array([[-1, 3]] * 5, type=pa.list_(pa.int32(), 2))
t = pa.table([a], ["fsl"])
- if LooseVersion(pa.__version__) < LooseVersion("14.0.0"):
- # PyArrow versions before 14.0.0 do not support casting FixedSizeListArray to ListArray
- with self.assertRaises(PySparkTypeError):
- df = self.spark.createDataFrame(t)
- else:
- df = self.spark.createDataFrame(t)
- self.assertIsInstance(df.schema["fsl"].dataType, ArrayType)
+ df = self.spark.createDataFrame(t)
+ self.assertIsInstance(df.schema["fsl"].dataType, ArrayType)
@unittest.skipIf(
diff --git a/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py b/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py
index 78e72e02836f9..a054261304c6f 100644
--- a/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py
+++ b/python/pyspark/sql/tests/arrow/test_arrow_python_udf.py
@@ -21,7 +21,7 @@
from pyspark.sql import Row
from pyspark.sql.functions import udf
from pyspark.sql.tests.test_udf import BaseUDFTestsMixin
-from pyspark.sql.types import VarcharType
+from pyspark.sql.types import DayTimeIntervalType, VarcharType, StructType, StructField, StringType
from pyspark.testing.sqlutils import (
have_pandas,
have_pyarrow,
@@ -180,6 +180,61 @@ def test_type_coercion_string_to_numeric(self):
with self.assertRaises(PythonException):
df_floating_value.select(udf(lambda x: x, "decimal")("value").alias("res")).collect()
+ def test_arrow_udf_int_to_decimal_coercion(self):
+ from decimal import Decimal
+
+ with self.sql_conf(
+ {"spark.sql.legacy.execution.pythonUDF.pandas.conversion.enabled": False}
+ ):
+ df = self.spark.range(0, 3)
+
+ @udf(returnType="decimal(10,2)", useArrow=True)
+ def int_to_decimal_udf(val):
+ values = [123, 456, 789]
+ return values[int(val) % len(values)]
+
+ # Test with coercion enabled
+ with self.sql_conf(
+ {"spark.sql.execution.pythonUDF.pandas.intToDecimalCoercionEnabled": True}
+ ):
+ result = df.select(int_to_decimal_udf("id").alias("decimal_val")).collect()
+ self.assertEqual(result[0]["decimal_val"], Decimal("123.00"))
+ self.assertEqual(result[1]["decimal_val"], Decimal("456.00"))
+ self.assertEqual(result[2]["decimal_val"], Decimal("789.00"))
+
+ # Test with coercion disabled (should fail)
+ with self.sql_conf(
+ {"spark.sql.execution.pythonUDF.pandas.intToDecimalCoercionEnabled": False}
+ ):
+ with self.assertRaisesRegex(
+ PythonException, "An exception was thrown from the Python worker"
+ ):
+ df.select(int_to_decimal_udf("id").alias("decimal_val")).collect()
+
+ @udf(returnType="decimal(25,1)", useArrow=True)
+ def high_precision_udf(val):
+ values = [1, 2, 3]
+ return values[int(val) % len(values)]
+
+ # Test high precision decimal with coercion enabled
+ with self.sql_conf(
+ {"spark.sql.execution.pythonUDF.pandas.intToDecimalCoercionEnabled": True}
+ ):
+ result = df.select(high_precision_udf("id").alias("decimal_val")).collect()
+ self.assertEqual(len(result), 3)
+ self.assertEqual(result[0]["decimal_val"], Decimal("1.0"))
+ self.assertEqual(result[1]["decimal_val"], Decimal("2.0"))
+ self.assertEqual(result[2]["decimal_val"], Decimal("3.0"))
+
+ # Test high precision decimal with coercion disabled (should fail)
+ with self.sql_conf(
+ {"spark.sql.execution.pythonUDF.pandas.intToDecimalCoercionEnabled": False}
+ ):
+ with self.assertRaisesRegex(
+ PythonException, "An exception was thrown from the Python worker"
+ ):
+ df.select(high_precision_udf("id").alias("decimal_val")).collect()
+
def test_err_return_type(self):
with self.assertRaises(PySparkNotImplementedError) as pe:
udf(lambda x: x, VarcharType(10), useArrow=True)
@@ -243,6 +298,88 @@ def test_udf_use_arrow_and_session_conf(self):
udf(lambda x: str(x), useArrow=False).evalType, PythonEvalType.SQL_BATCHED_UDF
)
+ def test_day_time_interval_type_casting(self):
+ """Test that DayTimeIntervalType UDFs work with Arrow and preserve field specifications."""
+
+ # HOUR TO SECOND
+ @udf(useArrow=True, returnType=DayTimeIntervalType(1, 3))
+ def return_interval(x):
+ return x
+
+ # UDF input: HOUR TO SECOND, UDF output: HOUR TO SECOND
+ df = self.spark.sql("SELECT INTERVAL '200:13:50.3' HOUR TO SECOND as value").select(
+ return_interval("value").alias("result")
+ )
+ self.assertEqual(df.schema.fields[0].dataType, DayTimeIntervalType(1, 3))
+ self.assertIsNotNone(df.collect()[0]["result"])
+
+ # UDF input: DAY TO SECOND, UDF output: HOUR TO SECOND
+ df2 = self.spark.sql("SELECT INTERVAL '1 10:30:45.123' DAY TO SECOND as value").select(
+ return_interval("value").alias("result")
+ )
+ self.assertEqual(df.schema.fields[0].dataType, DayTimeIntervalType(1, 3))
+ self.assertIsNotNone(df2.collect()[0]["result"])
+
+ def test_day_time_interval_in_struct(self):
+ """Test that DayTimeIntervalType works within StructType with Arrow UDFs."""
+
+ struct_type = StructType(
+ [
+ StructField("interval_field", DayTimeIntervalType(1, 3)),
+ StructField("name", StringType()),
+ ]
+ )
+
+ @udf(useArrow=True, returnType=struct_type)
+ def create_struct_with_interval(interval_val, name_val):
+ return Row(interval_field=interval_val, name=name_val)
+
+ df = self.spark.sql(
+ """
+ SELECT INTERVAL '15:30:45.678' HOUR TO SECOND as interval_val,
+ 'test_name' as name_val
+ """
+ ).select(create_struct_with_interval("interval_val", "name_val").alias("result"))
+
+ self.assertEqual(df.schema.fields[0].dataType, struct_type)
+ self.assertEqual(df.schema.fields[0].dataType.fields[0].dataType, DayTimeIntervalType(1, 3))
+ result = df.collect()[0]["result"]
+ self.assertIsNotNone(result["interval_field"])
+ self.assertEqual(result["name"], "test_name")
+
+
+@unittest.skipIf(
+ not have_pandas or not have_pyarrow, pandas_requirement_message or pyarrow_requirement_message
+)
+class ArrowPythonUDFLegacyTestsMixin(ArrowPythonUDFTestsMixin):
+ @classmethod
+ def setUpClass(cls):
+ super().setUpClass()
+ cls.spark.conf.set("spark.sql.legacy.execution.pythonUDF.pandas.conversion.enabled", "true")
+
+ @classmethod
+ def tearDownClass(cls):
+ try:
+ cls.spark.conf.unset("spark.sql.legacy.execution.pythonUDF.pandas.conversion.enabled")
+ finally:
+ super().tearDownClass()
+
+
+class ArrowPythonUDFNonLegacyTestsMixin(ArrowPythonUDFTestsMixin):
+ @classmethod
+ def setUpClass(cls):
+ super().setUpClass()
+ cls.spark.conf.set(
+ "spark.sql.legacy.execution.pythonUDF.pandas.conversion.enabled", "false"
+ )
+
+ @classmethod
+ def tearDownClass(cls):
+ try:
+ cls.spark.conf.unset("spark.sql.legacy.execution.pythonUDF.pandas.conversion.enabled")
+ finally:
+ super().tearDownClass()
+
class ArrowPythonUDFTests(ArrowPythonUDFTestsMixin, ReusedSQLTestCase):
@classmethod
@@ -258,18 +395,32 @@ def tearDownClass(cls):
super(ArrowPythonUDFTests, cls).tearDownClass()
-class AsyncArrowPythonUDFTests(ArrowPythonUDFTests):
+class ArrowPythonUDFLegacyTests(ArrowPythonUDFLegacyTestsMixin, ReusedSQLTestCase):
+ @classmethod
+ def setUpClass(cls):
+ super(ArrowPythonUDFLegacyTests, cls).setUpClass()
+ cls.spark.conf.set("spark.sql.execution.pythonUDF.arrow.enabled", "true")
+
+ @classmethod
+ def tearDownClass(cls):
+ try:
+ cls.spark.conf.unset("spark.sql.execution.pythonUDF.arrow.enabled")
+ finally:
+ super(ArrowPythonUDFLegacyTests, cls).tearDownClass()
+
+
+class ArrowPythonUDFNonLegacyTests(ArrowPythonUDFNonLegacyTestsMixin, ReusedSQLTestCase):
@classmethod
def setUpClass(cls):
- super(AsyncArrowPythonUDFTests, cls).setUpClass()
- cls.spark.conf.set("spark.sql.execution.pythonUDF.arrow.concurrency.level", "4")
+ super(ArrowPythonUDFNonLegacyTests, cls).setUpClass()
+ cls.spark.conf.set("spark.sql.execution.pythonUDF.arrow.enabled", "true")
@classmethod
def tearDownClass(cls):
try:
- cls.spark.conf.unset("spark.sql.execution.pythonUDF.arrow.concurrency.level")
+ cls.spark.conf.unset("spark.sql.execution.pythonUDF.arrow.enabled")
finally:
- super(AsyncArrowPythonUDFTests, cls).tearDownClass()
+ super(ArrowPythonUDFNonLegacyTests, cls).tearDownClass()
if __name__ == "__main__":
diff --git a/python/pyspark/sql/tests/arrow/test_arrow_udf.py b/python/pyspark/sql/tests/arrow/test_arrow_udf.py
index 052ee1c3be830..693d7ba484d80 100644
--- a/python/pyspark/sql/tests/arrow/test_arrow_udf.py
+++ b/python/pyspark/sql/tests/arrow/test_arrow_udf.py
@@ -15,12 +15,13 @@
# limitations under the License.
#
+import os
+import time
import unittest
import datetime
-# TODO: import arrow_udf from public API
-from pyspark.sql.pandas.functions import arrow_udf, ArrowUDFType, PandasUDFType
-from pyspark.sql import functions as F
+from pyspark.sql.functions import arrow_udf, ArrowUDFType, PandasUDFType
+from pyspark.sql import functions as F, Row
from pyspark.sql.types import (
DoubleType,
StructType,
@@ -98,6 +99,65 @@ def foo(x):
self.assertEqual(foo.returnType, DoubleType())
self.assertEqual(foo.evalType, PythonEvalType.SQL_SCALAR_ARROW_UDF)
+ def test_time_zone_against_map_in_arrow(self):
+ import pyarrow as pa
+
+ for tz in [
+ "Asia/Shanghai",
+ "Asia/Hong_Kong",
+ "America/Los_Angeles",
+ "Pacific/Honolulu",
+ "Europe/Amsterdam",
+ "US/Pacific",
+ ]:
+ with self.sql_conf({"spark.sql.session.timeZone": tz}):
+ # There is a time-zone conversion in df.collect:
+ # ts.astimezone().replace(tzinfo=None)
+ # it is controlled by env os.environ["TZ"].
+ # Note that if the env is not equvilent to spark.sql.session.timeZone,
+ # than there is a mismatch between the internal arrow data and df.collect.
+ os.environ["TZ"] = tz
+ time.tzset()
+
+ df = self.spark.sql("SELECT TIMESTAMP('2019-04-12 15:50:01') AS ts")
+
+ def check_value(t):
+ assert isinstance(t, pa.Array)
+ assert isinstance(t, pa.TimestampArray)
+ assert isinstance(t[0], pa.Scalar)
+ assert isinstance(t[0], pa.TimestampScalar)
+ ts = t[0].as_py()
+ assert isinstance(ts, datetime.datetime)
+ assert ts.year == 2019
+ assert ts.month == 4
+ assert ts.day == 12
+ assert ts.hour == 15
+ assert ts.minute == 50
+ assert ts.second == 1
+ # the timezone is still kept in the internal arrow data
+ assert ts.tzinfo is not None
+ assert str(ts.tzinfo) == tz, str(ts.tzinfo)
+
+ @arrow_udf("timestamp")
+ def identity(t):
+ check_value(t)
+ return t
+
+ expected = [Row(ts=datetime.datetime(2019, 4, 12, 15, 50, 1))]
+ self.assertEqual(expected, df.collect())
+
+ result1 = df.select(identity("ts").alias("ts"))
+ self.assertEqual(expected, result1.collect())
+
+ def identity2(iter):
+ for batch in iter:
+ t = batch["ts"]
+ check_value(t)
+ yield batch
+
+ result2 = df.mapInArrow(identity2, "ts timestamp")
+ self.assertEqual(expected, result2.collect())
+
def test_arrow_udf_wrong_arg(self):
with self.quiet():
with self.assertRaises(ParseException):
@@ -166,6 +226,13 @@ def foo(df):
def zero_with_type():
return 1
+ with self.assertRaisesRegex(ValueError, "0-arg arrow_udfs.*not.*supported"):
+
+ @arrow_udf(LongType(), ArrowUDFType.SCALAR_ITER)
+ def zero_with_type():
+ yield 1
+ yield 2
+
def test_arrow_udf_timestamp_ntz(self):
import pyarrow as pa
@@ -207,7 +274,10 @@ def noop(s: pa.Array) -> pa.Array:
class ArrowUDFTests(ArrowUDFTestsMixin, ReusedSQLTestCase):
- pass
+ def setUp(self):
+ tz = "America/Los_Angeles"
+ os.environ["TZ"] = tz
+ time.tzset()
if __name__ == "__main__":
diff --git a/python/pyspark/sql/tests/arrow/test_arrow_udf_grouped_agg.py b/python/pyspark/sql/tests/arrow/test_arrow_udf_grouped_agg.py
new file mode 100644
index 0000000000000..3545801c4b5aa
--- /dev/null
+++ b/python/pyspark/sql/tests/arrow/test_arrow_udf_grouped_agg.py
@@ -0,0 +1,883 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+import unittest
+
+from pyspark.sql.functions import arrow_udf, ArrowUDFType
+from pyspark.util import PythonEvalType
+from pyspark.sql import Row
+from pyspark.sql.types import ArrayType, YearMonthIntervalType
+from pyspark.sql import functions as sf
+from pyspark.errors import AnalysisException, PythonException
+from pyspark.testing.sqlutils import (
+ ReusedSQLTestCase,
+ have_pyarrow,
+ pyarrow_requirement_message,
+)
+
+
+@unittest.skipIf(not have_pyarrow, pyarrow_requirement_message)
+class GroupedAggArrowUDFTestsMixin:
+ @property
+ def data(self):
+ return (
+ self.spark.range(10)
+ .toDF("id")
+ .withColumn("vs", sf.array([sf.lit(i * 1.0) + sf.col("id") for i in range(20, 30)]))
+ .withColumn("v", sf.explode(sf.col("vs")))
+ .drop("vs")
+ .withColumn("w", sf.lit(1.0))
+ )
+
+ @property
+ def python_plus_one(self):
+ @sf.udf("double")
+ def plus_one(v):
+ assert isinstance(v, (int, float))
+ return float(v + 1)
+
+ return plus_one
+
+ @property
+ def arrow_scalar_plus_two(self):
+ import pyarrow as pa
+
+ @arrow_udf("double", ArrowUDFType.SCALAR)
+ def plus_two(v):
+ assert isinstance(v, pa.Array)
+ return pa.compute.add(v, 2).cast(pa.float64())
+
+ return plus_two
+
+ @property
+ def arrow_agg_mean_udf(self):
+ import pyarrow as pa
+
+ @arrow_udf("double", ArrowUDFType.GROUPED_AGG)
+ def avg(v):
+ assert isinstance(v, pa.Array)
+ return pa.compute.mean(v.cast(pa.float64()))
+
+ return avg
+
+ @property
+ def arrow_agg_mean_arr_udf(self):
+ import pyarrow as pa
+
+ @arrow_udf("array", ArrowUDFType.GROUPED_AGG)
+ def avg(v):
+ assert isinstance(v, pa.Array)
+ assert isinstance(v, pa.ListArray)
+ return [pa.compute.mean(v.flatten()).cast(pa.float64())]
+
+ return avg
+
+ @property
+ def arrow_agg_sum_udf(self):
+ import pyarrow as pa
+
+ @arrow_udf("double", ArrowUDFType.GROUPED_AGG)
+ def sum(v):
+ assert isinstance(v, pa.Array)
+ return pa.compute.sum(v).cast(pa.float64())
+
+ return sum
+
+ @property
+ def arrow_agg_weighted_mean_udf(self):
+ import pyarrow as pa
+ import numpy as np
+
+ @arrow_udf("double", ArrowUDFType.GROUPED_AGG)
+ def weighted_mean(v, w):
+ assert isinstance(v, pa.Array)
+ assert isinstance(w, pa.Array)
+ return float(np.average(v, weights=w))
+
+ return weighted_mean
+
+ def test_manual(self):
+ df = self.data
+ sum_udf = self.arrow_agg_sum_udf
+ mean_udf = self.arrow_agg_mean_udf
+ mean_arr_udf = self.arrow_agg_mean_arr_udf
+
+ result = (
+ df.groupby("id")
+ .agg(sum_udf(df.v), mean_udf(df.v), mean_arr_udf(sf.array(df.v)))
+ .sort("id")
+ )
+ expected = self.spark.createDataFrame(
+ [
+ [0, 245.0, 24.5, [24.5]],
+ [1, 255.0, 25.5, [25.5]],
+ [2, 265.0, 26.5, [26.5]],
+ [3, 275.0, 27.5, [27.5]],
+ [4, 285.0, 28.5, [28.5]],
+ [5, 295.0, 29.5, [29.5]],
+ [6, 305.0, 30.5, [30.5]],
+ [7, 315.0, 31.5, [31.5]],
+ [8, 325.0, 32.5, [32.5]],
+ [9, 335.0, 33.5, [33.5]],
+ ],
+ ["id", "sum(v)", "avg(v)", "avg(array(v))"],
+ ).collect()
+
+ self.assertEqual(expected, result.collect())
+
+ def test_basic(self):
+ df = self.data
+ weighted_mean_udf = self.arrow_agg_weighted_mean_udf
+
+ # Groupby one column and aggregate one UDF with literal
+ result1 = df.groupby("id").agg(weighted_mean_udf(df.v, sf.lit(1.0))).sort("id")
+ expected1 = (
+ df.groupby("id").agg(sf.mean(df.v).alias("weighted_mean(v, 1.0)")).sort("id").collect()
+ )
+ self.assertEqual(expected1, result1.collect())
+
+ # Groupby one expression and aggregate one UDF with literal
+ result2 = (
+ df.groupby((sf.col("id") + 1)).agg(weighted_mean_udf(df.v, sf.lit(1.0))).sort(df.id + 1)
+ )
+ expected2 = (
+ df.groupby((sf.col("id") + 1))
+ .agg(sf.mean(df.v).alias("weighted_mean(v, 1.0)"))
+ .sort(df.id + 1)
+ ).collect()
+ self.assertEqual(expected2, result2.collect())
+
+ # Groupby one column and aggregate one UDF without literal
+ result3 = df.groupby("id").agg(weighted_mean_udf(df.v, df.w)).sort("id")
+ expected3 = (
+ df.groupby("id").agg(sf.mean(df.v).alias("weighted_mean(v, w)")).sort("id").collect()
+ )
+ self.assertEqual(expected3, result3.collect())
+
+ # Groupby one expression and aggregate one UDF without literal
+ result4 = (
+ df.groupby((sf.col("id") + 1).alias("id")).agg(weighted_mean_udf(df.v, df.w)).sort("id")
+ )
+ expected4 = (
+ df.groupby((sf.col("id") + 1).alias("id"))
+ .agg(sf.mean(df.v).alias("weighted_mean(v, w)"))
+ .sort("id")
+ ).collect()
+ self.assertEqual(expected4, result4.collect())
+
+ def test_alias(self):
+ df = self.data
+ mean_udf = self.arrow_agg_mean_udf
+
+ result = df.groupby("id").agg(mean_udf(df.v).alias("mean_alias"))
+ expected = df.groupby("id").agg(sf.mean(df.v).alias("mean_alias")).collect()
+
+ self.assertEqual(expected, result.collect())
+
+ def test_mixed_sql(self):
+ """
+ Test mixing group aggregate pandas UDF with sql expression.
+ """
+ df = self.data
+ sum_udf = self.arrow_agg_sum_udf
+
+ # Mix group aggregate pandas UDF with sql expression
+ result1 = df.groupby("id").agg(sum_udf(df.v) + 1).sort("id")
+ expected1 = df.groupby("id").agg(sf.sum(df.v) + 1).sort("id").collect()
+
+ # Mix group aggregate pandas UDF with sql expression (order swapped)
+ result2 = df.groupby("id").agg(sum_udf(df.v + 1)).sort("id")
+
+ expected2 = df.groupby("id").agg(sf.sum(df.v + 1)).sort("id").collect()
+
+ # Wrap group aggregate pandas UDF with two sql expressions
+ result3 = df.groupby("id").agg(sum_udf(df.v + 1) + 2).sort("id")
+ expected3 = df.groupby("id").agg(sf.sum(df.v + 1) + 2).sort("id").collect()
+
+ self.assertEqual(expected1, result1.collect())
+ self.assertEqual(expected2, result2.collect())
+ self.assertEqual(expected3, result3.collect())
+
+ def test_mixed_udfs(self):
+ """
+ Test mixing group aggregate pandas UDF with python UDF and scalar pandas UDF.
+ """
+ df = self.data
+ plus_one = self.python_plus_one
+ plus_two = self.arrow_scalar_plus_two
+ sum_udf = self.arrow_agg_sum_udf
+
+ # Mix group aggregate pandas UDF and python UDF
+ result1 = df.groupby("id").agg(plus_one(sum_udf(df.v))).sort("id")
+ expected1 = df.groupby("id").agg(plus_one(sf.sum(df.v))).sort("id").collect()
+
+ # Mix group aggregate pandas UDF and python UDF (order swapped)
+ result2 = df.groupby("id").agg(sum_udf(plus_one(df.v))).sort("id")
+ expected2 = df.groupby("id").agg(sf.sum(plus_one(df.v))).sort("id").collect()
+
+ # Mix group aggregate pandas UDF and scalar pandas UDF
+ result3 = df.groupby("id").agg(sum_udf(plus_two(df.v))).sort("id")
+ expected3 = df.groupby("id").agg(sf.sum(plus_two(df.v))).sort("id").collect()
+
+ # Mix group aggregate pandas UDF and scalar pandas UDF (order swapped)
+ result4 = df.groupby("id").agg(plus_two(sum_udf(df.v))).sort("id")
+ expected4 = df.groupby("id").agg(plus_two(sf.sum(df.v))).sort("id").collect()
+
+ # Wrap group aggregate pandas UDF with two python UDFs and use python UDF in groupby
+ result5 = (
+ df.groupby(plus_one(df.id)).agg(plus_one(sum_udf(plus_one(df.v)))).sort("plus_one(id)")
+ )
+ expected5 = (
+ df.groupby(plus_one(df.id)).agg(plus_one(sf.sum(plus_one(df.v)))).sort("plus_one(id)")
+ ).collect()
+
+ # Wrap group aggregate pandas UDF with two scala pandas UDF and user scala pandas UDF in
+ # groupby
+ result6 = (
+ df.groupby(plus_two(df.id)).agg(plus_two(sum_udf(plus_two(df.v)))).sort("plus_two(id)")
+ )
+ expected6 = (
+ df.groupby(plus_two(df.id)).agg(plus_two(sf.sum(plus_two(df.v)))).sort("plus_two(id)")
+ ).collect()
+
+ self.assertEqual(expected1, result1.collect())
+ self.assertEqual(expected2, result2.collect())
+ self.assertEqual(expected3, result3.collect())
+ self.assertEqual(expected4, result4.collect())
+ self.assertEqual(expected5, result5.collect())
+ self.assertEqual(expected6, result6.collect())
+
+ def test_multiple_udfs(self):
+ """
+ Test multiple group aggregate pandas UDFs in one agg function.
+ """
+ df = self.data
+ mean_udf = self.arrow_agg_mean_udf
+ sum_udf = self.arrow_agg_sum_udf
+ weighted_mean_udf = self.arrow_agg_weighted_mean_udf
+
+ result = (
+ df.groupBy("id")
+ .agg(mean_udf(df.v), sum_udf(df.v), weighted_mean_udf(df.v, df.w))
+ .sort("id")
+ )
+ expected = (
+ df.groupBy("id")
+ .agg(sf.mean(df.v), sf.sum(df.v), sf.mean(df.v).alias("weighted_mean(v, w)"))
+ .sort("id")
+ .collect()
+ )
+
+ self.assertEqual(expected, result.collect())
+
+ def test_complex_groupby(self):
+ df = self.data
+ sum_udf = self.arrow_agg_sum_udf
+ plus_one = self.python_plus_one
+ plus_two = self.arrow_scalar_plus_two
+
+ # groupby one expression
+ result1 = df.groupby(df.v % 2).agg(sum_udf(df.v))
+ expected1 = df.groupby(df.v % 2).agg(sf.sum(df.v)).collect()
+
+ # empty groupby
+ result2 = df.groupby().agg(sum_udf(df.v))
+ expected2 = df.groupby().agg(sf.sum(df.v)).collect()
+
+ # groupby one column and one sql expression
+ result3 = df.groupby(df.id, df.v % 2).agg(sum_udf(df.v)).orderBy(df.id, df.v % 2)
+ expected3 = df.groupby(df.id, df.v % 2).agg(sf.sum(df.v)).orderBy(df.id, df.v % 2).collect()
+
+ # groupby one python UDF
+ result4 = df.groupby(plus_one(df.id)).agg(sum_udf(df.v)).sort("plus_one(id)")
+ expected4 = df.groupby(plus_one(df.id)).agg(sf.sum(df.v)).sort("plus_one(id)").collect()
+
+ # groupby one scalar pandas UDF
+ result5 = df.groupby(plus_two(df.id)).agg(sum_udf(df.v)).sort("sum(v)")
+ expected5 = df.groupby(plus_two(df.id)).agg(sf.sum(df.v)).sort("sum(v)").collect()
+
+ # groupby one expression and one python UDF
+ result6 = (
+ df.groupby(df.v % 2, plus_one(df.id))
+ .agg(sum_udf(df.v))
+ .sort(["(v % 2)", "plus_one(id)"])
+ )
+ expected6 = (
+ df.groupby(df.v % 2, plus_one(df.id))
+ .agg(sf.sum(df.v))
+ .sort(["(v % 2)", "plus_one(id)"])
+ ).collect()
+
+ # groupby one expression and one scalar pandas UDF
+ result7 = (
+ df.groupby(df.v % 2, plus_two(df.id))
+ .agg(sum_udf(df.v))
+ .sort(["sum(v)", "plus_two(id)"])
+ )
+ expected7 = (
+ df.groupby(df.v % 2, plus_two(df.id)).agg(sf.sum(df.v)).sort(["sum(v)", "plus_two(id)"])
+ ).collect()
+
+ self.assertEqual(expected1, result1.collect())
+ self.assertEqual(expected2, result2.collect())
+ self.assertEqual(expected3, result3.collect())
+ self.assertEqual(expected4, result4.collect())
+ self.assertEqual(expected5, result5.collect())
+ self.assertEqual(expected6, result6.collect())
+ self.assertEqual(expected7, result7.collect())
+
+ def test_complex_expressions(self):
+ df = self.data
+ plus_one = self.python_plus_one
+ plus_two = self.arrow_scalar_plus_two
+ sum_udf = self.arrow_agg_sum_udf
+
+ # Test complex expressions with sql expression, python UDF and
+ # group aggregate pandas UDF
+ result1 = (
+ df.withColumn("v1", plus_one(df.v))
+ .withColumn("v2", df.v + 2)
+ .groupby(df.id, df.v % 2)
+ .agg(
+ sum_udf(sf.col("v")),
+ sum_udf(sf.col("v1") + 3),
+ sum_udf(sf.col("v2")) + 5,
+ plus_one(sum_udf(sf.col("v1"))),
+ sum_udf(plus_one(sf.col("v2"))),
+ )
+ .sort(["id", "(v % 2)"])
+ )
+
+ expected1 = (
+ df.withColumn("v1", df.v + 1)
+ .withColumn("v2", df.v + 2)
+ .groupby(df.id, df.v % 2)
+ .agg(
+ sf.sum(sf.col("v")),
+ sf.sum(sf.col("v1") + 3),
+ sf.sum(sf.col("v2")) + 5,
+ plus_one(sf.sum(sf.col("v1"))),
+ sf.sum(plus_one(sf.col("v2"))),
+ )
+ .sort(["id", "(v % 2)"])
+ .collect()
+ )
+
+ # Test complex expressions with sql expression, scala pandas UDF and
+ # group aggregate pandas UDF
+ result2 = (
+ df.withColumn("v1", plus_one(df.v))
+ .withColumn("v2", df.v + 2)
+ .groupby(df.id, df.v % 2)
+ .agg(
+ sum_udf(sf.col("v")),
+ sum_udf(sf.col("v1") + 3),
+ sum_udf(sf.col("v2")) + 5,
+ plus_two(sum_udf(sf.col("v1"))),
+ sum_udf(plus_two(sf.col("v2"))),
+ )
+ .sort(["id", "(v % 2)"])
+ )
+
+ expected2 = (
+ df.withColumn("v1", df.v + 1)
+ .withColumn("v2", df.v + 2)
+ .groupby(df.id, df.v % 2)
+ .agg(
+ sf.sum(sf.col("v")),
+ sf.sum(sf.col("v1") + 3),
+ sf.sum(sf.col("v2")) + 5,
+ plus_two(sf.sum(sf.col("v1"))),
+ sf.sum(plus_two(sf.col("v2"))),
+ )
+ .sort(["id", "(v % 2)"])
+ .collect()
+ )
+
+ # Test sequential groupby aggregate
+ result3 = (
+ df.groupby("id")
+ .agg(sum_udf(df.v).alias("v"))
+ .groupby("id")
+ .agg(sum_udf(sf.col("v")))
+ .sort("id")
+ )
+
+ expected3 = (
+ df.groupby("id")
+ .agg(sf.sum(df.v).alias("v"))
+ .groupby("id")
+ .agg(sf.sum(sf.col("v")))
+ .sort("id")
+ .collect()
+ )
+
+ self.assertEqual(expected1, result1.collect())
+ self.assertEqual(expected2, result2.collect())
+ self.assertEqual(expected3, result3.collect())
+
+ def test_retain_group_columns(self):
+ with self.sql_conf({"spark.sql.retainGroupColumns": False}):
+ df = self.data
+ sum_udf = self.arrow_agg_sum_udf
+
+ result1 = df.groupby(df.id).agg(sum_udf(df.v))
+ expected1 = df.groupby(df.id).agg(sf.sum(df.v)).collect()
+ self.assertEqual(expected1, result1.collect())
+
+ def test_array_type(self):
+ df = self.data
+
+ array_udf = arrow_udf(lambda x: [1.0, 2.0], "array", ArrowUDFType.GROUPED_AGG)
+ result1 = df.groupby("id").agg(array_udf(df["v"]).alias("v2"))
+ self.assertEqual(result1.first()["v2"], [1.0, 2.0])
+
+ def test_invalid_args(self):
+ with self.quiet():
+ self.check_invalid_args()
+
+ def check_invalid_args(self):
+ df = self.data
+ plus_one = self.python_plus_one
+ mean_udf = self.arrow_agg_mean_udf
+ with self.assertRaisesRegex(AnalysisException, "[MISSING_AGGREGATION]"):
+ df.groupby(df.id).agg(plus_one(df.v)).collect()
+ with self.assertRaisesRegex(
+ AnalysisException, "aggregate function.*argument.*aggregate function"
+ ):
+ df.groupby(df.id).agg(mean_udf(mean_udf(df.v))).collect()
+ with self.assertRaisesRegex(
+ AnalysisException,
+ "The group aggregate pandas UDF `avg` cannot be invoked together with as other, "
+ "non-pandas aggregate functions.",
+ ):
+ df.groupby(df.id).agg(mean_udf(df.v), sf.mean(df.v)).collect()
+
+ def test_register_vectorized_udf_basic(self):
+ import pyarrow as pa
+
+ sum_arrow_udf = arrow_udf(
+ lambda v: pa.compute.sum(v).cast(pa.int32()),
+ "integer",
+ PythonEvalType.SQL_GROUPED_AGG_ARROW_UDF,
+ )
+
+ self.assertEqual(sum_arrow_udf.evalType, PythonEvalType.SQL_GROUPED_AGG_ARROW_UDF)
+ group_agg_pandas_udf = self.spark.udf.register("sum_arrow_udf", sum_arrow_udf)
+ self.assertEqual(group_agg_pandas_udf.evalType, PythonEvalType.SQL_GROUPED_AGG_ARROW_UDF)
+ q = "SELECT sum_arrow_udf(v1) FROM VALUES (3, 0), (2, 0), (1, 1) tbl(v1, v2) GROUP BY v2"
+ actual = sorted(map(lambda r: r[0], self.spark.sql(q).collect()))
+ expected = [1, 5]
+ self.assertEqual(actual, expected)
+
+ def test_grouped_with_empty_partition(self):
+ import pyarrow as pa
+
+ data = [Row(id=1, x=2), Row(id=1, x=3), Row(id=2, x=4)]
+ expected = [Row(id=1, sum=5), Row(id=2, x=4)]
+ num_parts = len(data) + 1
+ df = self.spark.createDataFrame(data).repartition(num_parts)
+
+ f = arrow_udf(lambda x: pa.compute.sum(x).cast(pa.int32()), "int", ArrowUDFType.GROUPED_AGG)
+
+ result = df.groupBy("id").agg(f(df["x"]).alias("sum")).sort("id")
+ self.assertEqual(result.collect(), expected)
+
+ def test_grouped_without_group_by_clause(self):
+ import pyarrow as pa
+
+ @arrow_udf("double", ArrowUDFType.GROUPED_AGG)
+ def max_udf(v):
+ return float(pa.compute.max(v).as_py())
+
+ df = self.spark.range(0, 100)
+ self.spark.udf.register("max_udf", max_udf)
+
+ with self.tempView("table"):
+ df.createTempView("table")
+
+ agg1 = df.agg(max_udf(df["id"]))
+ agg2 = self.spark.sql("select max_udf(id) from table")
+ self.assertEqual(agg1.collect(), agg2.collect())
+
+ def test_no_predicate_pushdown_through(self):
+ import pyarrow as pa
+
+ @arrow_udf("float", ArrowUDFType.GROUPED_AGG)
+ def mean(x):
+ return pa.compute.mean(x).cast(pa.float32())
+
+ df = self.spark.createDataFrame([Row(id=1, foo=42), Row(id=2, foo=1), Row(id=2, foo=2)])
+
+ agg = df.groupBy("id").agg(mean("foo").alias("mean"))
+ filtered = agg.filter(agg["mean"] > 40.0)
+
+ self.assertEqual(filtered.collect()[0]["mean"], 42.0)
+
+ assert filtered.collect()[0]["mean"] == 42.0
+
+ def test_named_arguments(self):
+ df = self.data
+ weighted_mean = self.arrow_agg_weighted_mean_udf
+
+ with self.tempView("v"):
+ df.createOrReplaceTempView("v")
+ self.spark.udf.register("weighted_mean", weighted_mean)
+
+ for i, aggregated in enumerate(
+ [
+ df.groupby("id").agg(weighted_mean(df.v, w=df.w).alias("wm")),
+ df.groupby("id").agg(weighted_mean(v=df.v, w=df.w).alias("wm")),
+ df.groupby("id").agg(weighted_mean(w=df.w, v=df.v).alias("wm")),
+ self.spark.sql("SELECT id, weighted_mean(v, w => w) as wm FROM v GROUP BY id"),
+ self.spark.sql(
+ "SELECT id, weighted_mean(v => v, w => w) as wm FROM v GROUP BY id"
+ ),
+ self.spark.sql(
+ "SELECT id, weighted_mean(w => w, v => v) as wm FROM v GROUP BY id"
+ ),
+ ]
+ ):
+ with self.subTest(query_no=i):
+ self.assertEqual(
+ aggregated.collect(),
+ df.groupby("id").agg(sf.mean(df.v).alias("wm")).collect(),
+ )
+
+ def test_named_arguments_negative(self):
+ df = self.data
+ weighted_mean = self.arrow_agg_weighted_mean_udf
+
+ with self.tempView("v"):
+ df.createOrReplaceTempView("v")
+ self.spark.udf.register("weighted_mean", weighted_mean)
+
+ with self.assertRaisesRegex(
+ AnalysisException,
+ "DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT.DOUBLE_NAMED_ARGUMENT_REFERENCE",
+ ):
+ self.spark.sql(
+ "SELECT id, weighted_mean(v => v, v => w) as wm FROM v GROUP BY id"
+ ).show()
+
+ with self.assertRaisesRegex(AnalysisException, "UNEXPECTED_POSITIONAL_ARGUMENT"):
+ self.spark.sql(
+ "SELECT id, weighted_mean(v => v, w) as wm FROM v GROUP BY id"
+ ).show()
+
+ with self.assertRaisesRegex(
+ PythonException, r"weighted_mean\(\) got an unexpected keyword argument 'x'"
+ ):
+ self.spark.sql(
+ "SELECT id, weighted_mean(v => v, x => w) as wm FROM v GROUP BY id"
+ ).show()
+
+ with self.assertRaisesRegex(
+ PythonException, r"weighted_mean\(\) got multiple values for argument 'v'"
+ ):
+ self.spark.sql(
+ "SELECT id, weighted_mean(v, v => w) as wm FROM v GROUP BY id"
+ ).show()
+
+ def test_kwargs(self):
+ df = self.data
+
+ @arrow_udf("double", ArrowUDFType.GROUPED_AGG)
+ def weighted_mean(**kwargs):
+ import numpy as np
+
+ return np.average(kwargs["v"], weights=kwargs["w"])
+
+ with self.tempView("v"):
+ df.createOrReplaceTempView("v")
+ self.spark.udf.register("weighted_mean", weighted_mean)
+
+ for i, aggregated in enumerate(
+ [
+ df.groupby("id").agg(weighted_mean(v=df.v, w=df.w).alias("wm")),
+ df.groupby("id").agg(weighted_mean(w=df.w, v=df.v).alias("wm")),
+ self.spark.sql(
+ "SELECT id, weighted_mean(v => v, w => w) as wm FROM v GROUP BY id"
+ ),
+ self.spark.sql(
+ "SELECT id, weighted_mean(w => w, v => v) as wm FROM v GROUP BY id"
+ ),
+ ]
+ ):
+ with self.subTest(query_no=i):
+ self.assertEqual(
+ aggregated.collect(),
+ df.groupby("id").agg(sf.mean(df.v).alias("wm")).collect(),
+ )
+
+ # negative
+ with self.assertRaisesRegex(
+ AnalysisException,
+ "DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT.DOUBLE_NAMED_ARGUMENT_REFERENCE",
+ ):
+ self.spark.sql(
+ "SELECT id, weighted_mean(v => v, v => w) as wm FROM v GROUP BY id"
+ ).show()
+
+ with self.assertRaisesRegex(AnalysisException, "UNEXPECTED_POSITIONAL_ARGUMENT"):
+ self.spark.sql(
+ "SELECT id, weighted_mean(v => v, w) as wm FROM v GROUP BY id"
+ ).show()
+
+ def test_named_arguments_and_defaults(self):
+ import pyarrow as pa
+
+ df = self.data
+
+ @arrow_udf("double", ArrowUDFType.GROUPED_AGG)
+ def biased_sum(v, w=None):
+ return pa.compute.sum(v).as_py() + (pa.compute.sum(w).as_py() if w is not None else 100)
+
+ with self.tempView("v"):
+ df.createOrReplaceTempView("v")
+ self.spark.udf.register("biased_sum", biased_sum)
+
+ # without "w"
+ for i, aggregated in enumerate(
+ [
+ df.groupby("id").agg(biased_sum(df.v).alias("s")),
+ df.groupby("id").agg(biased_sum(v=df.v).alias("s")),
+ self.spark.sql("SELECT id, biased_sum(v) as s FROM v GROUP BY id"),
+ self.spark.sql("SELECT id, biased_sum(v => v) as s FROM v GROUP BY id"),
+ ]
+ ):
+ with self.subTest(with_w=False, query_no=i):
+ self.assertEqual(
+ aggregated.collect(),
+ df.groupby("id").agg((sf.sum(df.v) + sf.lit(100)).alias("s")).collect(),
+ )
+
+ # with "w"
+ for i, aggregated in enumerate(
+ [
+ df.groupby("id").agg(biased_sum(df.v, w=df.w).alias("s")),
+ df.groupby("id").agg(biased_sum(v=df.v, w=df.w).alias("s")),
+ df.groupby("id").agg(biased_sum(w=df.w, v=df.v).alias("s")),
+ self.spark.sql("SELECT id, biased_sum(v, w => w) as s FROM v GROUP BY id"),
+ self.spark.sql("SELECT id, biased_sum(v => v, w => w) as s FROM v GROUP BY id"),
+ self.spark.sql("SELECT id, biased_sum(w => w, v => v) as s FROM v GROUP BY id"),
+ ]
+ ):
+ with self.subTest(with_w=True, query_no=i):
+ self.assertEqual(
+ aggregated.collect(),
+ df.groupby("id").agg((sf.sum(df.v) + sf.sum(df.w)).alias("s")).collect(),
+ )
+
+ def test_complex_agg_collect_set(self):
+ import pyarrow as pa
+
+ df = self.spark.createDataFrame([(1, 1), (1, 2), (2, 3), (2, 5), (2, 1)], ("id", "v"))
+
+ @arrow_udf("array")
+ def arrow_collect_set(v: pa.Array) -> pa.Scalar:
+ assert isinstance(v, pa.Array), str(type(v))
+ s = sorted([x.as_py() for x in pa.compute.unique(v)])
+ t = pa.list_(pa.int32())
+ return pa.scalar(value=s, type=t)
+
+ result1 = df.select(
+ arrow_collect_set(df["id"]).alias("ids"),
+ arrow_collect_set(df["v"]).alias("vs"),
+ )
+
+ expected1 = df.select(
+ sf.sort_array(sf.collect_set(df["id"])).alias("ids"),
+ sf.sort_array(sf.collect_set(df["v"])).alias("vs"),
+ )
+
+ self.assertEqual(expected1.collect(), result1.collect())
+
+ def test_complex_agg_collect_list(self):
+ import pyarrow as pa
+
+ df = self.spark.createDataFrame([(1, 1), (1, 2), (2, 3), (2, 5), (2, 1)], ("id", "v"))
+
+ @arrow_udf("array")
+ def arrow_collect_list(v: pa.Array) -> pa.Scalar:
+ assert isinstance(v, pa.Array), str(type(v))
+ s = sorted([x.as_py() for x in v])
+ t = pa.list_(pa.int32())
+ return pa.scalar(value=s, type=t)
+
+ result1 = df.select(
+ arrow_collect_list(df["id"]).alias("ids"),
+ arrow_collect_list(df["v"]).alias("vs"),
+ )
+
+ expected1 = df.select(
+ sf.sort_array(sf.collect_list(df["id"])).alias("ids"),
+ sf.sort_array(sf.collect_list(df["v"])).alias("vs"),
+ )
+
+ self.assertEqual(expected1.collect(), result1.collect())
+
+ def test_complex_agg_collect_as_map(self):
+ import pyarrow as pa
+
+ df = self.spark.createDataFrame([(1, 1), (2, 2), (3, 5)], ("id", "v"))
+
+ @arrow_udf("map")
+ def arrow_collect_as_map(id: pa.Array, v: pa.Array) -> pa.Scalar:
+ assert isinstance(id, pa.Array), str(type(id))
+ assert isinstance(v, pa.Array), str(type(v))
+ d = {i: j for i, j in zip(id.to_pylist(), v.to_pylist())}
+ t = pa.map_(pa.int32(), pa.int32())
+ return pa.scalar(value=d, type=t)
+
+ result1 = df.select(
+ arrow_collect_as_map("id", "v").alias("map"),
+ )
+
+ expected1 = df.select(
+ sf.map_from_arrays(sf.collect_list("id"), sf.collect_list("v")).alias("map"),
+ )
+
+ self.assertEqual(expected1.collect(), result1.collect())
+
+ def test_complex_agg_min_max_struct(self):
+ import pyarrow as pa
+
+ df = self.spark.createDataFrame([(1, 1), (2, 2), (3, 5)], ("id", "v"))
+
+ @arrow_udf("struct")
+ def arrow_collect_min_max(id: pa.Array, v: pa.Array) -> pa.Scalar:
+ assert isinstance(id, pa.Array), str(type(id))
+ assert isinstance(v, pa.Array), str(type(v))
+ m1 = pa.compute.min(id)
+ m2 = pa.compute.max(v)
+ t = pa.struct([pa.field("m1", pa.int32()), pa.field("m2", pa.int32())])
+ return pa.scalar(value={"m1": m1.as_py(), "m2": m2.as_py()}, type=t)
+
+ result1 = df.select(
+ arrow_collect_min_max("id", "v").alias("struct"),
+ )
+
+ expected1 = df.select(
+ sf.struct(sf.min("id").alias("m1"), sf.max("v").alias("m2")).alias("struct"),
+ )
+
+ self.assertEqual(expected1.collect(), result1.collect())
+
+ def test_time_min(self):
+ import pyarrow as pa
+
+ df = self.spark.sql(
+ """
+ SELECT * FROM VALUES
+ (1, TIME '12:34:56'),
+ (1, TIME '1:2:3'),
+ (2, TIME '0:58:59'),
+ (2, TIME '10:58:59'),
+ (2, TIME '10:00:03')
+ AS tab(i, t)
+ """
+ )
+
+ @arrow_udf("time", ArrowUDFType.GROUPED_AGG)
+ def agg_min_time(v):
+ assert isinstance(v, pa.Array)
+ assert isinstance(v, pa.Time64Array)
+ return pa.compute.min(v)
+
+ expected1 = df.select(sf.min("t").alias("res"))
+ result1 = df.select(agg_min_time("t").alias("res"))
+ self.assertEqual(expected1.collect(), result1.collect())
+
+ expected2 = df.groupby("i").agg(sf.min("t").alias("res")).sort("i")
+ result2 = df.groupby("i").agg(agg_min_time("t").alias("res")).sort("i")
+ self.assertEqual(expected2.collect(), result2.collect())
+
+ def test_return_type_coercion(self):
+ import pyarrow as pa
+
+ df = self.spark.range(10)
+
+ @arrow_udf("long", ArrowUDFType.GROUPED_AGG)
+ def agg_long(id: pa.Array) -> int:
+ assert isinstance(id, pa.Array), str(type(id))
+ return pa.scalar(value=len(id), type=pa.int64())
+
+ result1 = df.select(agg_long("id").alias("res"))
+ self.assertEqual(1, len(result1.collect()))
+
+ # long -> int coercion
+ @arrow_udf("int", ArrowUDFType.GROUPED_AGG)
+ def agg_int1(id: pa.Array) -> int:
+ assert isinstance(id, pa.Array), str(type(id))
+ return pa.scalar(value=len(id), type=pa.int64())
+
+ result2 = df.select(agg_int1("id").alias("res"))
+ self.assertEqual(1, len(result2.collect()))
+
+ # long -> int coercion, overflow
+ @arrow_udf("int", ArrowUDFType.GROUPED_AGG)
+ def agg_int2(id: pa.Array) -> int:
+ assert isinstance(id, pa.Array), str(type(id))
+ return pa.scalar(value=len(id) + 2147483647, type=pa.int64())
+
+ result3 = df.select(agg_int2("id").alias("res"))
+ with self.assertRaises(Exception):
+ # pyarrow.lib.ArrowInvalid:
+ # Integer value 2147483657 not in range: -2147483648 to 2147483647
+ result3.collect()
+
+ def test_unsupported_return_types(self):
+ import pyarrow as pa
+
+ with self.quiet():
+ with self.assertRaisesRegex(
+ NotImplementedError,
+ "Invalid return type with grouped aggregate "
+ "Arrow UDFs.*ArrayType.*YearMonthIntervalType",
+ ):
+ arrow_udf(
+ lambda x: x,
+ ArrayType(ArrayType(YearMonthIntervalType())),
+ ArrowUDFType.GROUPED_AGG,
+ )
+
+ with self.assertRaisesRegex(
+ NotImplementedError,
+ "Invalid return type with grouped aggregate "
+ "Arrow UDFs.*ArrayType.*YearMonthIntervalType",
+ ):
+
+ @arrow_udf(ArrayType(ArrayType(YearMonthIntervalType())), ArrowUDFType.GROUPED_AGG)
+ def func_a(a: pa.Array) -> pa.Scalar:
+ return pa.compute.max(a)
+
+
+class GroupedAggArrowUDFTests(GroupedAggArrowUDFTestsMixin, ReusedSQLTestCase):
+ pass
+
+
+if __name__ == "__main__":
+ from pyspark.sql.tests.arrow.test_arrow_udf_grouped_agg import * # noqa: F401
+
+ try:
+ import xmlrunner
+
+ testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2)
+ except ImportError:
+ testRunner = None
+ unittest.main(testRunner=testRunner, verbosity=2)
diff --git a/python/pyspark/sql/tests/arrow/test_arrow_udf_scalar.py b/python/pyspark/sql/tests/arrow/test_arrow_udf_scalar.py
index 052956db6e6e2..d6e010d8d2a98 100644
--- a/python/pyspark/sql/tests/arrow/test_arrow_udf_scalar.py
+++ b/python/pyspark/sql/tests/arrow/test_arrow_udf_scalar.py
@@ -19,14 +19,13 @@
import random
import time
import unittest
-from datetime import date, datetime, timezone
+import datetime
from decimal import Decimal
from typing import Iterator, Tuple
from pyspark.util import PythonEvalType
-# TODO: import arrow_udf from public API
-from pyspark.sql.pandas.functions import arrow_udf, ArrowUDFType
+from pyspark.sql.functions import arrow_udf, ArrowUDFType
from pyspark.sql import functions as F
from pyspark.sql.types import (
IntegerType,
@@ -44,6 +43,7 @@
Row,
MapType,
BinaryType,
+ YearMonthIntervalType,
)
from pyspark.errors import AnalysisException, PythonException
from pyspark.testing.sqlutils import (
@@ -98,7 +98,7 @@ def test_arrow_udf_output_nested_arrays(self):
df = self.spark.createDataFrame([("hi boo",), ("bye boo",)], ["vals"])
tokenize = arrow_udf(
- lambda s: pa.array([pa.compute.ascii_split_whitespace(s).to_pylist()]),
+ lambda s: pa.array([[v] for v in pa.compute.ascii_split_whitespace(s).to_pylist()]),
ArrayType(ArrayType(StringType())),
)
@@ -305,16 +305,17 @@ def test_arrow_udf_output_dates(self):
def build_date(y, m, d):
assert all(isinstance(x, pa.Array) for x in [y, m, d])
dates = [
- date(int(y[i].as_py()), int(m[i].as_py()), int(d[i].as_py())) for i in range(len(y))
+ datetime.date(int(y[i].as_py()), int(m[i].as_py()), int(d[i].as_py()))
+ for i in range(len(y))
]
return pa.array(dates, pa.date32())
result = df.select(build_date("y", "m", "d").alias("date"))
self.assertEqual(
[
- Row(date=date(2022, 1, 5)),
- Row(date=date(2023, 2, 6)),
- Row(date=date(2024, 3, 7)),
+ Row(date=datetime.date(2022, 1, 5)),
+ Row(date=datetime.date(2023, 2, 6)),
+ Row(date=datetime.date(2024, 3, 7)),
],
result.collect(),
)
@@ -361,14 +362,14 @@ def test_arrow_udf_output_timestamps_ltz(self):
def build_ts(y, m, d, h, mi, s):
assert all(isinstance(x, pa.Array) for x in [y, m, d, h, mi, s])
dates = [
- datetime(
+ datetime.datetime(
int(y[i].as_py()),
int(m[i].as_py()),
int(d[i].as_py()),
int(h[i].as_py()),
int(mi[i].as_py()),
int(s[i].as_py()),
- tzinfo=timezone.utc,
+ tzinfo=datetime.timezone.utc,
)
for i in range(len(y))
]
@@ -377,9 +378,9 @@ def build_ts(y, m, d, h, mi, s):
result = df.select(build_ts("y", "m", "d", "h", "mi", "s").alias("ts"))
self.assertEqual(
[
- Row(ts=datetime(2022, 1, 5, 7, 0, 1)),
- Row(ts=datetime(2023, 2, 6, 8, 1, 2)),
- Row(ts=datetime(2024, 3, 7, 9, 2, 3)),
+ Row(ts=datetime.datetime(2022, 1, 5, 7, 0, 1)),
+ Row(ts=datetime.datetime(2023, 2, 6, 8, 1, 2)),
+ Row(ts=datetime.datetime(2024, 3, 7, 9, 2, 3)),
],
result.collect(),
)
@@ -401,7 +402,7 @@ def test_arrow_udf_output_timestamps_ntz(self):
def build_ts(y, m, d, h, mi, s):
assert all(isinstance(x, pa.Array) for x in [y, m, d, h, mi, s])
dates = [
- datetime(
+ datetime.datetime(
int(y[i].as_py()),
int(m[i].as_py()),
int(d[i].as_py()),
@@ -416,13 +417,137 @@ def build_ts(y, m, d, h, mi, s):
result = df.select(build_ts("y", "m", "d", "h", "mi", "s").alias("ts"))
self.assertEqual(
[
- Row(ts=datetime(2022, 1, 5, 15, 0, 1)),
- Row(ts=datetime(2023, 2, 6, 16, 1, 2)),
- Row(ts=datetime(2024, 3, 7, 17, 2, 3)),
+ Row(ts=datetime.datetime(2022, 1, 5, 15, 0, 1)),
+ Row(ts=datetime.datetime(2023, 2, 6, 16, 1, 2)),
+ Row(ts=datetime.datetime(2024, 3, 7, 17, 2, 3)),
+ ],
+ result.collect(),
+ )
+
+ def test_arrow_udf_input_times(self):
+ import pyarrow as pa
+
+ df = self.spark.sql(
+ """
+ SELECT * FROM VALUES
+ (1, TIME '12:34:56'),
+ (2, TIME '1:2:3'),
+ (3, TIME '0:58:59')
+ AS tab(i, ts)
+ """
+ )
+
+ @arrow_udf("int")
+ def extract_second(v):
+ assert isinstance(v, pa.Array)
+ assert isinstance(v, pa.Time64Array), type(v)
+ return pa.array([t.as_py().second for t in v], pa.int32())
+
+ result = df.select(extract_second("ts").alias("sec"))
+ self.assertEqual(
+ [
+ Row(sec=56),
+ Row(sec=3),
+ Row(sec=59),
+ ],
+ result.collect(),
+ )
+
+ def test_arrow_udf_output_times(self):
+ import pyarrow as pa
+
+ df = self.spark.sql(
+ """
+ SELECT * FROM VALUES
+ (12, 34, 56),
+ (1, 2, 3),
+ (0, 58, 59)
+ AS tab(h, mi, s)
+ """
+ )
+
+ @arrow_udf("time")
+ def build_time(h, mi, s):
+ assert all(isinstance(x, pa.Array) for x in [h, mi, s])
+ dates = [
+ datetime.time(
+ int(h[i].as_py()),
+ int(mi[i].as_py()),
+ int(s[i].as_py()),
+ )
+ for i in range(len(h))
+ ]
+ return pa.array(dates, pa.time64("ns"))
+
+ result = df.select(build_time("h", "mi", "s").alias("t"))
+ self.assertEqual(
+ [
+ Row(t=datetime.time(12, 34, 56)),
+ Row(t=datetime.time(1, 2, 3)),
+ Row(t=datetime.time(0, 58, 59)),
],
result.collect(),
)
+ def test_arrow_udf_input_variant(self):
+ import pyarrow as pa
+
+ @arrow_udf("int")
+ def scalar_f(v: pa.Array) -> pa.Array:
+ assert isinstance(v, pa.Array)
+ assert isinstance(v, pa.StructArray)
+ assert isinstance(v.field("metadata"), pa.BinaryArray)
+ assert isinstance(v.field("value"), pa.BinaryArray)
+ return pa.compute.binary_length(v.field("value"))
+
+ @arrow_udf("int")
+ def iter_f(it: Iterator[pa.Array]) -> Iterator[pa.Array]:
+ for v in it:
+ assert isinstance(v, pa.Array)
+ assert isinstance(v, pa.StructArray)
+ assert isinstance(v.field("metadata"), pa.BinaryArray)
+ assert isinstance(v.field("value"), pa.BinaryArray)
+ yield pa.compute.binary_length(v.field("value"))
+
+ df = self.spark.range(0, 10).selectExpr("parse_json(cast(id as string)) v")
+ expected = [Row(l=2) for i in range(10)]
+
+ for f in [scalar_f, iter_f]:
+ result = df.select(f("v").alias("l")).collect()
+ self.assertEqual(result, expected)
+
+ def test_arrow_udf_output_variant(self):
+ # referring to test_udf_with_variant_output in test_pandas_udf_scalar
+ import pyarrow as pa
+
+ # referring to_arrow_type in to pyspark.sql.pandas.types
+ fields = [
+ pa.field("value", pa.binary(), nullable=False),
+ pa.field("metadata", pa.binary(), nullable=False, metadata={b"variant": b"true"}),
+ ]
+
+ @arrow_udf("variant")
+ def scalar_f(v: pa.Array) -> pa.Array:
+ assert isinstance(v, pa.Array)
+ v = pa.array([bytes([12, i.as_py()]) for i in v], pa.binary())
+ m = pa.array([bytes([1, 0, 0]) for i in v], pa.binary())
+ return pa.StructArray.from_arrays([v, m], fields=fields)
+
+ @arrow_udf("variant")
+ def iter_f(it: Iterator[pa.Array]) -> Iterator[pa.Array]:
+ for v in it:
+ assert isinstance(v, pa.Array)
+ v = pa.array([bytes([12, i.as_py()]) for i in v])
+ m = pa.array([bytes([1, 0, 0]) for i in v])
+ yield pa.StructArray.from_arrays([v, m], fields=fields)
+
+ df = self.spark.range(0, 10)
+ expected = [Row(l=i) for i in range(10)]
+
+ for f in [scalar_f, iter_f]:
+ result = df.select(f("id").cast("int").alias("l")).collect()
+ self.assertEqual(result, expected)
+
def test_arrow_udf_null_boolean(self):
data = [(True,), (True,), (None,), (False,)]
schema = StructType().add("bool", BooleanType())
@@ -536,7 +661,9 @@ def test_arrow_udf_datatype_string(self):
F.col("id").alias("long"),
F.col("id").cast("float").alias("float"),
F.col("id").cast("double").alias("double"),
- # F.col("id").cast("decimal").alias("decimal"),
+ F.col("id").cast("decimal").alias("decimal1"),
+ F.col("id").cast("decimal(10, 0)").alias("decimal2"),
+ F.col("id").cast("decimal(38, 18)").alias("decimal3"),
F.col("id").cast("boolean").alias("bool"),
)
@@ -549,7 +676,9 @@ def f(x):
long_f = arrow_udf(f, "long", udf_type)
float_f = arrow_udf(f, "float", udf_type)
double_f = arrow_udf(f, "double", udf_type)
- # decimal_f = arrow_udf(f, "decimal(38, 18)", udf_type)
+ decimal1_f = arrow_udf(f, "decimal", udf_type)
+ decimal2_f = arrow_udf(f, "decimal(10, 0)", udf_type)
+ decimal3_f = arrow_udf(f, "decimal(38, 18)", udf_type)
bool_f = arrow_udf(f, "boolean", udf_type)
res = df.select(
str_f(F.col("str")),
@@ -557,7 +686,9 @@ def f(x):
long_f(F.col("long")),
float_f(F.col("float")),
double_f(F.col("double")),
- # decimal_f("decimal"),
+ decimal1_f("decimal1"),
+ decimal2_f("decimal2"),
+ decimal3_f("decimal3"),
bool_f(F.col("bool")),
)
self.assertEqual(df.collect(), res.collect())
@@ -588,6 +719,22 @@ def test_udf_register_arrow_udf_basic(self):
self.assertEqual(expected.collect(), res1.collect())
self.assertEqual(expected.collect(), res2.collect())
+ @arrow_udf(LongType())
+ def scalar_iter_add(it: Iterator[Tuple[pa.Array, pa.Array]]) -> Iterator[pa.Array]:
+ for a, b in it:
+ yield pa.compute.add(a, b)
+
+ self.spark.sql("DROP TEMPORARY FUNCTION IF EXISTS add1")
+ new_add = self.spark.udf.register("add1", scalar_iter_add)
+
+ res3 = df.select(new_add(F.col("a"), F.col("b")))
+ res4 = self.spark.sql(
+ "SELECT add1(t.a, t.b) FROM (SELECT id as a, id as b FROM range(10)) t"
+ )
+ expected = df.select(F.expr("a + b"))
+ self.assertEqual(expected.collect(), res3.collect())
+ self.assertEqual(expected.collect(), res4.collect())
+
def test_catalog_register_arrow_udf_basic(self):
import pyarrow as pa
@@ -614,6 +761,22 @@ def test_catalog_register_arrow_udf_basic(self):
self.assertEqual(expected.collect(), res1.collect())
self.assertEqual(expected.collect(), res2.collect())
+ @arrow_udf(LongType())
+ def scalar_iter_add(it: Iterator[Tuple[pa.Array, pa.Array]]) -> Iterator[pa.Array]:
+ for a, b in it:
+ yield pa.compute.add(a, b)
+
+ self.spark.sql("DROP TEMPORARY FUNCTION IF EXISTS add1")
+ new_add = self.spark.catalog.registerFunction("add1", scalar_iter_add)
+
+ res3 = df.select(new_add(F.col("a"), F.col("b")))
+ res4 = self.spark.sql(
+ "SELECT add1(t.a, t.b) FROM (SELECT id as a, id as b FROM range(10)) t"
+ )
+ expected = df.select(F.expr("a + b"))
+ self.assertEqual(expected.collect(), res3.collect())
+ self.assertEqual(expected.collect(), res4.collect())
+
def test_udf_register_nondeterministic_arrow_udf(self):
import pyarrow as pa
@@ -963,6 +1126,48 @@ def multiple(it: Iterator[Tuple[pa.Array, pa.Array, pa.Array]]) -> Iterator[pa.A
result = df.select(multiple("a", "b", "c").alias("res"))
self.assertEqual(expected, result.collect())
+ def test_return_type_coercion(self):
+ import pyarrow as pa
+
+ df = self.spark.range(10)
+
+ scalar_long = arrow_udf(lambda x: pa.compute.add(x, 1), LongType())
+ result1 = df.select(scalar_long("id").alias("res"))
+ self.assertEqual(10, len(result1.collect()))
+
+ # long -> int coercion
+ scalar_int1 = arrow_udf(lambda x: pa.compute.add(x, 1), IntegerType())
+ result2 = df.select(scalar_int1("id").alias("res"))
+ self.assertEqual(10, len(result2.collect()))
+
+ # long -> int coercion, overflow
+ scalar_int2 = arrow_udf(lambda x: pa.compute.add(x, 2147483647), IntegerType())
+ result3 = df.select(scalar_int2("id").alias("res"))
+ with self.assertRaises(Exception):
+ # pyarrow.lib.ArrowInvalid:
+ # Integer value 2147483652 not in range: -2147483648 to 2147483647
+ result3.collect()
+
+ def test_unsupported_return_types(self):
+ import pyarrow as pa
+
+ with self.quiet():
+ for udf_type in [ArrowUDFType.SCALAR, ArrowUDFType.SCALAR_ITER]:
+ with self.assertRaisesRegex(
+ NotImplementedError,
+ "Invalid return type.*scalar Arrow UDF.*ArrayType.*YearMonthIntervalType",
+ ):
+ arrow_udf(lambda x: x, ArrayType(YearMonthIntervalType()), udf_type)
+
+ with self.assertRaisesRegex(
+ NotImplementedError,
+ "Invalid return type.*scalar Arrow UDF.*ArrayType.*YearMonthIntervalType",
+ ):
+
+ @arrow_udf(ArrayType(YearMonthIntervalType()))
+ def func_a(a: pa.Array) -> pa.Array:
+ return a
+
class ScalarArrowUDFTests(ScalarArrowUDFTestsMixin, ReusedSQLTestCase):
@classmethod
diff --git a/python/pyspark/sql/tests/arrow/test_arrow_udf_typehints.py b/python/pyspark/sql/tests/arrow/test_arrow_udf_typehints.py
new file mode 100644
index 0000000000000..81d980cc37212
--- /dev/null
+++ b/python/pyspark/sql/tests/arrow/test_arrow_udf_typehints.py
@@ -0,0 +1,351 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+import unittest
+from inspect import signature
+from typing import Union, Iterator, Tuple, get_type_hints
+
+from pyspark.sql import functions as sf
+from pyspark.testing.utils import (
+ have_pandas,
+ pandas_requirement_message,
+ have_pyarrow,
+ pyarrow_requirement_message,
+ have_numpy,
+ numpy_requirement_message,
+)
+from pyspark.testing.sqlutils import ReusedSQLTestCase
+from pyspark.sql.pandas.typehints import infer_eval_type
+from pyspark.sql.pandas.functions import arrow_udf, ArrowUDFType
+from pyspark.sql import Row
+
+if have_pyarrow:
+ import pyarrow as pa
+
+
+@unittest.skipIf(not have_pyarrow, pyarrow_requirement_message)
+class ArrowUDFTypeHintsTests(ReusedSQLTestCase):
+ def test_type_annotation_scalar(self):
+ def func(col: pa.Array) -> pa.Array:
+ pass
+
+ self.assertEqual(
+ infer_eval_type(signature(func), get_type_hints(func)), ArrowUDFType.SCALAR
+ )
+
+ def func(col: pa.Array, col1: pa.Array) -> pa.Array:
+ pass
+
+ self.assertEqual(
+ infer_eval_type(signature(func), get_type_hints(func)), ArrowUDFType.SCALAR
+ )
+
+ def func(col: pa.Array, *args: pa.Array) -> pa.Array:
+ pass
+
+ self.assertEqual(
+ infer_eval_type(signature(func), get_type_hints(func)), ArrowUDFType.SCALAR
+ )
+
+ def func(col: pa.Array, *args: pa.Array, **kwargs: pa.Array) -> pa.Array:
+ pass
+
+ self.assertEqual(
+ infer_eval_type(signature(func), get_type_hints(func)), ArrowUDFType.SCALAR
+ )
+
+ def func(col: pa.Array, *, col2: pa.Array) -> pa.Array:
+ pass
+
+ self.assertEqual(
+ infer_eval_type(signature(func), get_type_hints(func)), ArrowUDFType.SCALAR
+ )
+
+ # Union[pa.Array, pa.Array] equals to pa.Array
+ def func(col: Union[pa.Array, pa.Array], *, col2: pa.Array) -> pa.Array:
+ pass
+
+ self.assertEqual(
+ infer_eval_type(signature(func), get_type_hints(func)), ArrowUDFType.SCALAR
+ )
+
+ def test_type_annotation_scalar_iter(self):
+ def func(iter: Iterator[pa.Array]) -> Iterator[pa.Array]:
+ pass
+
+ self.assertEqual(
+ infer_eval_type(signature(func), get_type_hints(func)), ArrowUDFType.SCALAR_ITER
+ )
+
+ def func(iter: Iterator[Tuple[pa.Array, ...]]) -> Iterator[pa.Array]:
+ pass
+
+ self.assertEqual(
+ infer_eval_type(signature(func), get_type_hints(func)), ArrowUDFType.SCALAR_ITER
+ )
+
+ def test_type_annotation_tuple_generics(self):
+ def func(iter: Iterator[tuple[pa.Array, pa.Array]]) -> Iterator[pa.Array]:
+ pass
+
+ self.assertEqual(
+ infer_eval_type(signature(func), get_type_hints(func)), ArrowUDFType.SCALAR_ITER
+ )
+
+ def func(iter: Iterator[tuple[pa.Array, ...]]) -> Iterator[pa.Array]:
+ pass
+
+ self.assertEqual(
+ infer_eval_type(signature(func), get_type_hints(func)), ArrowUDFType.SCALAR_ITER
+ )
+
+ # Union[pa.Array, pa.Array] equals to pa.Array
+ def func(iter: Iterator[tuple[Union[pa.Array, pa.Array], ...]]) -> Iterator[pa.Array]:
+ pass
+
+ self.assertEqual(
+ infer_eval_type(signature(func), get_type_hints(func)), ArrowUDFType.SCALAR_ITER
+ )
+
+ def test_type_annotation_group_agg(self):
+ def func(col: pa.Array) -> str:
+ pass
+
+ self.assertEqual(
+ infer_eval_type(signature(func), get_type_hints(func)), ArrowUDFType.GROUPED_AGG
+ )
+
+ def func(col: pa.Array, col1: pa.Array) -> int:
+ pass
+
+ self.assertEqual(
+ infer_eval_type(signature(func), get_type_hints(func)), ArrowUDFType.GROUPED_AGG
+ )
+
+ def func(col: pa.Array, *args: pa.Array) -> Row:
+ pass
+
+ self.assertEqual(
+ infer_eval_type(signature(func), get_type_hints(func)), ArrowUDFType.GROUPED_AGG
+ )
+
+ def func(col: pa.Array, *args: pa.Array, **kwargs: pa.Array) -> str:
+ pass
+
+ self.assertEqual(
+ infer_eval_type(signature(func), get_type_hints(func)), ArrowUDFType.GROUPED_AGG
+ )
+
+ def func(col: pa.Array, *, col2: pa.Array) -> float:
+ pass
+
+ self.assertEqual(
+ infer_eval_type(signature(func), get_type_hints(func)), ArrowUDFType.GROUPED_AGG
+ )
+
+ # Union[pa.Array, pa.Array] equals to pa.Array
+ def func(col: Union[pa.Array, pa.Array], *, col2: pa.Array) -> float:
+ pass
+
+ self.assertEqual(
+ infer_eval_type(signature(func), get_type_hints(func)), ArrowUDFType.GROUPED_AGG
+ )
+
+ def test_type_annotation_negative(self):
+ def func(col: str) -> pa.Array:
+ pass
+
+ self.assertRaisesRegex(
+ NotImplementedError,
+ "Unsupported signature.*str",
+ infer_eval_type,
+ signature(func),
+ get_type_hints(func),
+ )
+
+ def func(col: pa.Array, col1: int) -> pa.Array:
+ pass
+
+ self.assertRaisesRegex(
+ NotImplementedError,
+ "Unsupported signature.*int",
+ infer_eval_type,
+ signature(func),
+ get_type_hints(func),
+ )
+
+ def func(col: Union[pa.Array, str], col1: int) -> pa.Array:
+ pass
+
+ self.assertRaisesRegex(
+ NotImplementedError,
+ "Unsupported signature.*str",
+ infer_eval_type,
+ signature(func),
+ get_type_hints(func),
+ )
+
+ def func(col: pa.Array) -> Tuple[pa.Array]:
+ pass
+
+ self.assertRaisesRegex(
+ NotImplementedError,
+ "Unsupported signature.*Tuple",
+ infer_eval_type,
+ signature(func),
+ get_type_hints(func),
+ )
+
+ def func(col, *args: pa.Array) -> pa.Array:
+ pass
+
+ self.assertRaisesRegex(
+ ValueError,
+ "should be specified.*Array",
+ infer_eval_type,
+ signature(func),
+ get_type_hints(func),
+ )
+
+ def func(col: pa.Array, *args: pa.Array, **kwargs: pa.Array):
+ pass
+
+ self.assertRaisesRegex(
+ ValueError,
+ "should be specified.*Array",
+ infer_eval_type,
+ signature(func),
+ get_type_hints(func),
+ )
+
+ def func(col: pa.Array, *, col2) -> pa.Array:
+ pass
+
+ self.assertRaisesRegex(
+ ValueError,
+ "should be specified.*Array",
+ infer_eval_type,
+ signature(func),
+ get_type_hints(func),
+ )
+
+ def test_scalar_udf_type_hint(self):
+ df = self.spark.range(10).selectExpr("id", "id as v")
+
+ def plus_one(v: pa.Array) -> pa.Array:
+ return pa.compute.add(v, 1)
+
+ plus_one = arrow_udf("long")(plus_one)
+ actual = df.select(plus_one(df.v).alias("plus_one"))
+ expected = df.selectExpr("(v + 1) as plus_one")
+ self.assertEqual(expected.collect(), actual.collect())
+
+ def test_scalar_iter_udf_type_hint(self):
+ df = self.spark.range(10).selectExpr("id", "id as v")
+
+ def plus_one(itr: Iterator[pa.Array]) -> Iterator[pa.Array]:
+ for s in itr:
+ yield pa.compute.add(s, 1)
+
+ plus_one = arrow_udf("long")(plus_one)
+
+ actual = df.select(plus_one(df.v).alias("plus_one"))
+ expected = df.selectExpr("(v + 1) as plus_one")
+ self.assertEqual(expected.collect(), actual.collect())
+
+ @unittest.skipIf(not have_numpy, numpy_requirement_message)
+ def test_group_agg_udf_type_hint(self):
+ import numpy as np
+
+ df = self.spark.range(10).selectExpr("id", "id as v")
+
+ def weighted_mean(v: pa.Array, w: pa.Array) -> np.float64:
+ return np.average(v, weights=w)
+
+ weighted_mean = arrow_udf("double")(weighted_mean)
+
+ actual = df.groupby("id").agg(weighted_mean(df.v, sf.lit(1.0))).sort("id")
+ expected = df.groupby("id").agg(sf.mean(df.v).alias("weighted_mean(v, 1.0)")).sort("id")
+ self.assertEqual(expected.collect(), actual.collect())
+
+ def test_string_type_annotation(self):
+ def func(col: "pa.Array") -> "pa.Array":
+ pass
+
+ self.assertEqual(
+ infer_eval_type(signature(func), get_type_hints(func)), ArrowUDFType.SCALAR
+ )
+
+ def func(col: "pa.Array", col1: "pa.Array") -> "pa.Array":
+ pass
+
+ self.assertEqual(
+ infer_eval_type(signature(func), get_type_hints(func)), ArrowUDFType.SCALAR
+ )
+
+ def func(col: "pa.Array", *args: "pa.Array") -> "pa.Array":
+ pass
+
+ self.assertEqual(
+ infer_eval_type(signature(func), get_type_hints(func)), ArrowUDFType.SCALAR
+ )
+
+ def func(col: "pa.Array", *args: "pa.Array", **kwargs: "pa.Array") -> "pa.Array":
+ pass
+
+ self.assertEqual(
+ infer_eval_type(signature(func), get_type_hints(func)), ArrowUDFType.SCALAR
+ )
+
+ def func(col: "pa.Array", *, col2: "pa.Array") -> "pa.Array":
+ pass
+
+ self.assertEqual(
+ infer_eval_type(signature(func), get_type_hints(func)), ArrowUDFType.SCALAR
+ )
+
+ # Union[pa.Array, pa.Array] equals to pa.Array
+ def func(col: Union["pa.Array", "pa.Array"], *, col2: "pa.Array") -> "pa.Array":
+ pass
+
+ self.assertEqual(
+ infer_eval_type(signature(func), get_type_hints(func)), ArrowUDFType.SCALAR
+ )
+
+ @unittest.skipIf(not have_pandas, pandas_requirement_message)
+ def test_negative_with_pandas_udf(self):
+ import pandas as pd
+
+ with self.assertRaisesRegex(
+ Exception,
+ "Unsupported signature:.*pandas.core.series.Series.",
+ ):
+
+ @arrow_udf("long")
+ def multiply_pandas(a: pd.Series, b: pd.Series) -> pd.Series:
+ return a * b
+
+
+if __name__ == "__main__":
+ from pyspark.sql.tests.arrow.test_arrow_udf_typehints import * # noqa: #401
+
+ try:
+ import xmlrunner
+
+ testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2)
+ except ImportError:
+ testRunner = None
+ unittest.main(testRunner=testRunner, verbosity=2)
diff --git a/python/pyspark/sql/tests/arrow/test_arrow_udf_window.py b/python/pyspark/sql/tests/arrow/test_arrow_udf_window.py
new file mode 100644
index 0000000000000..fde9d72433757
--- /dev/null
+++ b/python/pyspark/sql/tests/arrow/test_arrow_udf_window.py
@@ -0,0 +1,735 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+import unittest
+
+from pyspark.sql.functions import arrow_udf, ArrowUDFType
+from pyspark.util import PythonEvalType
+from pyspark.sql import functions as sf
+from pyspark.sql.window import Window
+from pyspark.errors import AnalysisException, PythonException, PySparkTypeError
+from pyspark.testing.sqlutils import (
+ ReusedSQLTestCase,
+ have_pyarrow,
+ pyarrow_requirement_message,
+)
+
+
+@unittest.skipIf(not have_pyarrow, pyarrow_requirement_message)
+class WindowArrowUDFTestsMixin:
+ @property
+ def data(self):
+ return (
+ self.spark.range(10)
+ .toDF("id")
+ .withColumn("vs", sf.array([sf.lit(i * 1.0) + sf.col("id") for i in range(20, 30)]))
+ .withColumn("v", sf.explode(sf.col("vs")))
+ .drop("vs")
+ .withColumn("w", sf.lit(1.0))
+ )
+
+ @property
+ def python_plus_one(self):
+ @sf.udf("double")
+ def plus_one(v):
+ assert isinstance(v, float)
+ return v + 1
+
+ return plus_one
+
+ @property
+ def arrow_scalar_time_two(self):
+ import pyarrow as pa
+
+ return arrow_udf(lambda v: pa.compute.multiply(v, 2), "double")
+
+ @property
+ def arrow_agg_count_udf(self):
+ @arrow_udf("long", ArrowUDFType.GROUPED_AGG)
+ def count(v):
+ return len(v)
+
+ return count
+
+ @property
+ def arrow_agg_mean_udf(self):
+ import pyarrow as pa
+
+ @arrow_udf("double", ArrowUDFType.GROUPED_AGG)
+ def avg(v):
+ return pa.compute.mean(v)
+
+ return avg
+
+ @property
+ def arrow_agg_max_udf(self):
+ import pyarrow as pa
+
+ @arrow_udf("double", ArrowUDFType.GROUPED_AGG)
+ def max(v):
+ return pa.compute.max(v)
+
+ return max
+
+ @property
+ def arrow_agg_min_udf(self):
+ import pyarrow as pa
+
+ @arrow_udf("double", ArrowUDFType.GROUPED_AGG)
+ def min(v):
+ return pa.compute.min(v)
+
+ return min
+
+ @property
+ def arrow_agg_weighted_mean_udf(self):
+ import numpy as np
+
+ @arrow_udf("double", ArrowUDFType.GROUPED_AGG)
+ def weighted_mean(v, w):
+ return np.average(v, weights=w)
+
+ return weighted_mean
+
+ @property
+ def unbounded_window(self):
+ return (
+ Window.partitionBy("id")
+ .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)
+ .orderBy("v")
+ )
+
+ @property
+ def ordered_window(self):
+ return Window.partitionBy("id").orderBy("v")
+
+ @property
+ def unpartitioned_window(self):
+ return Window.partitionBy()
+
+ @property
+ def sliding_row_window(self):
+ return Window.partitionBy("id").orderBy("v").rowsBetween(-2, 1)
+
+ @property
+ def sliding_range_window(self):
+ return Window.partitionBy("id").orderBy("v").rangeBetween(-2, 4)
+
+ @property
+ def growing_row_window(self):
+ return Window.partitionBy("id").orderBy("v").rowsBetween(Window.unboundedPreceding, 3)
+
+ @property
+ def growing_range_window(self):
+ return Window.partitionBy("id").orderBy("v").rangeBetween(Window.unboundedPreceding, 4)
+
+ @property
+ def shrinking_row_window(self):
+ return Window.partitionBy("id").orderBy("v").rowsBetween(-2, Window.unboundedFollowing)
+
+ @property
+ def shrinking_range_window(self):
+ return Window.partitionBy("id").orderBy("v").rangeBetween(-3, Window.unboundedFollowing)
+
+ def test_simple(self):
+ df = self.data
+ w = self.unbounded_window
+
+ mean_udf = self.arrow_agg_mean_udf
+
+ result1 = df.withColumn("mean_v", mean_udf(df["v"]).over(w))
+ expected1 = df.withColumn("mean_v", sf.mean(df["v"]).over(w))
+
+ result2 = df.select(mean_udf(df["v"]).over(w))
+ expected2 = df.select(sf.mean(df["v"]).over(w))
+
+ self.assertEqual(expected1.collect(), result1.collect())
+ self.assertEqual(expected2.collect(), result2.collect())
+
+ def test_multiple_udfs(self):
+ df = self.data
+ w = self.unbounded_window
+
+ result1 = (
+ df.withColumn("mean_v", self.arrow_agg_mean_udf(df["v"]).over(w))
+ .withColumn("max_v", self.arrow_agg_max_udf(df["v"]).over(w))
+ .withColumn("min_w", self.arrow_agg_min_udf(df["w"]).over(w))
+ )
+
+ expected1 = (
+ df.withColumn("mean_v", sf.mean(df["v"]).over(w))
+ .withColumn("max_v", sf.max(df["v"]).over(w))
+ .withColumn("min_w", sf.min(df["w"]).over(w))
+ )
+
+ self.assertEqual(expected1.collect(), result1.collect())
+
+ def test_replace_existing(self):
+ df = self.data
+ w = self.unbounded_window
+
+ result1 = df.withColumn("v", self.arrow_agg_mean_udf(df["v"]).over(w))
+ expected1 = df.withColumn("v", sf.mean(df["v"]).over(w))
+
+ self.assertEqual(expected1.collect(), result1.collect())
+
+ def test_mixed_sql(self):
+ df = self.data
+ w = self.unbounded_window
+ mean_udf = self.arrow_agg_mean_udf
+
+ result1 = df.withColumn("v", mean_udf(df["v"] * 2).over(w) + 1)
+ expected1 = df.withColumn("v", sf.mean(df["v"] * 2).over(w) + 1)
+
+ self.assertEqual(expected1.collect(), result1.collect())
+
+ def test_mixed_udf(self):
+ df = self.data
+ w = self.unbounded_window
+
+ plus_one = self.python_plus_one
+ time_two = self.arrow_scalar_time_two
+ mean_udf = self.arrow_agg_mean_udf
+
+ result1 = df.withColumn("v2", plus_one(mean_udf(plus_one(df["v"])).over(w)))
+ expected1 = df.withColumn("v2", plus_one(sf.mean(plus_one(df["v"])).over(w)))
+
+ result2 = df.withColumn("v2", time_two(mean_udf(time_two(df["v"])).over(w)))
+ expected2 = df.withColumn("v2", time_two(sf.mean(time_two(df["v"])).over(w)))
+
+ self.assertEqual(expected1.collect(), result1.collect())
+ self.assertEqual(expected2.collect(), result2.collect())
+
+ def test_without_partitionBy(self):
+ df = self.data
+ w = self.unpartitioned_window
+ mean_udf = self.arrow_agg_mean_udf
+
+ result1 = df.withColumn("v2", mean_udf(df["v"]).over(w))
+ expected1 = df.withColumn("v2", sf.mean(df["v"]).over(w))
+
+ result2 = df.select(mean_udf(df["v"]).over(w))
+ expected2 = df.select(sf.mean(df["v"]).over(w))
+
+ self.assertEqual(expected1.collect(), result1.collect())
+ self.assertEqual(expected2.collect(), result2.collect())
+
+ def test_mixed_sql_and_udf(self):
+ df = self.data
+ w = self.unbounded_window
+ ow = self.ordered_window
+ max_udf = self.arrow_agg_max_udf
+ min_udf = self.arrow_agg_min_udf
+
+ result1 = df.withColumn("v_diff", max_udf(df["v"]).over(w) - min_udf(df["v"]).over(w))
+ expected1 = df.withColumn("v_diff", sf.max(df["v"]).over(w) - sf.min(df["v"]).over(w))
+
+ # Test mixing sql window function and window udf in the same expression
+ result2 = df.withColumn("v_diff", max_udf(df["v"]).over(w) - sf.min(df["v"]).over(w))
+ expected2 = expected1
+
+ # Test chaining sql aggregate function and udf
+ result3 = (
+ df.withColumn("max_v", max_udf(df["v"]).over(w))
+ .withColumn("min_v", sf.min(df["v"]).over(w))
+ .withColumn("v_diff", sf.col("max_v") - sf.col("min_v"))
+ .drop("max_v", "min_v")
+ )
+ expected3 = expected1
+
+ # Test mixing sql window function and udf
+ result4 = df.withColumn("max_v", max_udf(df["v"]).over(w)).withColumn(
+ "rank", sf.rank().over(ow)
+ )
+ expected4 = df.withColumn("max_v", sf.max(df["v"]).over(w)).withColumn(
+ "rank", sf.rank().over(ow)
+ )
+
+ self.assertEqual(expected1.collect(), result1.collect())
+ self.assertEqual(expected2.collect(), result2.collect())
+ self.assertEqual(expected3.collect(), result3.collect())
+ self.assertEqual(expected4.collect(), result4.collect())
+
+ def test_array_type(self):
+ df = self.data
+ w = self.unbounded_window
+
+ array_udf = arrow_udf(lambda x: [1.0, 2.0], "array", ArrowUDFType.GROUPED_AGG)
+ result1 = df.withColumn("v2", array_udf(df["v"]).over(w))
+ self.assertEqual(result1.first()["v2"], [1.0, 2.0])
+
+ def test_invalid_args(self):
+ with self.quiet():
+ self.check_invalid_args()
+
+ def check_invalid_args(self):
+ df = self.data
+ w = self.unbounded_window
+
+ with self.assertRaises(PySparkTypeError):
+ foo_udf = arrow_udf(lambda x: x, "v double", PythonEvalType.SQL_GROUPED_MAP_ARROW_UDF)
+ df.withColumn("v2", foo_udf(df["v"]).over(w)).schema
+
+ def test_bounded_simple(self):
+ df = self.data
+ w1 = self.sliding_row_window
+ w2 = self.shrinking_range_window
+
+ plus_one = self.python_plus_one
+ count_udf = self.arrow_agg_count_udf
+ mean_udf = self.arrow_agg_mean_udf
+ max_udf = self.arrow_agg_max_udf
+ min_udf = self.arrow_agg_min_udf
+
+ result1 = (
+ df.withColumn("mean_v", mean_udf(plus_one(df["v"])).over(w1))
+ .withColumn("count_v", count_udf(df["v"]).over(w2))
+ .withColumn("max_v", max_udf(df["v"]).over(w2))
+ .withColumn("min_v", min_udf(df["v"]).over(w1))
+ )
+
+ expected1 = (
+ df.withColumn("mean_v", sf.mean(plus_one(df["v"])).over(w1))
+ .withColumn("count_v", sf.count(df["v"]).over(w2))
+ .withColumn("max_v", sf.max(df["v"]).over(w2))
+ .withColumn("min_v", sf.min(df["v"]).over(w1))
+ )
+
+ self.assertEqual(expected1.collect(), result1.collect())
+
+ def test_growing_window(self):
+ df = self.data
+ w1 = self.growing_row_window
+ w2 = self.growing_range_window
+
+ mean_udf = self.arrow_agg_mean_udf
+
+ result1 = df.withColumn("m1", mean_udf(df["v"]).over(w1)).withColumn(
+ "m2", mean_udf(df["v"]).over(w2)
+ )
+
+ expected1 = df.withColumn("m1", sf.mean(df["v"]).over(w1)).withColumn(
+ "m2", sf.mean(df["v"]).over(w2)
+ )
+
+ self.assertEqual(expected1.collect(), result1.collect())
+
+ def test_sliding_window(self):
+ df = self.data
+ w1 = self.sliding_row_window
+ w2 = self.sliding_range_window
+
+ mean_udf = self.arrow_agg_mean_udf
+
+ result1 = df.withColumn("m1", mean_udf(df["v"]).over(w1)).withColumn(
+ "m2", mean_udf(df["v"]).over(w2)
+ )
+
+ expected1 = df.withColumn("m1", sf.mean(df["v"]).over(w1)).withColumn(
+ "m2", sf.mean(df["v"]).over(w2)
+ )
+
+ self.assertEqual(expected1.collect(), result1.collect())
+
+ def test_shrinking_window(self):
+ df = self.data
+ w1 = self.shrinking_row_window
+ w2 = self.shrinking_range_window
+
+ mean_udf = self.arrow_agg_mean_udf
+
+ result1 = df.withColumn("m1", mean_udf(df["v"]).over(w1)).withColumn(
+ "m2", mean_udf(df["v"]).over(w2)
+ )
+
+ expected1 = df.withColumn("m1", sf.mean(df["v"]).over(w1)).withColumn(
+ "m2", sf.mean(df["v"]).over(w2)
+ )
+
+ self.assertEqual(expected1.collect(), result1.collect())
+
+ def test_bounded_mixed(self):
+ df = self.data
+ w1 = self.sliding_row_window
+ w2 = self.unbounded_window
+
+ mean_udf = self.arrow_agg_mean_udf
+ max_udf = self.arrow_agg_max_udf
+
+ result1 = (
+ df.withColumn("mean_v", mean_udf(df["v"]).over(w1))
+ .withColumn("max_v", max_udf(df["v"]).over(w2))
+ .withColumn("mean_unbounded_v", mean_udf(df["v"]).over(w1))
+ )
+
+ expected1 = (
+ df.withColumn("mean_v", sf.mean(df["v"]).over(w1))
+ .withColumn("max_v", sf.max(df["v"]).over(w2))
+ .withColumn("mean_unbounded_v", sf.mean(df["v"]).over(w1))
+ )
+
+ self.assertEqual(expected1.collect(), result1.collect())
+
+ def test_named_arguments(self):
+ df = self.data
+ weighted_mean = self.arrow_agg_weighted_mean_udf
+
+ for w, bound in [(self.sliding_row_window, True), (self.unbounded_window, False)]:
+ for i, windowed in enumerate(
+ [
+ df.withColumn("wm", weighted_mean(df.v, w=df.w).over(w)),
+ df.withColumn("wm", weighted_mean(v=df.v, w=df.w).over(w)),
+ df.withColumn("wm", weighted_mean(w=df.w, v=df.v).over(w)),
+ ]
+ ):
+ with self.subTest(bound=bound, query_no=i):
+ self.assertEqual(
+ windowed.collect(), df.withColumn("wm", sf.mean(df.v).over(w)).collect()
+ )
+
+ with self.tempView("v"):
+ df.createOrReplaceTempView("v")
+ self.spark.udf.register("weighted_mean", weighted_mean)
+
+ for w in [
+ "ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING",
+ "ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING",
+ ]:
+ window_spec = f"PARTITION BY id ORDER BY v {w}"
+ for i, func_call in enumerate(
+ [
+ "weighted_mean(v, w => w)",
+ "weighted_mean(v => v, w => w)",
+ "weighted_mean(w => w, v => v)",
+ ]
+ ):
+ with self.subTest(window_spec=window_spec, query_no=i):
+ self.assertEqual(
+ self.spark.sql(
+ f"SELECT id, {func_call} OVER ({window_spec}) as wm FROM v"
+ ).collect(),
+ self.spark.sql(
+ f"SELECT id, mean(v) OVER ({window_spec}) as wm FROM v"
+ ).collect(),
+ )
+
+ def test_named_arguments_negative(self):
+ df = self.data
+ weighted_mean = self.arrow_agg_weighted_mean_udf
+
+ with self.tempView("v"):
+ df.createOrReplaceTempView("v")
+ self.spark.udf.register("weighted_mean", weighted_mean)
+
+ base_sql = "SELECT id, {func_call} OVER ({window_spec}) as wm FROM v"
+
+ for w in [
+ "ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING",
+ "ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING",
+ ]:
+ window_spec = f"PARTITION BY id ORDER BY v {w}"
+ with self.subTest(window_spec=window_spec):
+ with self.assertRaisesRegex(
+ AnalysisException,
+ "DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT.DOUBLE_NAMED_ARGUMENT_REFERENCE",
+ ):
+ self.spark.sql(
+ base_sql.format(
+ func_call="weighted_mean(v => v, v => w)", window_spec=window_spec
+ )
+ ).show()
+
+ with self.assertRaisesRegex(
+ AnalysisException, "UNEXPECTED_POSITIONAL_ARGUMENT"
+ ):
+ self.spark.sql(
+ base_sql.format(
+ func_call="weighted_mean(v => v, w)", window_spec=window_spec
+ )
+ ).show()
+
+ with self.assertRaisesRegex(
+ PythonException, r"weighted_mean\(\) got an unexpected keyword argument 'x'"
+ ):
+ self.spark.sql(
+ base_sql.format(
+ func_call="weighted_mean(v => v, x => w)", window_spec=window_spec
+ )
+ ).show()
+
+ with self.assertRaisesRegex(
+ PythonException, r"weighted_mean\(\) got multiple values for argument 'v'"
+ ):
+ self.spark.sql(
+ base_sql.format(
+ func_call="weighted_mean(v, v => w)", window_spec=window_spec
+ )
+ ).show()
+
+ def test_kwargs(self):
+ df = self.data
+
+ @arrow_udf("double", ArrowUDFType.GROUPED_AGG)
+ def weighted_mean(**kwargs):
+ import numpy as np
+
+ return np.average(kwargs["v"], weights=kwargs["w"])
+
+ for w, bound in [(self.sliding_row_window, True), (self.unbounded_window, False)]:
+ for i, windowed in enumerate(
+ [
+ df.withColumn("wm", weighted_mean(v=df.v, w=df.w).over(w)),
+ df.withColumn("wm", weighted_mean(w=df.w, v=df.v).over(w)),
+ ]
+ ):
+ with self.subTest(bound=bound, query_no=i):
+ self.assertEqual(
+ windowed.collect(), df.withColumn("wm", sf.mean(df.v).over(w)).collect()
+ )
+
+ with self.tempView("v"):
+ df.createOrReplaceTempView("v")
+ self.spark.udf.register("weighted_mean", weighted_mean)
+
+ base_sql = "SELECT id, {func_call} OVER ({window_spec}) as wm FROM v"
+
+ for w in [
+ "ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING",
+ "ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING",
+ ]:
+ window_spec = f"PARTITION BY id ORDER BY v {w}"
+ with self.subTest(window_spec=window_spec):
+ for i, func_call in enumerate(
+ [
+ "weighted_mean(v => v, w => w)",
+ "weighted_mean(w => w, v => v)",
+ ]
+ ):
+ with self.subTest(query_no=i):
+ self.assertEqual(
+ self.spark.sql(
+ base_sql.format(func_call=func_call, window_spec=window_spec)
+ ).collect(),
+ self.spark.sql(
+ base_sql.format(func_call="mean(v)", window_spec=window_spec)
+ ).collect(),
+ )
+
+ # negative
+ with self.assertRaisesRegex(
+ AnalysisException,
+ "DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT.DOUBLE_NAMED_ARGUMENT_REFERENCE",
+ ):
+ self.spark.sql(
+ base_sql.format(
+ func_call="weighted_mean(v => v, v => w)", window_spec=window_spec
+ )
+ ).show()
+
+ with self.assertRaisesRegex(
+ AnalysisException, "UNEXPECTED_POSITIONAL_ARGUMENT"
+ ):
+ self.spark.sql(
+ base_sql.format(
+ func_call="weighted_mean(v => v, w)", window_spec=window_spec
+ )
+ ).show()
+
+ def test_complex_window_collect_set(self):
+ import pyarrow as pa
+
+ df = self.spark.createDataFrame([(1, 1), (1, 2), (2, 3), (2, 5), (2, 3)], ("id", "v"))
+ w = Window.partitionBy("id").orderBy("v")
+
+ @arrow_udf("array")
+ def arrow_collect_set(v: pa.Array) -> pa.Scalar:
+ assert isinstance(v, pa.Array), str(type(v))
+ s = sorted([x.as_py() for x in pa.compute.unique(v)])
+ t = pa.list_(pa.int32())
+ return pa.scalar(value=s, type=t)
+
+ result1 = df.select(
+ arrow_collect_set(df["v"]).over(w).alias("vs"),
+ )
+
+ expected1 = df.select(
+ sf.sort_array(sf.collect_set(df["v"]).over(w)).alias("vs"),
+ )
+
+ self.assertEqual(expected1.collect(), result1.collect())
+
+ def test_complex_window_collect_list(self):
+ import pyarrow as pa
+
+ df = self.spark.createDataFrame([(1, 1), (1, 2), (2, 3), (2, 5), (2, 3)], ("id", "v"))
+ w = Window.partitionBy("id").orderBy("v")
+
+ @arrow_udf("array")
+ def arrow_collect_list(v: pa.Array) -> pa.Scalar:
+ assert isinstance(v, pa.Array), str(type(v))
+ s = sorted([x.as_py() for x in v])
+ t = pa.list_(pa.int32())
+ return pa.scalar(value=s, type=t)
+
+ result1 = df.select(
+ arrow_collect_list(df["v"]).over(w).alias("vs"),
+ )
+
+ expected1 = df.select(
+ sf.sort_array(sf.collect_list(df["v"]).over(w)).alias("vs"),
+ )
+
+ self.assertEqual(expected1.collect(), result1.collect())
+
+ def test_complex_window_collect_as_map(self):
+ import pyarrow as pa
+
+ df = self.spark.createDataFrame(
+ [(1, 2, 1), (1, 3, 2), (2, 4, 3), (2, 5, 5), (2, 6, 3)], ("id", "k", "v")
+ )
+ w = Window.partitionBy("id").orderBy("v")
+
+ @arrow_udf("map")
+ def arrow_collect_as_map(id: pa.Array, v: pa.Array) -> pa.Scalar:
+ assert isinstance(id, pa.Array), str(type(id))
+ assert isinstance(v, pa.Array), str(type(v))
+ d = {i: j for i, j in zip(id.to_pylist(), v.to_pylist())}
+ t = pa.map_(pa.int32(), pa.int32())
+ return pa.scalar(value=d, type=t)
+
+ result1 = df.select(
+ arrow_collect_as_map("k", "v").over(w).alias("map"),
+ )
+
+ expected1 = df.select(
+ sf.map_from_arrays(
+ sf.collect_list("k").over(w),
+ sf.collect_list("v").over(w),
+ ).alias("map")
+ )
+
+ self.assertEqual(expected1.collect(), result1.collect())
+
+ def test_complex_window_min_max_struct(self):
+ import pyarrow as pa
+
+ df = self.spark.createDataFrame([(1, 1), (1, 2), (2, 3), (2, 5), (2, 3)], ("id", "v"))
+ w = Window.partitionBy("id").orderBy("v")
+
+ @arrow_udf("struct")
+ def arrow_collect_min_max(id: pa.Array, v: pa.Array) -> pa.Scalar:
+ assert isinstance(id, pa.Array), str(type(id))
+ assert isinstance(v, pa.Array), str(type(v))
+ m1 = pa.compute.min(id)
+ m2 = pa.compute.max(v)
+ t = pa.struct([pa.field("m1", pa.int32()), pa.field("m2", pa.int32())])
+ return pa.scalar(value={"m1": m1.as_py(), "m2": m2.as_py()}, type=t)
+
+ result1 = df.select(
+ arrow_collect_min_max("id", "v").over(w).alias("struct"),
+ )
+
+ expected1 = df.select(
+ sf.struct(
+ sf.min("id").over(w).alias("m1"),
+ sf.max("v").over(w).alias("m2"),
+ ).alias("struct")
+ )
+
+ self.assertEqual(expected1.collect(), result1.collect())
+
+ def test_time_min(self):
+ import pyarrow as pa
+
+ df = self.spark.sql(
+ """
+ SELECT * FROM VALUES
+ (1, TIME '12:34:56'),
+ (1, TIME '1:2:3'),
+ (2, TIME '0:58:59'),
+ (2, TIME '10:58:59'),
+ (2, TIME '10:00:03')
+ AS tab(i, t)
+ """
+ )
+ w1 = Window.partitionBy("i").orderBy("t")
+ w2 = Window.orderBy("t")
+
+ @arrow_udf("time", ArrowUDFType.GROUPED_AGG)
+ def agg_min_time(v):
+ assert isinstance(v, pa.Array)
+ assert isinstance(v, pa.Time64Array)
+ return pa.compute.min(v)
+
+ expected1 = df.withColumn("res", sf.min("t").over(w1))
+ result1 = df.withColumn("res", agg_min_time("t").over(w1))
+ self.assertEqual(expected1.collect(), result1.collect())
+
+ expected2 = df.withColumn("res", sf.min("t").over(w2))
+ result2 = df.withColumn("res", agg_min_time("t").over(w2))
+ self.assertEqual(expected2.collect(), result2.collect())
+
+ def test_return_type_coercion(self):
+ import pyarrow as pa
+
+ df = self.spark.range(10).withColumn("v", sf.lit(1))
+ w = Window.partitionBy("id").orderBy("v")
+
+ @arrow_udf("long", ArrowUDFType.GROUPED_AGG)
+ def agg_long(id: pa.Array) -> int:
+ assert isinstance(id, pa.Array), str(type(id))
+ return pa.scalar(value=len(id), type=pa.int64())
+
+ result1 = df.select(agg_long("v").over(w).alias("res"))
+ self.assertEqual(10, len(result1.collect()))
+
+ # long -> int coercion
+ @arrow_udf("int", ArrowUDFType.GROUPED_AGG)
+ def agg_int1(id: pa.Array) -> int:
+ assert isinstance(id, pa.Array), str(type(id))
+ return pa.scalar(value=len(id), type=pa.int64())
+
+ result2 = df.select(agg_int1("v").over(w).alias("res"))
+ self.assertEqual(10, len(result2.collect()))
+
+ # long -> int coercion, overflow
+ @arrow_udf("int", ArrowUDFType.GROUPED_AGG)
+ def agg_int2(id: pa.Array) -> int:
+ assert isinstance(id, pa.Array), str(type(id))
+ return pa.scalar(value=len(id) + 2147483647, type=pa.int64())
+
+ result3 = df.select(agg_int2("id").alias("res"))
+ with self.assertRaises(Exception):
+ # pyarrow.lib.ArrowInvalid:
+ # Integer value 2147483657 not in range: -2147483648 to 2147483647
+ result3.collect()
+
+
+class WindowArrowUDFTests(WindowArrowUDFTestsMixin, ReusedSQLTestCase):
+ pass
+
+
+if __name__ == "__main__":
+ from pyspark.sql.tests.arrow.test_arrow_udf_window import * # noqa: F401
+
+ try:
+ import xmlrunner
+
+ testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2)
+ except ImportError:
+ testRunner = None
+ unittest.main(testRunner=testRunner, verbosity=2)
diff --git a/python/pyspark/sql/tests/arrow/test_arrow_udtf.py b/python/pyspark/sql/tests/arrow/test_arrow_udtf.py
new file mode 100644
index 0000000000000..d655639faafe4
--- /dev/null
+++ b/python/pyspark/sql/tests/arrow/test_arrow_udtf.py
@@ -0,0 +1,624 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+import unittest
+from typing import Iterator
+
+from pyspark.errors import PySparkAttributeError
+from pyspark.errors import PythonException
+from pyspark.sql.functions import arrow_udtf, lit
+from pyspark.sql.types import Row, StructType, StructField, IntegerType
+from pyspark.testing.sqlutils import ReusedSQLTestCase, have_pyarrow, pyarrow_requirement_message
+from pyspark.testing import assertDataFrameEqual
+
+if have_pyarrow:
+ import pyarrow as pa
+ import pyarrow.compute as pc
+
+
+@unittest.skipIf(not have_pyarrow, pyarrow_requirement_message)
+class ArrowUDTFTestsMixin:
+ def test_arrow_udtf_zero_args(self):
+ @arrow_udtf(returnType="id int, value string")
+ class TestUDTF:
+ def eval(self) -> Iterator["pa.Table"]:
+ result_table = pa.table(
+ {
+ "id": pa.array([1, 2, 3], type=pa.int32()),
+ "value": pa.array(["a", "b", "c"], type=pa.string()),
+ }
+ )
+ yield result_table
+
+ # Test direct DataFrame API usage
+ result_df = TestUDTF()
+ expected_df = self.spark.createDataFrame(
+ [(1, "a"), (2, "b"), (3, "c")], "id int, value string"
+ )
+ assertDataFrameEqual(result_df, expected_df)
+
+ # Test SQL registration and usage
+ self.spark.udtf.register("test_zero_args_udtf", TestUDTF)
+ sql_result_df = self.spark.sql("SELECT * FROM test_zero_args_udtf()")
+ assertDataFrameEqual(sql_result_df, expected_df)
+
+ def test_arrow_udtf_scalar_args_only(self):
+ @arrow_udtf(returnType="x int, y int, sum int")
+ class ScalarArgsUDTF:
+ def eval(self, x: "pa.Array", y: "pa.Array") -> Iterator["pa.Table"]:
+ assert isinstance(x, pa.Array), f"Expected pa.Array, got {type(x)}"
+ assert isinstance(y, pa.Array), f"Expected pa.Array, got {type(y)}"
+
+ x_val = x[0].as_py()
+ y_val = y[0].as_py()
+ result_table = pa.table(
+ {
+ "x": pa.array([x_val], type=pa.int32()),
+ "y": pa.array([y_val], type=pa.int32()),
+ "sum": pa.array([x_val + y_val], type=pa.int32()),
+ }
+ )
+ yield result_table
+
+ # Test direct DataFrame API usage
+ result_df = ScalarArgsUDTF(lit(5), lit(10))
+ expected_df = self.spark.createDataFrame([(5, 10, 15)], "x int, y int, sum int")
+ assertDataFrameEqual(result_df, expected_df)
+
+ # Test SQL registration and usage
+ self.spark.udtf.register("ScalarArgsUDTF", ScalarArgsUDTF)
+ sql_result_df = self.spark.sql("SELECT * FROM ScalarArgsUDTF(5, 10)")
+ assertDataFrameEqual(sql_result_df, expected_df)
+
+ # Test with different values via SQL
+ sql_result_df2 = self.spark.sql("SELECT * FROM ScalarArgsUDTF(4, 7)")
+ expected_df2 = self.spark.createDataFrame([(4, 7, 11)], "x int, y int, sum int")
+ assertDataFrameEqual(sql_result_df2, expected_df2)
+
+ def test_arrow_udtf_record_batch_iterator(self):
+ @arrow_udtf(returnType="batch_id int, name string, count int")
+ class RecordBatchUDTF:
+ def eval(self, batch_size: "pa.Array") -> Iterator["pa.RecordBatch"]:
+ assert isinstance(
+ batch_size, pa.Array
+ ), f"Expected pa.Array, got {type(batch_size)}"
+
+ size = batch_size[0].as_py()
+
+ for batch_id in range(3):
+ # Create arrays for each column
+ batch_id_array = pa.array([batch_id] * size, type=pa.int32())
+ name_array = pa.array([f"batch_{batch_id}"] * size, type=pa.string())
+ count_array = pa.array(list(range(size)), type=pa.int32())
+
+ # Create record batch from arrays and names
+ batch = pa.record_batch(
+ [batch_id_array, name_array, count_array],
+ names=["batch_id", "name", "count"],
+ )
+ yield batch
+
+ # Test direct DataFrame API usage
+ result_df = RecordBatchUDTF(lit(2))
+ expected_data = [
+ (0, "batch_0", 0),
+ (0, "batch_0", 1),
+ (1, "batch_1", 0),
+ (1, "batch_1", 1),
+ (2, "batch_2", 0),
+ (2, "batch_2", 1),
+ ]
+ expected_df = self.spark.createDataFrame(
+ expected_data, "batch_id int, name string, count int"
+ )
+ assertDataFrameEqual(result_df, expected_df)
+
+ # Test SQL registration and usage
+ self.spark.udtf.register("record_batch_udtf", RecordBatchUDTF)
+ sql_result_df = self.spark.sql(
+ "SELECT * FROM record_batch_udtf(2) ORDER BY batch_id, count"
+ )
+ assertDataFrameEqual(sql_result_df, expected_df)
+
+ # Test with different batch size via SQL
+ sql_result_df2 = self.spark.sql("SELECT * FROM record_batch_udtf(1) ORDER BY batch_id")
+ expected_data2 = [
+ (0, "batch_0", 0),
+ (1, "batch_1", 0),
+ (2, "batch_2", 0),
+ ]
+ expected_df2 = self.spark.createDataFrame(
+ expected_data2, "batch_id int, name string, count int"
+ )
+ assertDataFrameEqual(sql_result_df2, expected_df2)
+
+ def test_arrow_udtf_error_not_iterator(self):
+ @arrow_udtf(returnType="x int, y string")
+ class NotIteratorUDTF:
+ def eval(self) -> "pa.Table":
+ return pa.table(
+ {"x": pa.array([1], type=pa.int32()), "y": pa.array(["test"], type=pa.string())}
+ )
+
+ with self.assertRaisesRegex(PythonException, "UDTF_RETURN_NOT_ITERABLE"):
+ result_df = NotIteratorUDTF()
+ result_df.collect()
+
+ def test_arrow_udtf_error_wrong_yield_type(self):
+ @arrow_udtf(returnType="x int, y string")
+ class WrongYieldTypeUDTF:
+ def eval(self) -> Iterator["pa.Table"]:
+ yield {"x": [1], "y": ["test"]}
+
+ with self.assertRaisesRegex(PythonException, "UDTF_ARROW_TYPE_CONVERSION_ERROR"):
+ result_df = WrongYieldTypeUDTF()
+ result_df.collect()
+
+ def test_arrow_udtf_error_invalid_arrow_type(self):
+ @arrow_udtf(returnType="x int, y string")
+ class InvalidArrowTypeUDTF:
+ def eval(self) -> Iterator["pa.Table"]:
+ yield "not_an_arrow_table"
+
+ with self.assertRaisesRegex(PythonException, "UDTF_ARROW_TYPE_CONVERSION_ERROR"):
+ result_df = InvalidArrowTypeUDTF()
+ result_df.collect()
+
+ def test_arrow_udtf_error_mismatched_schema(self):
+ @arrow_udtf(returnType="x int, y string")
+ class MismatchedSchemaUDTF:
+ def eval(self) -> Iterator["pa.Table"]:
+ result_table = pa.table(
+ {
+ "wrong_col": pa.array([1], type=pa.int32()),
+ "another_wrong_col": pa.array([2.5], type=pa.float64()),
+ }
+ )
+ yield result_table
+
+ with self.assertRaisesRegex(PythonException, "Schema at index 0 was different"):
+ result_df = MismatchedSchemaUDTF()
+ result_df.collect()
+
+ def test_arrow_udtf_sql_with_aggregation(self):
+ @arrow_udtf(returnType="category string, count int")
+ class CategoryCountUDTF:
+ def eval(self, categories: "pa.Array") -> Iterator["pa.Table"]:
+ # The input is a single array element, extract the array contents
+ cat_array = categories[0].as_py() # Get the array from the first (and only) element
+
+ # Count occurrences
+ counts = {}
+ for cat in cat_array:
+ if cat is not None:
+ counts[cat] = counts.get(cat, 0) + 1
+
+ if counts:
+ result_table = pa.table(
+ {
+ "category": pa.array(list(counts.keys()), type=pa.string()),
+ "count": pa.array(list(counts.values()), type=pa.int32()),
+ }
+ )
+ yield result_table
+
+ self.spark.udtf.register("category_count_udtf", CategoryCountUDTF)
+
+ # Test with array input
+ result_df = self.spark.sql(
+ "SELECT * FROM category_count_udtf(array('A', 'B', 'A', 'C', 'B', 'A')) "
+ "ORDER BY category"
+ )
+ expected_df = self.spark.createDataFrame(
+ [("A", 3), ("B", 2), ("C", 1)], "category string, count int"
+ )
+ assertDataFrameEqual(result_df, expected_df)
+
+ def test_arrow_udtf_sql_with_struct_output(self):
+ @arrow_udtf(returnType="person struct, status string")
+ class PersonStatusUDTF:
+ def eval(self, name: "pa.Array", age: "pa.Array") -> Iterator["pa.Table"]:
+ name_val = name[0].as_py()
+ age_val = age[0].as_py()
+
+ status = "adult" if age_val >= 18 else "minor"
+
+ # Create struct array
+ person_array = pa.array(
+ [{"name": name_val, "age": age_val}],
+ type=pa.struct([("name", pa.string()), ("age", pa.int32())]),
+ )
+
+ result_table = pa.table(
+ {
+ "person": person_array,
+ "status": pa.array([status], type=pa.string()),
+ }
+ )
+ yield result_table
+
+ self.spark.udtf.register("person_status_udtf", PersonStatusUDTF)
+
+ result_df = self.spark.sql("SELECT * FROM person_status_udtf('John', 25)")
+ # Note: Using Row constructor for the expected struct value
+ expected_df = self.spark.createDataFrame(
+ [(Row(name="John", age=25), "adult")],
+ "person struct, status string",
+ )
+ assertDataFrameEqual(result_df, expected_df)
+
+ def test_arrow_udtf_sql_conditional_yield(self):
+ @arrow_udtf(returnType="number int, type string")
+ class FilterNumbersUDTF:
+ def eval(self, start: "pa.Array", end: "pa.Array") -> Iterator["pa.Table"]:
+ start_val = start[0].as_py()
+ end_val = end[0].as_py()
+
+ numbers = []
+ types = []
+
+ for i in range(start_val, end_val + 1):
+ if i % 2 == 0: # Only yield even numbers
+ numbers.append(i)
+ types.append("even")
+
+ if numbers: # Only yield if we have data
+ result_table = pa.table(
+ {
+ "number": pa.array(numbers, type=pa.int32()),
+ "type": pa.array(types, type=pa.string()),
+ }
+ )
+ yield result_table
+
+ self.spark.udtf.register("filter_numbers_udtf", FilterNumbersUDTF)
+
+ result_df = self.spark.sql("SELECT * FROM filter_numbers_udtf(1, 10) ORDER BY number")
+ expected_df = self.spark.createDataFrame(
+ [(2, "even"), (4, "even"), (6, "even"), (8, "even"), (10, "even")],
+ "number int, type string",
+ )
+ assertDataFrameEqual(result_df, expected_df)
+
+ def test_arrow_udtf_sql_empty_result(self):
+ @arrow_udtf(returnType="value int")
+ class EmptyResultUDTF:
+ def eval(self, condition: "pa.Array") -> Iterator["pa.Table"]:
+ # Only yield if condition is true
+ if condition[0].as_py():
+ result_table = pa.table(
+ {
+ "value": pa.array([42], type=pa.int32()),
+ }
+ )
+ yield result_table
+ # If condition is false, don't yield anything
+
+ self.spark.udtf.register("empty_result_udtf", EmptyResultUDTF)
+
+ # Test with true condition
+ result_df_true = self.spark.sql("SELECT * FROM empty_result_udtf(true)")
+ expected_df_true = self.spark.createDataFrame([(42,)], "value int")
+ assertDataFrameEqual(result_df_true, expected_df_true)
+
+ # Test with false condition (empty result)
+ result_df_false = self.spark.sql("SELECT * FROM empty_result_udtf(false)")
+ expected_df_false = self.spark.createDataFrame([], "value int")
+ assertDataFrameEqual(result_df_false, expected_df_false)
+
+ def test_arrow_udtf_type_coercion_long_to_int(self):
+ @arrow_udtf(returnType="id int")
+ class LongToIntUDTF:
+ def eval(self) -> Iterator["pa.Table"]:
+ result_table = pa.table(
+ {
+ "id": pa.array([1, 2, 3], type=pa.int64()), # long values
+ }
+ )
+ yield result_table
+
+ with self.assertRaisesRegex(PythonException, "Schema at index 0 was different"):
+ result_df = LongToIntUDTF()
+ result_df.collect()
+
+ def test_arrow_udtf_type_coercion_string_to_int(self):
+ @arrow_udtf(returnType="id int")
+ class StringToIntUDTF:
+ def eval(self) -> Iterator["pa.Table"]:
+ # Return string values that cannot be coerced to int
+ result_table = pa.table(
+ {
+ "id": pa.array(["abc", "def", "xyz"], type=pa.string()),
+ }
+ )
+ yield result_table
+
+ with self.assertRaisesRegex(PythonException, "Schema at index 0 was different"):
+ result_df = StringToIntUDTF()
+ result_df.collect()
+
+ def test_arrow_udtf_with_empty_column_result(self):
+ @arrow_udtf(returnType=StructType())
+ class EmptyResultUDTF:
+ def eval(self) -> Iterator["pa.Table"]:
+ yield pa.Table.from_struct_array(pa.array([{}] * 3))
+
+ assertDataFrameEqual(EmptyResultUDTF(), [Row(), Row(), Row()])
+
+ @arrow_udtf(returnType="id int")
+ class InvalidEmptyResultUDTF:
+ def eval(self) -> Iterator["pa.Table"]:
+ yield pa.Table.from_struct_array(pa.array([{}] * 3))
+
+ with self.assertRaisesRegex(PythonException, "UDTF_RETURN_SCHEMA_MISMATCH"):
+ result_df = InvalidEmptyResultUDTF()
+ result_df.collect()
+
+ def test_arrow_udtf_blocks_analyze_method_none_return_type(self):
+ with self.assertRaises(PySparkAttributeError) as cm:
+
+ @arrow_udtf
+ class AnalyzeUDTF:
+ def eval(self, input_col: "pa.Array") -> Iterator["pa.Table"]:
+ yield pa.table({"result": pa.array([1, 2, 3])})
+
+ @staticmethod
+ def analyze(arg):
+ from pyspark.sql.udtf import AnalyzeResult
+
+ return AnalyzeResult(
+ schema=StructType([StructField("result", IntegerType(), True)])
+ )
+
+ self.assertIn("INVALID_ARROW_UDTF_WITH_ANALYZE", str(cm.exception))
+
+ def test_arrow_udtf_blocks_analyze_method_with_return_type(self):
+ with self.assertRaises(PySparkAttributeError) as cm:
+
+ @arrow_udtf(returnType="result: int")
+ class AnalyzeUDTF:
+ def eval(self, input_col: "pa.Array") -> Iterator["pa.Table"]:
+ yield pa.table({"result": pa.array([1, 2, 3])})
+
+ @staticmethod
+ def analyze(arg):
+ from pyspark.sql.udtf import AnalyzeResult
+
+ return AnalyzeResult(
+ schema=StructType([StructField("result", IntegerType(), True)])
+ )
+
+ self.assertIn("INVALID_UDTF_BOTH_RETURN_TYPE_AND_ANALYZE", str(cm.exception))
+
+ def test_arrow_udtf_with_table_argument_basic(self):
+ @arrow_udtf(returnType="filtered_id bigint") # Use bigint to match int64
+ class TableArgUDTF:
+ def eval(self, table_data: "pa.RecordBatch") -> Iterator["pa.Table"]:
+ assert isinstance(
+ table_data, pa.RecordBatch
+ ), f"Expected pa.RecordBatch, got {type(table_data)}"
+
+ # Convert record batch to table to work with it more easily
+ table = pa.table(table_data)
+
+ # Filter rows where id > 5
+ id_column = table.column("id")
+ mask = pa.compute.greater(id_column, pa.scalar(5))
+ filtered_table = table.filter(mask)
+
+ if filtered_table.num_rows > 0:
+ result_table = pa.table(
+ {"filtered_id": filtered_table.column("id")} # Keep original type (int64)
+ )
+ yield result_table
+
+ # Test with DataFrame API using asTable()
+ input_df = self.spark.range(8)
+ result_df = TableArgUDTF(input_df.asTable())
+ expected_df = self.spark.createDataFrame([(6,), (7,)], "filtered_id bigint")
+ assertDataFrameEqual(result_df, expected_df)
+
+ # Test SQL registration and usage with TABLE() syntax
+ self.spark.udtf.register("test_table_arg_udtf", TableArgUDTF)
+ sql_result_df = self.spark.sql(
+ "SELECT * FROM test_table_arg_udtf(TABLE(SELECT id FROM range(0, 8)))"
+ )
+ assertDataFrameEqual(sql_result_df, expected_df)
+
+ def test_arrow_udtf_with_table_argument_and_scalar(self):
+ @arrow_udtf(returnType="filtered_id bigint") # Use bigint to match int64
+ class MixedArgsUDTF:
+ def eval(
+ self, table_data: "pa.RecordBatch", threshold: "pa.Array"
+ ) -> Iterator["pa.Table"]:
+ assert isinstance(
+ threshold, pa.Array
+ ), f"Expected pa.Array for threshold, got {type(threshold)}"
+ assert isinstance(
+ table_data, pa.RecordBatch
+ ), f"Expected pa.RecordBatch for table_data, got {type(table_data)}"
+
+ threshold_val = threshold[0].as_py()
+
+ # Convert record batch to table
+ table = pa.table(table_data)
+ id_column = table.column("id")
+ mask = pa.compute.greater(id_column, pa.scalar(threshold_val))
+ filtered_table = table.filter(mask)
+
+ if filtered_table.num_rows > 0:
+ result_table = pa.table(
+ {"filtered_id": filtered_table.column("id")} # Keep original type
+ )
+ yield result_table
+
+ # # Test with DataFrame API
+ input_df = self.spark.range(8)
+ result_df = MixedArgsUDTF(input_df.asTable(), lit(5))
+ expected_df = self.spark.createDataFrame([(6,), (7,)], "filtered_id bigint")
+ assertDataFrameEqual(result_df, expected_df)
+
+ # Test SQL registration and usage
+ self.spark.udtf.register("test_mixed_args_udtf", MixedArgsUDTF)
+ sql_result_df = self.spark.sql(
+ "SELECT * FROM test_mixed_args_udtf(TABLE(SELECT id FROM range(0, 8)), 5)"
+ )
+ assertDataFrameEqual(sql_result_df, expected_df)
+
+ def test_arrow_udtf_lateral_join_disallowed(self):
+ @arrow_udtf(returnType="x int, result int")
+ class SimpleArrowUDTF:
+ def eval(self, input_val: "pa.Array") -> Iterator["pa.Table"]:
+ val = input_val[0].as_py()
+ result_table = pa.table(
+ {
+ "x": pa.array([val], type=pa.int32()),
+ "result": pa.array([val * 2], type=pa.int32()),
+ }
+ )
+ yield result_table
+
+ self.spark.udtf.register("simple_arrow_udtf", SimpleArrowUDTF)
+
+ test_df = self.spark.createDataFrame([(1,), (2,), (3,)], "id int")
+ test_df.createOrReplaceTempView("test_table")
+
+ with self.assertRaisesRegex(Exception, "LATERAL_JOIN_WITH_ARROW_UDTF_UNSUPPORTED"):
+ self.spark.sql(
+ """
+ SELECT t.id, f.x, f.result
+ FROM test_table t, LATERAL simple_arrow_udtf(t.id) f
+ """
+ )
+
+ def test_arrow_udtf_lateral_join_with_table_argument_disallowed(self):
+ @arrow_udtf(returnType="filtered_id bigint")
+ class MixedArgsUDTF:
+ def eval(self, input_table: "pa.Table") -> Iterator["pa.Table"]:
+ filtered_data = input_table.filter(pc.greater(input_table["id"], 5))
+ result_table = pa.table({"filtered_id": filtered_data["id"]})
+ yield result_table
+
+ self.spark.udtf.register("mixed_args_udtf", MixedArgsUDTF)
+
+ test_df1 = self.spark.createDataFrame([(1,), (2,), (3,)], "id int")
+ test_df1.createOrReplaceTempView("test_table1")
+
+ test_df2 = self.spark.createDataFrame([(6,), (7,), (8,)], "id bigint")
+ test_df2.createOrReplaceTempView("test_table2")
+
+ # Table arguments create nested lateral joins where our CheckAnalysis rule doesn't trigger
+ # because the Arrow UDTF is in the inner lateral join, not the outer one our rule checks.
+ # So Spark's general lateral join validation catches this first with
+ # NON_DETERMINISTIC_LATERAL_SUBQUERIES.
+ with self.assertRaisesRegex(
+ Exception,
+ "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.NON_DETERMINISTIC_LATERAL_SUBQUERIES",
+ ):
+ self.spark.sql(
+ """
+ SELECT t1.id, f.filtered_id
+ FROM test_table1 t1, LATERAL mixed_args_udtf(table(SELECT * FROM test_table2)) f
+ """
+ )
+
+ def test_arrow_udtf_with_table_argument_then_lateral_join_allowed(self):
+ @arrow_udtf(returnType="processed_id bigint")
+ class TableArgUDTF:
+ def eval(self, input_table: "pa.Table") -> Iterator["pa.Table"]:
+ processed_data = pc.add(input_table["id"], 100)
+ result_table = pa.table({"processed_id": processed_data})
+ yield result_table
+
+ self.spark.udtf.register("table_arg_udtf", TableArgUDTF)
+
+ source_df = self.spark.createDataFrame([(1,), (2,), (3,)], "id bigint")
+ source_df.createOrReplaceTempView("source_table")
+
+ join_df = self.spark.createDataFrame([("A",), ("B",), ("C",)], "label string")
+ join_df.createOrReplaceTempView("join_table")
+
+ result_df = self.spark.sql(
+ """
+ SELECT f.processed_id, j.label
+ FROM table_arg_udtf(table(SELECT * FROM source_table)) f,
+ join_table j
+ ORDER BY f.processed_id, j.label
+ """
+ )
+
+ expected_data = [
+ (101, "A"),
+ (101, "B"),
+ (101, "C"),
+ (102, "A"),
+ (102, "B"),
+ (102, "C"),
+ (103, "A"),
+ (103, "B"),
+ (103, "C"),
+ ]
+ expected_df = self.spark.createDataFrame(expected_data, "processed_id bigint, label string")
+ assertDataFrameEqual(result_df, expected_df)
+
+ def test_arrow_udtf_table_argument_with_regular_udtf_lateral_join_allowed(self):
+ @arrow_udtf(returnType="computed_value int")
+ class ComputeUDTF:
+ def eval(self, input_table: "pa.Table") -> Iterator["pa.Table"]:
+ total = pc.sum(input_table["value"]).as_py()
+ result_table = pa.table({"computed_value": pa.array([total], type=pa.int32())})
+ yield result_table
+
+ from pyspark.sql.functions import udtf
+ from pyspark.sql.types import StructType, StructField, IntegerType
+
+ @udtf(returnType=StructType([StructField("multiplied", IntegerType())]))
+ class MultiplyUDTF:
+ def eval(self, input_val: int):
+ yield (input_val * 3,)
+
+ self.spark.udtf.register("compute_udtf", ComputeUDTF)
+ self.spark.udtf.register("multiply_udtf", MultiplyUDTF)
+
+ values_df = self.spark.createDataFrame([(10,), (20,), (30,)], "value int")
+ values_df.createOrReplaceTempView("values_table")
+
+ result_df = self.spark.sql(
+ """
+ SELECT c.computed_value, m.multiplied
+ FROM compute_udtf(table(SELECT * FROM values_table) WITH SINGLE PARTITION) c,
+ LATERAL multiply_udtf(c.computed_value) m
+ """
+ )
+
+ expected_df = self.spark.createDataFrame([(60, 180)], "computed_value int, multiplied int")
+ assertDataFrameEqual(result_df, expected_df)
+
+
+class ArrowUDTFTests(ArrowUDTFTestsMixin, ReusedSQLTestCase):
+ pass
+
+
+if __name__ == "__main__":
+ from pyspark.sql.tests.arrow.test_arrow_udtf import * # noqa: F401
+
+ try:
+ import xmlrunner
+
+ testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2)
+ except ImportError:
+ testRunner = None
+ unittest.main(testRunner=testRunner, verbosity=2)
diff --git a/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_python_udf.py b/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_python_udf.py
index 8a5fe6131bd3d..6af55bb0b0bd1 100644
--- a/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_python_udf.py
+++ b/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_python_udf.py
@@ -33,6 +33,64 @@ def tearDownClass(cls):
super(ArrowPythonUDFParityTests, cls).tearDownClass()
+class ArrowPythonUDFParityLegacyTestsMixin(ArrowPythonUDFTestsMixin):
+ @classmethod
+ def setUpClass(cls):
+ super().setUpClass()
+ cls.spark.conf.set("spark.sql.legacy.execution.pythonUDF.pandas.conversion.enabled", "true")
+
+ @classmethod
+ def tearDownClass(cls):
+ try:
+ cls.spark.conf.unset("spark.sql.legacy.execution.pythonUDF.pandas.conversion.enabled")
+ finally:
+ super().tearDownClass()
+
+
+class ArrowPythonUDFParityNonLegacyTestsMixin(ArrowPythonUDFTestsMixin):
+ @classmethod
+ def setUpClass(cls):
+ super().setUpClass()
+ cls.spark.conf.set(
+ "spark.sql.legacy.execution.pythonUDF.pandas.conversion.enabled", "false"
+ )
+
+ @classmethod
+ def tearDownClass(cls):
+ try:
+ cls.spark.conf.unset("spark.sql.legacy.execution.pythonUDF.pandas.conversion.enabled")
+ finally:
+ super().tearDownClass()
+
+
+class ArrowPythonUDFParityLegacyTests(UDFParityTests, ArrowPythonUDFParityLegacyTestsMixin):
+ @classmethod
+ def setUpClass(cls):
+ super(ArrowPythonUDFParityLegacyTests, cls).setUpClass()
+ cls.spark.conf.set("spark.sql.execution.pythonUDF.arrow.enabled", "true")
+
+ @classmethod
+ def tearDownClass(cls):
+ try:
+ cls.spark.conf.unset("spark.sql.execution.pythonUDF.arrow.enabled")
+ finally:
+ super(ArrowPythonUDFParityLegacyTests, cls).tearDownClass()
+
+
+class ArrowPythonUDFParityNonLegacyTests(UDFParityTests, ArrowPythonUDFParityNonLegacyTestsMixin):
+ @classmethod
+ def setUpClass(cls):
+ super(ArrowPythonUDFParityNonLegacyTests, cls).setUpClass()
+ cls.spark.conf.set("spark.sql.execution.pythonUDF.arrow.enabled", "true")
+
+ @classmethod
+ def tearDownClass(cls):
+ try:
+ cls.spark.conf.unset("spark.sql.execution.pythonUDF.arrow.enabled")
+ finally:
+ super(ArrowPythonUDFParityNonLegacyTests, cls).tearDownClass()
+
+
if __name__ == "__main__":
import unittest
from pyspark.sql.tests.connect.arrow.test_parity_arrow_python_udf import * # noqa: F401
diff --git a/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_udf.py b/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_udf.py
index 567ab6913d30b..d3dd6d7f87037 100644
--- a/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_udf.py
+++ b/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_udf.py
@@ -15,12 +15,18 @@
# limitations under the License.
#
+import os
+import time
+
from pyspark.sql.tests.arrow.test_arrow_udf import ArrowUDFTestsMixin
from pyspark.testing.connectutils import ReusedConnectTestCase
class ArrowPythonUDFParityTests(ArrowUDFTestsMixin, ReusedConnectTestCase):
- pass
+ def setUp(self):
+ tz = "America/Los_Angeles"
+ os.environ["TZ"] = tz
+ time.tzset()
if __name__ == "__main__":
diff --git a/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_udf_grouped_agg.py b/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_udf_grouped_agg.py
new file mode 100644
index 0000000000000..fea43edb5f824
--- /dev/null
+++ b/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_udf_grouped_agg.py
@@ -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.
+#
+
+import os
+import time
+
+from pyspark.sql.tests.arrow.test_arrow_udf_grouped_agg import GroupedAggArrowUDFTestsMixin
+from pyspark.testing.connectutils import ReusedConnectTestCase
+
+
+class GroupedAggArrowPythonUDFParityTests(GroupedAggArrowUDFTestsMixin, ReusedConnectTestCase):
+ @classmethod
+ def setUpClass(cls):
+ ReusedConnectTestCase.setUpClass()
+
+ # Synchronize default timezone between Python and Java
+ cls.tz_prev = os.environ.get("TZ", None) # save current tz if set
+ tz = "America/Los_Angeles"
+ os.environ["TZ"] = tz
+ time.tzset()
+
+ cls.spark.conf.set("spark.sql.session.timeZone", tz)
+
+ @classmethod
+ def tearDownClass(cls):
+ del os.environ["TZ"]
+ if cls.tz_prev is not None:
+ os.environ["TZ"] = cls.tz_prev
+ time.tzset()
+ ReusedConnectTestCase.tearDownClass()
+
+
+if __name__ == "__main__":
+ import unittest
+ from pyspark.sql.tests.connect.arrow.test_parity_arrow_udf_grouped_agg import * # noqa: F401
+
+ try:
+ import xmlrunner # type: ignore[import]
+
+ testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2)
+ except ImportError:
+ testRunner = None
+ unittest.main(testRunner=testRunner, verbosity=2)
diff --git a/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_udf_window.py b/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_udf_window.py
new file mode 100644
index 0000000000000..5b834e291df51
--- /dev/null
+++ b/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_udf_window.py
@@ -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.
+#
+
+import os
+import time
+
+from pyspark.sql.tests.arrow.test_arrow_udf_window import WindowArrowUDFTestsMixin
+from pyspark.testing.connectutils import ReusedConnectTestCase
+
+
+class WindowArrowPythonUDFParityTests(WindowArrowUDFTestsMixin, ReusedConnectTestCase):
+ @classmethod
+ def setUpClass(cls):
+ ReusedConnectTestCase.setUpClass()
+
+ # Synchronize default timezone between Python and Java
+ cls.tz_prev = os.environ.get("TZ", None) # save current tz if set
+ tz = "America/Los_Angeles"
+ os.environ["TZ"] = tz
+ time.tzset()
+
+ cls.spark.conf.set("spark.sql.session.timeZone", tz)
+
+ @classmethod
+ def tearDownClass(cls):
+ del os.environ["TZ"]
+ if cls.tz_prev is not None:
+ os.environ["TZ"] = cls.tz_prev
+ time.tzset()
+ ReusedConnectTestCase.tearDownClass()
+
+
+if __name__ == "__main__":
+ import unittest
+ from pyspark.sql.tests.connect.arrow.test_parity_arrow_udf_window import * # noqa: F401
+
+ try:
+ import xmlrunner # type: ignore[import]
+
+ testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2)
+ except ImportError:
+ testRunner = None
+ unittest.main(testRunner=testRunner, verbosity=2)
diff --git a/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_udtf.py b/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_udtf.py
new file mode 100644
index 0000000000000..18227f493a0bd
--- /dev/null
+++ b/python/pyspark/sql/tests/connect/arrow/test_parity_arrow_udtf.py
@@ -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.
+#
+
+import unittest
+
+from pyspark.sql.tests.arrow.test_arrow_udtf import ArrowUDTFTestsMixin
+from pyspark.testing.connectutils import ReusedConnectTestCase
+
+
+class ArrowUDTFParityTests(ArrowUDTFTestsMixin, ReusedConnectTestCase):
+ # TODO(SPARK-53323): Support table arguments in Spark Connect Arrow UDTFs
+ @unittest.skip("asTable() is not supported in Spark Connect")
+ def test_arrow_udtf_with_table_argument_basic(self):
+ super().test_arrow_udtf_with_table_argument_basic()
+
+ # TODO(SPARK-53323): Support table arguments in Spark Connect Arrow UDTFs
+ @unittest.skip("asTable() is not supported in Spark Connect")
+ def test_arrow_udtf_with_table_argument_and_scalar(self):
+ super().test_arrow_udtf_with_table_argument_and_scalar()
+
+
+if __name__ == "__main__":
+ from pyspark.sql.tests.connect.arrow.test_parity_arrow_udtf import * # noqa: F401
+
+ try:
+ import xmlrunner # type: ignore[import]
+
+ testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2)
+ except ImportError:
+ testRunner = None
+ unittest.main(testRunner=testRunner, verbosity=2)
diff --git a/python/pyspark/sql/tests/connect/client/test_client.py b/python/pyspark/sql/tests/connect/client/test_client.py
index 647b950fd20f4..c3954827bae54 100644
--- a/python/pyspark/sql/tests/connect/client/test_client.py
+++ b/python/pyspark/sql/tests/connect/client/test_client.py
@@ -34,7 +34,8 @@
DefaultPolicy,
)
from pyspark.sql.connect.client.reattach import ExecutePlanResponseReattachableIterator
- from pyspark.errors import PySparkRuntimeError, RetriesExceeded
+ from pyspark.sql.connect.session import SparkSession as RemoteSparkSession
+ from pyspark.errors import PySparkRuntimeError
import pyspark.sql.connect.proto as proto
class TestPolicy(DefaultPolicy):
@@ -226,40 +227,43 @@ def test_is_closed(self):
client.close()
self.assertTrue(client.is_closed)
- def test_retry(self):
- client = SparkConnectClient("sc://foo/;token=bar")
-
- total_sleep = 0
-
- def sleep(t):
- nonlocal total_sleep
- total_sleep += t
-
- try:
- for attempt in Retrying(client._retry_policies, sleep=sleep):
- with attempt:
- raise TestException("Retryable error", grpc.StatusCode.UNAVAILABLE)
- except RetriesExceeded:
- pass
+ def test_channel_builder_with_session(self):
+ dummy = str(uuid.uuid4())
+ chan = DefaultChannelBuilder(f"sc://foo/;session_id={dummy}")
+ client = SparkConnectClient(chan)
+ self.assertEqual(client._session_id, chan.session_id)
- # tolerated at least 10 mins of fails
- self.assertGreaterEqual(total_sleep, 600)
+ def test_session_hook(self):
+ inits = 0
+ calls = 0
- def test_retry_client_unit(self):
- client = SparkConnectClient("sc://foo/;token=bar")
+ class TestHook(RemoteSparkSession.Hook):
+ def __init__(self, _session):
+ nonlocal inits
+ inits += 1
- policyA = TestPolicy()
- policyB = DefaultPolicy()
+ def on_execute_plan(self, req):
+ nonlocal calls
+ calls += 1
+ return req
- client.set_retry_policies([policyA, policyB])
+ session = (
+ RemoteSparkSession.builder.remote("sc://foo")._registerHook(TestHook).getOrCreate()
+ )
+ self.assertEqual(inits, 1)
+ self.assertEqual(calls, 0)
+ session.client._stub = MockService(session.client._session_id)
+ session.client.disable_reattachable_execute()
- self.assertEqual(client.get_retry_policies(), [policyA, policyB])
+ # Called from _execute_and_fetch_as_iterator
+ session.range(1).collect()
+ self.assertEqual(inits, 1)
+ self.assertEqual(calls, 1)
- def test_channel_builder_with_session(self):
- dummy = str(uuid.uuid4())
- chan = DefaultChannelBuilder(f"sc://foo/;session_id={dummy}")
- client = SparkConnectClient(chan)
- self.assertEqual(client._session_id, chan.session_id)
+ # Called from _execute
+ session.udf.register("test_func", lambda x: x + 1)
+ self.assertEqual(inits, 1)
+ self.assertEqual(calls, 2)
def test_custom_operation_id(self):
client = SparkConnectClient("sc://foo/;token=bar", use_reattachable_execute=False)
@@ -390,7 +394,7 @@ def not_found():
def checks():
self.assertEqual(2, stub.execute_calls)
self.assertEqual(0, stub.attach_calls)
- self.assertEqual(0, stub.release_calls)
+ self.assertEqual(1, stub.release_calls)
self.assertEqual(0, stub.release_until_calls)
eventually(timeout=1, catch_assertions=True)(checks)()
diff --git a/python/pyspark/sql/tests/connect/client/test_client_retries.py b/python/pyspark/sql/tests/connect/client/test_client_retries.py
new file mode 100644
index 0000000000000..fb3263e2824f7
--- /dev/null
+++ b/python/pyspark/sql/tests/connect/client/test_client_retries.py
@@ -0,0 +1,248 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+import unittest
+import warnings
+
+from pyspark.testing.connectutils import should_test_connect, connect_requirement_message
+
+if should_test_connect:
+ import grpc
+ import google.protobuf.any_pb2 as any_pb2
+ import google.protobuf.duration_pb2 as duration_pb2
+ from google.rpc import status_pb2
+ from google.rpc import error_details_pb2
+ from pyspark.sql.connect.client import SparkConnectClient
+ from pyspark.sql.connect.client.retries import (
+ Retrying,
+ DefaultPolicy,
+ )
+ from pyspark.sql.tests.connect.client.test_client import (
+ TestPolicy,
+ TestException,
+ )
+
+ class SleepTimeTracker:
+ """Tracks sleep times in ms for testing purposes."""
+
+ def __init__(self):
+ self._times = []
+
+ def sleep(self, t: float):
+ self._times.append(int(1000 * t))
+
+ @property
+ def times(self):
+ return list(self._times)
+
+ def create_test_exception_with_details(
+ msg: str,
+ code: grpc.StatusCode = grpc.StatusCode.INTERNAL,
+ retry_delay: int = 0,
+ ) -> TestException:
+ """Helper function for creating TestException with additional error details
+ like retry_delay.
+ """
+ retry_delay_msg = duration_pb2.Duration()
+ retry_delay_msg.FromMilliseconds(retry_delay)
+ retry_info = error_details_pb2.RetryInfo()
+ retry_info.retry_delay.CopyFrom(retry_delay_msg)
+
+ # Pack RetryInfo into an Any type
+ retry_info_any = any_pb2.Any()
+ retry_info_any.Pack(retry_info)
+ status = status_pb2.Status(
+ code=code.value[0],
+ message=msg,
+ details=[retry_info_any],
+ )
+ return TestException(msg=msg, code=code, trailing_status=status)
+
+ def get_client_policies_map(client: SparkConnectClient) -> dict:
+ return {type(policy): policy for policy in client.get_retry_policies()}
+
+
+@unittest.skipIf(not should_test_connect, connect_requirement_message)
+class SparkConnectClientRetriesTestCase(unittest.TestCase):
+ def assertListsAlmostEqual(self, first, second, places=None, msg=None, delta=None):
+ self.assertEqual(len(first), len(second), msg)
+ for i in range(len(first)):
+ self.assertAlmostEqual(first[i], second[i], places, msg, delta)
+
+ def test_retry(self):
+ client = SparkConnectClient("sc://foo/;token=bar")
+
+ sleep_tracker = SleepTimeTracker()
+ try:
+ for attempt in Retrying(client._retry_policies, sleep=sleep_tracker.sleep):
+ with attempt:
+ raise TestException("Retryable error", grpc.StatusCode.UNAVAILABLE)
+ except TestException:
+ pass
+
+ # tolerated at least 10 mins of fails
+ self.assertGreaterEqual(sum(sleep_tracker.times), 600)
+
+ def test_retry_client_unit(self):
+ client = SparkConnectClient("sc://foo/;token=bar")
+
+ policyA = TestPolicy()
+ policyB = DefaultPolicy()
+
+ client.set_retry_policies([policyA, policyB])
+
+ self.assertEqual(client.get_retry_policies(), [policyA, policyB])
+
+ def test_warning_works(self):
+ client = SparkConnectClient("sc://foo/;token=bar")
+ policy = get_client_policies_map(client).get(DefaultPolicy)
+ self.assertIsNotNone(policy)
+
+ sleep_tracker = SleepTimeTracker()
+ with warnings.catch_warnings(record=True) as warning_list:
+ warnings.simplefilter("always")
+ try:
+ for attempt in Retrying(client._retry_policies, sleep=sleep_tracker.sleep):
+ with attempt:
+ raise TestException(
+ msg="Some error message", code=grpc.StatusCode.UNAVAILABLE
+ )
+ except TestException:
+ pass
+ self.assertEqual(len(sleep_tracker.times), policy.max_retries)
+ self.assertEqual(len(warning_list), 1)
+ self.assertEqual(
+ str(warning_list[0].message),
+ "[RETRIES_EXCEEDED] The maximum number of retries has been exceeded.",
+ )
+
+ def test_default_policy_retries_retry_info(self):
+ client = SparkConnectClient("sc://foo/;token=bar")
+ policy = get_client_policies_map(client).get(DefaultPolicy)
+ self.assertIsNotNone(policy)
+
+ # retry delay = 0, error code not matched by any policy.
+ # Testing if errors with RetryInfo are being retried by the DefaultPolicy.
+ retry_delay = 0
+ sleep_tracker = SleepTimeTracker()
+ try:
+ for attempt in Retrying(client._retry_policies, sleep=sleep_tracker.sleep):
+ with attempt:
+ raise create_test_exception_with_details(
+ msg="Some error message",
+ code=grpc.StatusCode.UNIMPLEMENTED,
+ retry_delay=retry_delay,
+ )
+ except TestException:
+ pass
+ expected_times = [
+ min(policy.max_backoff, policy.initial_backoff * policy.backoff_multiplier**i)
+ for i in range(policy.max_retries)
+ ]
+ self.assertListsAlmostEqual(sleep_tracker.times, expected_times, delta=policy.jitter)
+
+ def test_retry_delay_overrides_max_backoff(self):
+ client = SparkConnectClient("sc://foo/;token=bar")
+ policy = get_client_policies_map(client).get(DefaultPolicy)
+ self.assertIsNotNone(policy)
+
+ # retry delay = 5 mins.
+ # Testing if retry_delay overrides max_backoff.
+ retry_delay = 5 * 60 * 1000
+ sleep_tracker = SleepTimeTracker()
+ # assert that retry_delay is greater than max_backoff to make sure the test is valid
+ self.assertGreaterEqual(retry_delay, policy.max_backoff)
+ try:
+ for attempt in Retrying(client._retry_policies, sleep=sleep_tracker.sleep):
+ with attempt:
+ raise create_test_exception_with_details(
+ "Some error message",
+ grpc.StatusCode.UNAVAILABLE,
+ retry_delay,
+ )
+ except TestException:
+ pass
+ expected_times = [retry_delay] * policy.max_retries
+ self.assertListsAlmostEqual(sleep_tracker.times, expected_times, delta=policy.jitter)
+
+ def test_max_server_retry_delay(self):
+ client = SparkConnectClient("sc://foo/;token=bar")
+ policy = get_client_policies_map(client).get(DefaultPolicy)
+ self.assertIsNotNone(policy)
+
+ # retry delay = 10 hours
+ # Testing if max_server_retry_delay limit works.
+ retry_delay = 10 * 60 * 60 * 1000
+ sleep_tracker = SleepTimeTracker()
+ try:
+ for attempt in Retrying(client._retry_policies, sleep=sleep_tracker.sleep):
+ with attempt:
+ raise create_test_exception_with_details(
+ "Some error message",
+ grpc.StatusCode.UNAVAILABLE,
+ retry_delay,
+ )
+ except TestException:
+ pass
+
+ expected_times = [policy.max_server_retry_delay] * policy.max_retries
+ self.assertListsAlmostEqual(sleep_tracker.times, expected_times, delta=policy.jitter)
+
+ def test_return_to_exponential_backoff(self):
+ client = SparkConnectClient("sc://foo/;token=bar")
+ policy = get_client_policies_map(client).get(DefaultPolicy)
+ self.assertIsNotNone(policy)
+
+ # Start with retry_delay = 5 mins, then set it to zero.
+ # Test if backoff goes back to client's exponential strategy.
+ initial_retry_delay = 5 * 60 * 1000
+ sleep_tracker = SleepTimeTracker()
+ try:
+ for i, attempt in enumerate(
+ Retrying(client._retry_policies, sleep=sleep_tracker.sleep)
+ ):
+ if i < 2:
+ retry_delay = initial_retry_delay
+ elif i < 5:
+ retry_delay = 0
+ else:
+ break
+ with attempt:
+ raise create_test_exception_with_details(
+ "Some error message",
+ grpc.StatusCode.UNAVAILABLE,
+ retry_delay,
+ )
+ except TestException:
+ pass
+
+ expected_times = [initial_retry_delay] * 2 + [
+ policy.initial_backoff * policy.backoff_multiplier**i for i in range(2, 5)
+ ]
+ self.assertListsAlmostEqual(sleep_tracker.times, expected_times, delta=policy.jitter)
+
+
+if __name__ == "__main__":
+ from pyspark.sql.tests.connect.client.test_client_retries import * # noqa: F401
+
+ try:
+ import xmlrunner # type: ignore
+
+ testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2)
+ except ImportError:
+ testRunner = None
+ unittest.main(testRunner=testRunner, verbosity=2)
diff --git a/python/pyspark/sql/tests/connect/test_connect_column.py b/python/pyspark/sql/tests/connect/test_connect_column.py
index 4873006fbbb90..8983d45d42d14 100644
--- a/python/pyspark/sql/tests/connect/test_connect_column.py
+++ b/python/pyspark/sql/tests/connect/test_connect_column.py
@@ -25,6 +25,7 @@
MapType,
NullType,
DateType,
+ TimeType,
TimestampType,
TimestampNTZType,
ByteType,
@@ -396,6 +397,7 @@ def test_literal_with_acceptable_type(self):
("sss", StringType()),
(datetime.date(2022, 12, 13), DateType()),
(datetime.datetime.now(), DateType()),
+ (datetime.time(1, 0, 0), TimeType()),
(datetime.datetime.now(), TimestampType()),
(datetime.datetime.now(), TimestampNTZType()),
(datetime.timedelta(1, 2, 3), DayTimeIntervalType()),
@@ -441,6 +443,7 @@ def test_literal_null(self):
DoubleType(),
DecimalType(),
DateType(),
+ TimeType(),
TimestampType(),
TimestampNTZType(),
DayTimeIntervalType(),
diff --git a/python/pyspark/sql/tests/connect/test_connect_creation.py b/python/pyspark/sql/tests/connect/test_connect_creation.py
index 26e3596fc67df..917320d354e2b 100644
--- a/python/pyspark/sql/tests/connect/test_connect_creation.py
+++ b/python/pyspark/sql/tests/connect/test_connect_creation.py
@@ -530,12 +530,15 @@ def test_create_dataframe_from_pandas_with_ns_timestamp(self):
from pandas import Timestamp
import pandas as pd
+ # Nanoseconds are truncated to microseconds in the serializer
+ # Arrow will throw an error if precision is lost
+ # (i.e., nanoseconds cannot be represented in microseconds)
pdf = pd.DataFrame(
{
"naive": [datetime(2019, 1, 1, 0)],
"aware": [
Timestamp(
- year=2019, month=1, day=1, nanosecond=500, tz=timezone(timedelta(hours=-8))
+ year=2019, month=1, day=1, nanosecond=0, tz=timezone(timedelta(hours=-8))
)
],
}
diff --git a/python/pyspark/sql/tests/connect/test_connect_dataframe_property.py b/python/pyspark/sql/tests/connect/test_connect_dataframe_property.py
index 76007137bc7a6..d5f5f994fe124 100644
--- a/python/pyspark/sql/tests/connect/test_connect_dataframe_property.py
+++ b/python/pyspark/sql/tests/connect/test_connect_dataframe_property.py
@@ -72,6 +72,15 @@ def test_cached_property_is_copied(self):
df_columns.remove(col)
assert len(df.columns) == 4
+ cdf = self.connect.createDataFrame(data, schema)
+ cdf_schema = cdf.schema
+ assert len(cdf._cached_schema_serialized) > 0
+ assert cdf_schema.jsonValue() == cdf._cached_schema.jsonValue()
+ assert len(cdf_schema.fields) == 4
+ cdf_schema.fields.pop(0)
+ assert cdf.schema.jsonValue() == cdf._cached_schema.jsonValue()
+ assert len(cdf.schema.fields) == 4
+
def test_cached_schema_to(self):
rows = [Row(id=x, name=str(x)) for x in range(100)]
cdf = self.connect.createDataFrame(rows)
diff --git a/python/pyspark/sql/tests/connect/test_connect_function.py b/python/pyspark/sql/tests/connect/test_connect_function.py
index 20ce6b88e390a..b906f5c5cef44 100644
--- a/python/pyspark/sql/tests/connect/test_connect_function.py
+++ b/python/pyspark/sql/tests/connect/test_connect_function.py
@@ -551,8 +551,6 @@ def test_aggregation_functions(self):
(CF.approx_count_distinct, SF.approx_count_distinct),
(CF.approxCountDistinct, SF.approxCountDistinct),
(CF.avg, SF.avg),
- (CF.collect_list, SF.collect_list),
- (CF.collect_set, SF.collect_set),
(CF.listagg, SF.listagg),
(CF.listagg_distinct, SF.listagg_distinct),
(CF.string_agg, SF.string_agg),
@@ -589,6 +587,25 @@ def test_aggregation_functions(self):
check_exact=False,
)
+ for cfunc, sfunc in [
+ (CF.collect_list, SF.collect_list),
+ (CF.collect_set, SF.collect_set),
+ ]:
+ self.assert_eq(
+ cdf.select(CF.sort_array(cfunc("b")), CF.sort_array(cfunc(cdf.c))).toPandas(),
+ sdf.select(SF.sort_array(sfunc("b")), SF.sort_array(sfunc(sdf.c))).toPandas(),
+ check_exact=False,
+ )
+ self.assert_eq(
+ cdf.groupBy("a")
+ .agg(CF.sort_array(cfunc("b")), CF.sort_array(cfunc(cdf.c)))
+ .toPandas(),
+ sdf.groupBy("a")
+ .agg(SF.sort_array(sfunc("b")), SF.sort_array(sfunc(sdf.c)))
+ .toPandas(),
+ check_exact=False,
+ )
+
for cfunc, sfunc in [
(CF.corr, SF.corr),
(CF.covar_pop, SF.covar_pop),
@@ -601,8 +618,8 @@ def test_aggregation_functions(self):
sdf.select(sfunc(sdf.b, "c")).toPandas(),
)
self.assert_eq(
- cdf.groupBy("a").agg(cfunc(cdf.b, "c")).toPandas(),
- sdf.groupBy("a").agg(sfunc(sdf.b, "c")).toPandas(),
+ cdf.groupBy("a").agg(cfunc(cdf.b, "c")).orderBy("a").toPandas(),
+ sdf.groupBy("a").agg(sfunc(sdf.b, "c")).orderBy("a").toPandas(),
)
# test grouping
@@ -628,13 +645,13 @@ def test_aggregation_functions(self):
check_exact=False,
)
self.assert_eq(
- cdf.groupBy("a").agg(CF.percentile_approx("b", 0.5)).toPandas(),
- sdf.groupBy("a").agg(SF.percentile_approx("b", 0.5)).toPandas(),
+ cdf.groupBy("a").agg(CF.percentile_approx("b", 0.5)).orderBy("a").toPandas(),
+ sdf.groupBy("a").agg(SF.percentile_approx("b", 0.5)).orderBy("a").toPandas(),
check_exact=False,
)
self.assert_eq(
- cdf.groupBy("a").agg(CF.percentile_approx(cdf.b, [0.1, 0.9])).toPandas(),
- sdf.groupBy("a").agg(SF.percentile_approx(sdf.b, [0.1, 0.9])).toPandas(),
+ cdf.groupBy("a").agg(CF.percentile_approx(cdf.b, [0.1, 0.9])).orderBy("a").toPandas(),
+ sdf.groupBy("a").agg(SF.percentile_approx(sdf.b, [0.1, 0.9])).orderBy("a").toPandas(),
check_exact=False,
)
@@ -652,9 +669,11 @@ def test_aggregation_functions(self):
self.assert_eq(
cdf.groupBy("a")
.agg(CF.count_distinct("b").alias("x"), CF.count_distinct(cdf.c).alias("y"))
+ .orderBy("a")
.toPandas(),
sdf.groupBy("a")
.agg(SF.count_distinct("b").alias("x"), SF.count_distinct(sdf.c).alias("y"))
+ .orderBy("a")
.toPandas(),
)
diff --git a/python/pyspark/sql/tests/connect/test_connect_plan.py b/python/pyspark/sql/tests/connect/test_connect_plan.py
index a03cd30c733fb..d25799f0c9f26 100644
--- a/python/pyspark/sql/tests/connect/test_connect_plan.py
+++ b/python/pyspark/sql/tests/connect/test_connect_plan.py
@@ -893,7 +893,7 @@ def test_float_nan_inf(self):
self.assertIsNotNone(inf_lit.to_plan(None))
def test_datetime_literal_types(self):
- """Test the different timestamp, date, and timedelta types."""
+ """Test the different timestamp, date, time, and timedelta types."""
datetime_lit = lit(datetime.datetime.now())
p = datetime_lit.to_plan(None)
@@ -908,6 +908,10 @@ def test_datetime_literal_types(self):
# (24 * 3600 + 2) * 1000000 + 3
self.assertEqual(86402000003, time_delta.to_plan(None).literal.day_time_interval)
+ time_lit = lit(datetime.time(23, 59, 59, 999999))
+ self.assertIsNotNone(time_lit.to_plan(None))
+ self.assertEqual(time_lit.to_plan(None).literal.time.nano, 86399999999000)
+
def test_list_to_literal(self):
"""Test conversion of lists to literals"""
empty_list = []
@@ -1024,6 +1028,7 @@ def test_literal_to_any_conversion(self):
decimal.Decimal(1.234567),
"sss",
datetime.date(2022, 12, 13),
+ datetime.time(12, 13, 14),
datetime.datetime.now(),
datetime.timedelta(1, 2, 3),
[1, 2, 3, 4, 5, 6],
diff --git a/python/pyspark/sql/tests/connect/test_connect_retry.py b/python/pyspark/sql/tests/connect/test_connect_retry.py
index f51e062479284..21796869e385d 100644
--- a/python/pyspark/sql/tests/connect/test_connect_retry.py
+++ b/python/pyspark/sql/tests/connect/test_connect_retry.py
@@ -18,7 +18,6 @@
import unittest
from collections import defaultdict
-from pyspark.errors import RetriesExceeded
from pyspark.testing.connectutils import (
should_test_connect,
connect_requirement_message,
@@ -88,7 +87,7 @@ def test_below_limit(self):
def test_exceed_retries(self):
# Exceed the retries.
- with self.assertRaises(RetriesExceeded):
+ with self.assertRaises(TestError):
for attempt in Retrying(TestPolicy(max_retries=2)):
with attempt:
self.stub(5, grpc.StatusCode.INTERNAL)
@@ -117,7 +116,7 @@ def test_specific_exception(self):
def test_specific_exception_exceed_retries(self):
# Exceed the retries.
policy = TestPolicySpecificError(max_retries=2, specific_code=grpc.StatusCode.UNAVAILABLE)
- with self.assertRaises(RetriesExceeded):
+ with self.assertRaises(TestError):
for attempt in Retrying(policy):
with attempt:
self.stub(5, grpc.StatusCode.UNAVAILABLE)
@@ -157,13 +156,13 @@ def test_multiple_policies_exceed(self):
policy1 = TestPolicySpecificError(max_retries=2, specific_code=grpc.StatusCode.INTERNAL)
policy2 = TestPolicySpecificError(max_retries=4, specific_code=grpc.StatusCode.INTERNAL)
- with self.assertRaises(RetriesExceeded):
+ with self.assertRaises(TestError):
for attempt in Retrying([policy1, policy2]):
with attempt:
self.stub(10, grpc.StatusCode.INTERNAL)
- self.assertEqual(self.call_wrap["attempts"], 7)
- self.assertEqual(self.call_wrap["raised"], 7)
+ self.assertEqual(self.call_wrap["attempts"], 3)
+ self.assertEqual(self.call_wrap["raised"], 3)
if __name__ == "__main__":
diff --git a/python/pyspark/sql/tests/connect/test_parity_udf_combinations.py b/python/pyspark/sql/tests/connect/test_parity_udf_combinations.py
new file mode 100644
index 0000000000000..bc63aa7aeb505
--- /dev/null
+++ b/python/pyspark/sql/tests/connect/test_parity_udf_combinations.py
@@ -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.
+#
+
+import unittest
+
+from pyspark.sql.tests.test_udf_combinations import UDFCombinationsTestsMixin
+from pyspark.testing.connectutils import ReusedConnectTestCase
+
+
+class UDFCombinationsParityTests(UDFCombinationsTestsMixin, ReusedConnectTestCase):
+ @classmethod
+ def setUpClass(cls):
+ ReusedConnectTestCase.setUpClass()
+ cls.spark.conf.set("spark.sql.execution.pythonUDF.arrow.enabled", "false")
+
+
+if __name__ == "__main__":
+ from pyspark.sql.tests.connect.test_parity_udf_combinations import * # noqa: F401
+
+ try:
+ import xmlrunner # type: ignore[import]
+
+ testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2)
+ except ImportError:
+ testRunner = None
+ unittest.main(testRunner=testRunner, verbosity=2)
diff --git a/python/pyspark/sql/tests/pandas/helper/helper_pandas_transform_with_state.py b/python/pyspark/sql/tests/pandas/helper/helper_pandas_transform_with_state.py
index 8c35e0e3b486f..d258f693ccb88 100644
--- a/python/pyspark/sql/tests/pandas/helper/helper_pandas_transform_with_state.py
+++ b/python/pyspark/sql/tests/pandas/helper/helper_pandas_transform_with_state.py
@@ -31,6 +31,8 @@
LongType,
BooleanType,
FloatType,
+ ArrayType,
+ MapType,
)
from pyspark.testing.sqlutils import have_pandas
@@ -227,6 +229,14 @@ def row(self):
return RowMinEventTimeStatefulProcessor()
+class StatefulProcessorCompositeTypeFactory(StatefulProcessorFactory):
+ def pandas(self):
+ return PandasStatefulProcessorCompositeType()
+
+ def row(self):
+ return RowStatefulProcessorCompositeType()
+
+
# StatefulProcessor implementations
@@ -1615,3 +1625,200 @@ def handleInputRows(self, key, rows, timerValues) -> Iterator[Row]:
def close(self) -> None:
pass
+
+
+# A stateful processor that contains composite python type inside Value, List and Map state variable
+class PandasStatefulProcessorCompositeType(StatefulProcessor):
+ TAGS = [["dummy1", "dummy2"], ["dummy3"]]
+ METADATA = [{"key": "env", "value": "prod"}, {"key": "region", "value": "us-west"}]
+ ATTRIBUTES_MAP = {"key1": [1], "key2": [10]}
+ CONFS_MAP = {"e1": {"e2": 5, "e3": 10}}
+
+ def init(self, handle: StatefulProcessorHandle) -> None:
+ obj_schema = StructType(
+ [
+ StructField("id", ArrayType(IntegerType())),
+ StructField("tags", ArrayType(ArrayType(StringType()))),
+ StructField(
+ "metadata",
+ ArrayType(
+ StructType(
+ [StructField("key", StringType()), StructField("value", StringType())]
+ )
+ ),
+ ),
+ ]
+ )
+
+ map_value_schema = StructType(
+ [
+ StructField("id", IntegerType(), True),
+ StructField("attributes", MapType(StringType(), ArrayType(IntegerType())), True),
+ StructField(
+ "confs", MapType(StringType(), MapType(StringType(), IntegerType()), True), True
+ ),
+ ]
+ )
+
+ self.obj_state = handle.getValueState("obj_state", obj_schema)
+ self.list_state = handle.getListState("list_state", obj_schema)
+ self.map_state = handle.getMapState("map_state", "name string", map_value_schema)
+
+ def _update_obj_state(self, total_temperature):
+ if self.obj_state.exists():
+ ids, tags, metadata = self.obj_state.get()
+ assert tags == self.TAGS, f"Tag mismatch: {tags}"
+ assert metadata == [Row(**m) for m in self.METADATA], f"Metadata mismatch: {metadata}"
+ ids = [int(x + total_temperature) for x in ids]
+ else:
+ ids = [0]
+ self.obj_state.update((ids, self.TAGS, self.METADATA))
+ return ids
+
+ def _update_list_state(self, total_temperature, initial_obj):
+ existing_list = self.list_state.get()
+ updated_list = []
+ for ids, tags, metadata in existing_list:
+ ids.append(total_temperature)
+ updated_list.append((ids, tags, [row.asDict() for row in metadata]))
+ if not updated_list:
+ updated_list.append(initial_obj)
+ self.list_state.put(updated_list)
+ return [id_val for ids, _, _ in updated_list for id_val in ids]
+
+ def _update_map_state(self, key, total_temperature):
+ if not self.map_state.containsKey(key):
+ self.map_state.updateValue(key, (0, self.ATTRIBUTES_MAP, self.CONFS_MAP))
+ else:
+ id_val, attributes, confs = self.map_state.getValue(key)
+ attributes[key] = [total_temperature]
+ confs.setdefault("e1", {})[key] = total_temperature
+ self.map_state.updateValue(key, (id_val, attributes, confs))
+ return self.map_state.getValue(key)[1], self.map_state.getValue(key)[2]
+
+ def handleInputRows(self, key, rows, timerValues) -> Iterator[pd.DataFrame]:
+ key = key[0]
+ total_temperature = sum(pdf["temperature"].astype(int).sum() for pdf in rows)
+
+ updated_ids = self._update_obj_state(total_temperature)
+ flattened_ids = self._update_list_state(
+ total_temperature, (updated_ids, self.TAGS, self.METADATA)
+ )
+ attributes_map, confs_map = self._update_map_state(key, total_temperature)
+
+ import json
+ import numpy as np
+
+ def np_int64_to_int(x):
+ if isinstance(x, np.int64):
+ return int(x)
+ return x
+
+ yield pd.DataFrame(
+ {
+ "id": [key],
+ "value_arr": [",".join(map(str, updated_ids))],
+ "list_state_arr": [",".join(map(str, flattened_ids))],
+ "map_state_arr": [
+ json.dumps(attributes_map, default=np_int64_to_int, sort_keys=True)
+ ],
+ "nested_map_state_arr": [
+ json.dumps(confs_map, default=np_int64_to_int, sort_keys=True)
+ ],
+ }
+ )
+
+ def close(self) -> None:
+ pass
+
+
+class RowStatefulProcessorCompositeType(StatefulProcessor):
+ TAGS = [["dummy1", "dummy2"], ["dummy3"]]
+ METADATA = [{"key": "env", "value": "prod"}, {"key": "region", "value": "us-west"}]
+ ATTRIBUTES_MAP = {"key1": [1], "key2": [10]}
+ CONFS_MAP = {"e1": {"e2": 5, "e3": 10}}
+
+ def init(self, handle: StatefulProcessorHandle) -> None:
+ obj_schema = StructType(
+ [
+ StructField("id", ArrayType(IntegerType())),
+ StructField("tags", ArrayType(ArrayType(StringType()))),
+ StructField(
+ "metadata",
+ ArrayType(
+ StructType(
+ [StructField("key", StringType()), StructField("value", StringType())]
+ )
+ ),
+ ),
+ ]
+ )
+
+ map_value_schema = StructType(
+ [
+ StructField("id", IntegerType(), True),
+ StructField("attributes", MapType(StringType(), ArrayType(IntegerType())), True),
+ StructField(
+ "confs", MapType(StringType(), MapType(StringType(), IntegerType()), True), True
+ ),
+ ]
+ )
+
+ self.obj_state = handle.getValueState("obj_state", obj_schema)
+ self.list_state = handle.getListState("list_state", obj_schema)
+ self.map_state = handle.getMapState("map_state", "name string", map_value_schema)
+
+ def _update_obj_state(self, total_temperature):
+ if self.obj_state.exists():
+ ids, tags, metadata = self.obj_state.get()
+ assert tags == self.TAGS, f"Tag mismatch: {tags}"
+ assert metadata == [Row(**m) for m in self.METADATA], f"Metadata mismatch: {metadata}"
+ ids = [int(x + total_temperature) for x in ids]
+ else:
+ ids = [0]
+ self.obj_state.update((ids, self.TAGS, self.METADATA))
+ return ids
+
+ def _update_list_state(self, total_temperature, initial_obj):
+ existing_list = self.list_state.get()
+ updated_list = []
+ for ids, tags, metadata in existing_list:
+ ids.append(total_temperature)
+ updated_list.append((ids, tags, [row.asDict() for row in metadata]))
+ if not updated_list:
+ updated_list.append(initial_obj)
+ self.list_state.put(updated_list)
+ return [id_val for ids, _, _ in updated_list for id_val in ids]
+
+ def _update_map_state(self, key, total_temperature):
+ if not self.map_state.containsKey(key):
+ self.map_state.updateValue(key, (0, self.ATTRIBUTES_MAP, self.CONFS_MAP))
+ else:
+ id_val, attributes, confs = self.map_state.getValue(key)
+ attributes[key] = [total_temperature]
+ confs.setdefault("e1", {})[key] = total_temperature
+ self.map_state.updateValue(key, (id_val, attributes, confs))
+ return self.map_state.getValue(key)[1], self.map_state.getValue(key)[2]
+
+ def handleInputRows(self, key, rows, timerValues) -> Iterator[Row]:
+ key = key[0]
+ total_temperature = sum(int(row.temperature) for row in rows)
+
+ updated_ids = self._update_obj_state(total_temperature)
+ flattened_ids = self._update_list_state(
+ total_temperature, (updated_ids, self.TAGS, self.METADATA)
+ )
+ attributes_map, confs_map = self._update_map_state(key, total_temperature)
+
+ import json
+
+ yield Row(
+ id=key,
+ value_arr=",".join(map(str, updated_ids)),
+ list_state_arr=",".join(map(str, flattened_ids)),
+ map_state_arr=json.dumps(attributes_map, sort_keys=True),
+ nested_map_state_arr=json.dumps(confs_map, sort_keys=True),
+ )
+
+ def close(self) -> None:
+ pass
diff --git a/python/pyspark/sql/tests/pandas/test_pandas_cogrouped_map.py b/python/pyspark/sql/tests/pandas/test_pandas_cogrouped_map.py
index 1f9532352679a..d23252abf6a92 100644
--- a/python/pyspark/sql/tests/pandas/test_pandas_cogrouped_map.py
+++ b/python/pyspark/sql/tests/pandas/test_pandas_cogrouped_map.py
@@ -262,7 +262,7 @@ def check_apply_in_pandas_returning_incompatible_type(self):
"`spark.sql.execution.pandas.convertToArrowArraySafely`."
)
self._test_merge_error(
- fn=lambda lft, rgt: pd.DataFrame({"id": [1], "k": ["2.0"]}),
+ fn=lambda lft, rgt: pd.DataFrame({"id": [1], "k": ["test_string"]}),
output_schema="id long, k double",
errorClass=PythonException,
error_message_regex=expected,
@@ -281,6 +281,54 @@ def check_apply_in_pandas_returning_incompatible_type(self):
error_message_regex=expected,
)
+ def test_cogroup_apply_int_to_decimal_coercion(self):
+ left = self.data1.limit(3)
+ right = self.data2.limit(3)
+
+ def int_to_decimal_merge(lft, rgt):
+ return pd.DataFrame(
+ [
+ {
+ "id": 1,
+ "decimal_result": 98765,
+ "left_count": len(lft),
+ "right_count": len(rgt),
+ }
+ ]
+ )
+
+ with self.sql_conf(
+ {"spark.sql.execution.pythonUDF.pandas.intToDecimalCoercionEnabled": True}
+ ):
+ result = (
+ left.groupby("id")
+ .cogroup(right.groupby("id"))
+ .applyInPandas(
+ int_to_decimal_merge,
+ "id long, decimal_result decimal(10,2), left_count long, right_count long",
+ )
+ .collect()
+ )
+ self.assertTrue(len(result) > 0)
+ for row in result:
+ self.assertEqual(row.decimal_result, 98765.00)
+
+ with self.sql_conf(
+ {"spark.sql.execution.pythonUDF.pandas.intToDecimalCoercionEnabled": False}
+ ):
+ with self.assertRaisesRegex(
+ PythonException, "Exception thrown when converting pandas.Series"
+ ):
+ (
+ left.groupby("id")
+ .cogroup(right.groupby("id"))
+ .applyInPandas(
+ int_to_decimal_merge,
+ "id long, decimal_result decimal(10,2), left_count long, right_count long",
+ )
+ .collect()
+ )
+
def test_mixed_scalar_udfs_followed_by_cogrouby_apply(self):
df = self.spark.range(0, 10).toDF("v1")
df = df.withColumn("v2", udf(lambda x: x + 1, "int")(df["v1"])).withColumn(
diff --git a/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py b/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py
index 4ef334549ef5a..b8f1afc5130f9 100644
--- a/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py
+++ b/python/pyspark/sql/tests/pandas/test_pandas_grouped_map.py
@@ -221,8 +221,10 @@ def check_register_grouped_map_udf(self):
exception=pe.exception,
errorClass="INVALID_UDF_EVAL_TYPE",
messageParameters={
- "eval_type": "SQL_BATCHED_UDF, SQL_ARROW_BATCHED_UDF, SQL_SCALAR_PANDAS_UDF, "
- "SQL_SCALAR_PANDAS_ITER_UDF or SQL_GROUPED_AGG_PANDAS_UDF"
+ "eval_type": "SQL_BATCHED_UDF, SQL_ARROW_BATCHED_UDF, "
+ "SQL_SCALAR_PANDAS_UDF, SQL_SCALAR_ARROW_UDF, "
+ "SQL_SCALAR_PANDAS_ITER_UDF, SQL_SCALAR_ARROW_ITER_UDF, "
+ "SQL_GROUPED_AGG_PANDAS_UDF or SQL_GROUPED_AGG_ARROW_UDF"
},
)
@@ -370,7 +372,7 @@ def check_apply_in_pandas_returning_incompatible_type(self):
)
with self.assertRaisesRegex(PythonException, expected + "\n"):
self._test_apply_in_pandas(
- lambda key, pdf: pd.DataFrame([key + (str(pdf.v.mean()),)]),
+ lambda key, pdf: pd.DataFrame([key + ("test_string",)]),
output_schema="id long, mean double",
)
@@ -386,6 +388,37 @@ def check_apply_in_pandas_returning_incompatible_type(self):
output_schema="id long, mean string",
)
+ def test_apply_in_pandas_int_to_decimal_coercion(self):
+ def int_to_decimal_func(key, pdf):
+ return pd.DataFrame([{"id": key[0], "decimal_result": 12345}])
+
+ with self.sql_conf(
+ {"spark.sql.execution.pythonUDF.pandas.intToDecimalCoercionEnabled": True}
+ ):
+ result = (
+ self.data.groupby("id")
+ .applyInPandas(int_to_decimal_func, schema="id long, decimal_result decimal(10,2)")
+ .collect()
+ )
+
+ self.assertTrue(len(result) > 0)
+ for row in result:
+ self.assertEqual(row.decimal_result, 12345.00)
+
+ with self.sql_conf(
+ {"spark.sql.execution.pythonUDF.pandas.intToDecimalCoercionEnabled": False}
+ ):
+ with self.assertRaisesRegex(
+ PythonException, "Exception thrown when converting pandas.Series"
+ ):
+ (
+ self.data.groupby("id")
+ .applyInPandas(
+ int_to_decimal_func, schema="id long, decimal_result decimal(10,2)"
+ )
+ .collect()
+ )
+
def test_datatype_string(self):
df = self.data
@@ -868,6 +901,51 @@ def _test_apply_in_pandas_returning_empty_dataframe_error(self, empty_df, error)
with self.assertRaisesRegex(PythonException, error):
self._test_apply_in_pandas_returning_empty_dataframe(empty_df)
+ def test_arrow_cast_enabled_numeric_to_decimal(self):
+ import numpy as np
+
+ columns = [
+ "int8",
+ "int16",
+ "int32",
+ "uint8",
+ "uint16",
+ "uint32",
+ "float64",
+ ]
+
+ pdf = pd.DataFrame({key: np.arange(1, 2).astype(key) for key in columns})
+ df = self.spark.range(2).repartition(1)
+
+ for column in columns:
+ with self.subTest(column=column):
+ v = pdf[column].iloc[:1]
+ schema_str = "id long, value decimal(10,0)"
+
+ @pandas_udf(schema_str, PandasUDFType.GROUPED_MAP)
+ def test(pdf):
+ return pdf.assign(**{"value": v})
+
+ row = df.groupby("id").apply(test).first()
+ res = row[1]
+ self.assertEqual(res, Decimal("1"))
+
+ def test_arrow_cast_enabled_str_to_numeric(self):
+ df = self.spark.range(2).repartition(1)
+
+ types = ["int", "long", "float", "double"]
+
+ for type_str in types:
+ with self.subTest(type=type_str):
+ schema_str = "id long, value " + type_str
+
+ @pandas_udf(schema_str, PandasUDFType.GROUPED_MAP)
+ def test(pdf):
+ return pdf.assign(value=pd.Series(["123"]))
+
+ row = df.groupby("id").apply(test).first()
+ self.assertEqual(row[1], 123)
+
class GroupedApplyInPandasTests(GroupedApplyInPandasTestsMixin, ReusedSQLTestCase):
pass
diff --git a/python/pyspark/sql/tests/pandas/test_pandas_grouped_map_with_state.py b/python/pyspark/sql/tests/pandas/test_pandas_grouped_map_with_state.py
index e1b8d7c76d183..e4f307ebc96ae 100644
--- a/python/pyspark/sql/tests/pandas/test_pandas_grouped_map_with_state.py
+++ b/python/pyspark/sql/tests/pandas/test_pandas_grouped_map_with_state.py
@@ -23,6 +23,7 @@
import unittest
from typing import cast
+from decimal import Decimal
from pyspark.sql.streaming.state import GroupStateTimeout, GroupState
from pyspark.sql.types import (
@@ -31,6 +32,7 @@
StructType,
StructField,
Row,
+ DecimalType,
)
from pyspark.testing.sqlutils import (
ReusedSQLTestCase,
@@ -59,7 +61,12 @@ def conf(cls):
cfg.set("spark.sql.shuffle.partitions", "5")
return cfg
- def _test_apply_in_pandas_with_state_basic(self, func, check_results):
+ def _test_apply_in_pandas_with_state_basic(self, func, check_results, output_type=None):
+ if output_type is None:
+ output_type = StructType(
+ [StructField("key", StringType()), StructField("countAsString", StringType())]
+ )
+
input_path = tempfile.mkdtemp()
def prepare_test_resource():
@@ -75,9 +82,6 @@ def prepare_test_resource():
q.stop()
self.assertTrue(df.isStreaming)
- output_type = StructType(
- [StructField("key", StringType()), StructField("countAsString", StringType())]
- )
state_type = StructType([StructField("c", LongType())])
q = (
@@ -314,6 +318,26 @@ def assert_test():
finally:
q.stop()
+ def test_apply_in_pandas_with_state_int_to_decimal_coercion(self):
+ def func(key, pdf_iter, state):
+ assert isinstance(state, GroupState)
+ yield pd.DataFrame({"key": [key[0]], "decimal_sum": [1]})
+
+ def check_results(batch_df, _):
+ assert set(batch_df.sort("key").collect()) == {
+ Row(key="hello", decimal_sum=Decimal("1.00")),
+ Row(key="this", decimal_sum=Decimal("1.00")),
+ }, "Decimal coercion failed: " + str(batch_df.sort("key").collect())
+
+ output_type = StructType(
+ [StructField("key", StringType()), StructField("decimal_sum", DecimalType(10, 2))]
+ )
+
+ with self.sql_conf(
+ {"spark.sql.execution.pythonUDF.pandas.intToDecimalCoercionEnabled": True}
+ ):
+ self._test_apply_in_pandas_with_state_basic(func, check_results, output_type)
+
class GroupedApplyInPandasWithStateTests(
GroupedApplyInPandasWithStateTestsMixin, ReusedSQLTestCase
diff --git a/python/pyspark/sql/tests/pandas/test_pandas_map.py b/python/pyspark/sql/tests/pandas/test_pandas_map.py
index 7debe8035f611..b241b91e02a29 100644
--- a/python/pyspark/sql/tests/pandas/test_pandas_map.py
+++ b/python/pyspark/sql/tests/pandas/test_pandas_map.py
@@ -276,16 +276,17 @@ def test_dataframes_with_incompatible_types(self):
self.check_dataframes_with_incompatible_types()
def check_dataframes_with_incompatible_types(self):
- def func(iterator):
- for pdf in iterator:
- yield pdf.assign(id=pdf["id"].apply(str))
-
for safely in [True, False]:
with self.subTest(convertToArrowArraySafely=safely), self.sql_conf(
{"spark.sql.execution.pandas.convertToArrowArraySafely": safely}
):
# sometimes we see ValueErrors
with self.subTest(convert="string to double"):
+
+ def func(iterator):
+ for pdf in iterator:
+ yield pdf.assign(id="test_string")
+
expected = (
r"ValueError: Exception thrown when converting pandas.Series "
r"\(object\) with name 'id' to Arrow Array \(double\)."
@@ -304,18 +305,31 @@ def func(iterator):
.collect()
)
- # sometimes we see TypeErrors
- with self.subTest(convert="double to string"):
- with self.assertRaisesRegex(
- PythonException,
- r"TypeError: Exception thrown when converting pandas.Series "
- r"\(float64\) with name 'id' to Arrow Array \(string\).\n",
- ):
- (
- self.spark.range(10, numPartitions=3)
- .select(col("id").cast("double"))
- .mapInPandas(self.identity_dataframes_iter("id"), "id string")
- .collect()
+ with self.subTest(convert="float to int precision loss"):
+
+ def func(iterator):
+ for pdf in iterator:
+ yield pdf.assign(id=pdf["id"] + 0.1)
+
+ df = (
+ self.spark.range(10, numPartitions=3)
+ .select(col("id").cast("double"))
+ .mapInPandas(func, "id int")
+ )
+ if safely:
+ expected = (
+ r"ValueError: Exception thrown when converting pandas.Series "
+ r"\(float64\) with name 'id' to Arrow Array \(int32\)."
+ " It can be caused by overflows or other "
+ "unsafe conversions warned by Arrow. Arrow safe type check "
+ "can be disabled by using SQL config "
+ "`spark.sql.execution.pandas.convertToArrowArraySafely`."
+ )
+ with self.assertRaisesRegex(PythonException, expected + "\n"):
+ df.collect()
+ else:
+ self.assertEqual(
+ df.collect(), self.spark.range(10, numPartitions=3).collect()
)
def test_empty_iterator(self):
diff --git a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py
index 007ed5de2fbd7..d3bda545e1c99 100644
--- a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py
+++ b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py
@@ -35,6 +35,7 @@
Row,
IntegerType,
TimestampType,
+ DecimalType,
)
from pyspark.testing import assertDataFrameEqual
from pyspark.testing.sqlutils import (
@@ -68,6 +69,7 @@
ReorderedFieldsProcessorFactory,
UpcastProcessorFactory,
MinEventTimeStatefulProcessorFactory,
+ StatefulProcessorCompositeTypeFactory,
)
@@ -138,6 +140,12 @@ def _test_transform_with_state_basic(
timeMode="None",
checkpoint_path=None,
initial_state=None,
+ output_schema=StructType(
+ [
+ StructField("id", StringType(), True),
+ StructField("countAsString", StringType(), True),
+ ]
+ ),
):
input_path = tempfile.mkdtemp()
if checkpoint_path is None:
@@ -153,13 +161,6 @@ def _test_transform_with_state_basic(
q.stop()
self.assertTrue(df.isStreaming)
- output_schema = StructType(
- [
- StructField("id", StringType(), True),
- StructField("countAsString", StringType(), True),
- ]
- )
-
stateful_processor = self.get_processor(stateful_processor_factory)
if self.use_pandas():
tws_df = df.groupBy("id").transformWithStateInPandas(
@@ -194,6 +195,7 @@ def _test_transform_with_state_basic(
def test_transform_with_state_basic(self):
def check_results(batch_df, batch_id):
+ batch_df.collect()
if batch_id == 0:
assert set(batch_df.sort("id").collect()) == {
Row(id="0", countAsString="2"),
@@ -209,6 +211,7 @@ def check_results(batch_df, batch_id):
def test_transform_with_state_non_exist_value_state(self):
def check_results(batch_df, _):
+ batch_df.collect()
assert set(batch_df.sort("id").collect()) == {
Row(id="0", countAsString="0"),
Row(id="1", countAsString="0"),
@@ -294,6 +297,7 @@ def test_transform_with_state_query_restarts(self):
def test_transform_with_state_list_state(self):
def check_results(batch_df, _):
+ batch_df.collect()
assert set(batch_df.sort("id").collect()) == {
Row(id="0", countAsString="2"),
Row(id="1", countAsString="2"),
@@ -305,6 +309,7 @@ def check_results(batch_df, _):
def test_transform_with_state_list_state_large_list(self):
def check_results(batch_df, batch_id):
+ batch_df.collect()
if batch_id == 0:
expected_prev_elements = ""
expected_updated_elements = ",".join(map(lambda x: str(x), range(90)))
@@ -379,6 +384,7 @@ def check_results(batch_df, batch_id):
# test list state with ttl has the same behavior as list state when state doesn't expire.
def test_transform_with_state_list_state_large_ttl(self):
def check_results(batch_df, batch_id):
+ batch_df.collect()
assert set(batch_df.sort("id").collect()) == {
Row(id="0", countAsString="2"),
Row(id="1", countAsString="2"),
@@ -390,6 +396,7 @@ def check_results(batch_df, batch_id):
def test_transform_with_state_map_state(self):
def check_results(batch_df, _):
+ batch_df.collect()
assert set(batch_df.sort("id").collect()) == {
Row(id="0", countAsString="2"),
Row(id="1", countAsString="2"),
@@ -400,6 +407,7 @@ def check_results(batch_df, _):
# test map state with ttl has the same behavior as map state when state doesn't expire.
def test_transform_with_state_map_state_large_ttl(self):
def check_results(batch_df, batch_id):
+ batch_df.collect()
assert set(batch_df.sort("id").collect()) == {
Row(id="0", countAsString="2"),
Row(id="1", countAsString="2"),
@@ -413,6 +421,7 @@ def check_results(batch_df, batch_id):
# state doesn't expire.
def test_value_state_ttl_basic(self):
def check_results(batch_df, batch_id):
+ batch_df.collect()
if batch_id == 0:
assert set(batch_df.sort("id").collect()) == {
Row(id="0", countAsString="2"),
@@ -432,6 +441,7 @@ def check_results(batch_df, batch_id):
@unittest.skip("test is flaky and it is only a timing issue, skipping until we can resolve")
def test_value_state_ttl_expiration(self):
def check_results(batch_df, batch_id):
+ batch_df.collect()
if batch_id == 0:
assertDataFrameEqual(
batch_df,
@@ -580,6 +590,8 @@ def _test_transform_with_state_proc_timer(self, stateful_processor_factory, chec
def test_transform_with_state_proc_timer(self):
def check_results(batch_df, batch_id):
+ batch_df.collect()
+
# helper function to check expired timestamp is smaller than current processing time
def check_timestamp(batch_df):
expired_df = (
@@ -695,6 +707,7 @@ def prepare_batch3(input_path):
def test_transform_with_state_event_time(self):
def check_results(batch_df, batch_id):
+ batch_df.collect()
if batch_id == 0:
# watermark for late event = 0
# watermark for eviction = 0
@@ -726,6 +739,7 @@ def check_results(batch_df, batch_id):
def test_transform_with_state_with_wmark_and_non_event_time(self):
def check_results(batch_df, batch_id):
+ batch_df.collect()
if batch_id == 0:
# watermark for late event = 0 and min event = 20
assert set(batch_df.sort("id").collect()) == {
@@ -823,6 +837,7 @@ def _test_transform_with_state_init_state(
def test_transform_with_state_init_state(self):
def check_results(batch_df, batch_id):
+ batch_df.collect()
if batch_id == 0:
# for key 0, initial state was processed and it was only processed once;
# for key 1, it did not appear in the initial state df;
@@ -846,6 +861,7 @@ def check_results(batch_df, batch_id):
def test_transform_with_state_init_state_with_extra_transformation(self):
def check_results(batch_df, batch_id):
+ batch_df.collect()
if batch_id == 0:
# for key 0, initial state was processed and it was only processed once;
# for key 1, it did not appear in the initial state df;
@@ -924,6 +940,7 @@ def _test_transform_with_state_non_contiguous_grouping_cols(
def test_transform_with_state_non_contiguous_grouping_cols(self):
def check_results(batch_df, batch_id):
+ batch_df.collect()
assert set(batch_df.collect()) == {
Row(id1="0", id2="1", value=str(123 + 46)),
Row(id1="1", id2="2", value=str(146 + 346)),
@@ -935,6 +952,7 @@ def check_results(batch_df, batch_id):
def test_transform_with_state_non_contiguous_grouping_cols_with_init_state(self):
def check_results(batch_df, batch_id):
+ batch_df.collect()
# initial state for key (0, 1) is processed
assert set(batch_df.collect()) == {
Row(id1="0", id2="1", value=str(789 + 123 + 46)),
@@ -1017,6 +1035,7 @@ def _test_transform_with_state_chaining_ops(
def test_transform_with_state_chaining_ops(self):
def check_results(batch_df, batch_id):
+ batch_df.collect()
import datetime
if batch_id == 0:
@@ -1052,6 +1071,7 @@ def check_results(batch_df, batch_id):
def test_transform_with_state_init_state_with_timers(self):
def check_results(batch_df, batch_id):
+ batch_df.collect()
if batch_id == 0:
# timers are registered and handled in the first batch for
# rows in initial state; For key=0 and key=3 which contains
@@ -1176,6 +1196,7 @@ def _test_transform_with_map_state_metadata(self, initial_state):
expected_operator_name = "transformWithStateInPySparkExec"
def check_results(batch_df, batch_id):
+ batch_df.collect()
if batch_id == 0:
assert set(batch_df.sort("id").collect()) == {
Row(id="0", countAsString="2"),
@@ -1292,6 +1313,7 @@ def test_transform_with_list_state_metadata(self):
checkpoint_path = tempfile.mktemp()
def check_results(batch_df, batch_id):
+ batch_df.collect()
if batch_id == 0:
assert set(batch_df.sort("id").collect()) == {
Row(id="0", countAsString="2"),
@@ -1371,6 +1393,7 @@ def test_transform_with_value_state_metadata(self):
checkpoint_path = tempfile.mktemp()
def check_results(batch_df, batch_id):
+ batch_df.collect()
if batch_id == 0:
assert set(batch_df.sort("id").collect()) == {
Row(id="0", countAsString="2"),
@@ -1458,12 +1481,14 @@ def check_results(batch_df, batch_id):
def test_transform_with_state_restart_with_multiple_rows_init_state(self):
def check_results(batch_df, _):
+ batch_df.collect()
assert set(batch_df.sort("id").collect()) == {
Row(id="0", countAsString="2"),
Row(id="1", countAsString="2"),
}
def check_results_for_new_query(batch_df, batch_id):
+ batch_df.collect()
if batch_id == 0:
assert set(batch_df.sort("id").collect()) == {
Row(id="0", value=str(123 + 46)),
@@ -1513,6 +1538,63 @@ def dataframe_to_value_list(output_df):
initial_state=init_df,
)
+ def test_transform_with_state_in_pandas_composite_type(self):
+ def check_results(batch_df, batch_id):
+ if batch_id == 0:
+ map_val = {"key1": [1], "key2": [10]}
+ nested_map_val = {"e1": {"e2": 5, "e3": 10}}
+ assert set(batch_df.sort("id").collect()) == {
+ Row(
+ id="0",
+ value_arr="0",
+ list_state_arr="0",
+ map_state_arr=json.dumps(map_val, sort_keys=True),
+ nested_map_state_arr=json.dumps(nested_map_val, sort_keys=True),
+ ),
+ Row(
+ id="1",
+ value_arr="0",
+ list_state_arr="0",
+ map_state_arr=json.dumps(map_val, sort_keys=True),
+ nested_map_state_arr=json.dumps(nested_map_val, sort_keys=True),
+ ),
+ }, f"batch id: {batch_id}, real df is: {batch_df.collect()}"
+ else:
+ map_val_0 = {"key1": [1], "key2": [10], "0": [669]}
+ map_val_1 = {"key1": [1], "key2": [10], "1": [252]}
+ nested_map_val_0 = {"e1": {"e2": 5, "e3": 10, "0": 669}}
+ nested_map_val_1 = {"e1": {"e2": 5, "e3": 10, "1": 252}}
+ assert set(batch_df.sort("id").collect()) == {
+ Row(
+ id="0",
+ countAsString="669",
+ list_state_arr="0,669",
+ map_state_arr=json.dumps(map_val_0, sort_keys=True),
+ nested_map_state_arr=json.dumps(nested_map_val_0, sort_keys=True),
+ ),
+ Row(
+ id="1",
+ countAsString="252",
+ list_state_arr="0,252",
+ map_state_arr=json.dumps(map_val_1, sort_keys=True),
+ nested_map_state_arr=json.dumps(nested_map_val_1, sort_keys=True),
+ ),
+ }, f"batch id: {batch_id}, real df is: {batch_df.collect()}"
+
+ output_schema = StructType(
+ [
+ StructField("id", StringType(), True),
+ StructField("value_arr", StringType(), True),
+ StructField("list_state_arr", StringType(), True),
+ StructField("map_state_arr", StringType(), True),
+ StructField("nested_map_state_arr", StringType(), True),
+ ]
+ )
+
+ self._test_transform_with_state_basic(
+ StatefulProcessorCompositeTypeFactory(), check_results, output_schema=output_schema
+ )
+
# run the same test suites again but with single shuffle partition
def test_transform_with_state_with_timers_single_partition(self):
with self.sql_conf({"spark.sql.shuffle.partitions": "1"}):
@@ -1714,6 +1796,74 @@ def check_basic_state(batch_df, batch_id):
and "column family state must be nullable" in error_msg
)
+ def test_transform_with_state_int_to_decimal_coercion(self):
+ if not self.use_pandas():
+ return
+
+ class IntToDecimalProcessor(StatefulProcessor):
+ def init(self, handle):
+ count_schema = StructType([StructField("value", IntegerType(), True)])
+ self.count_state = handle.getValueState("count", count_schema)
+
+ def handleInputRows(self, key, rows, timerValues):
+ if self.count_state.exists():
+ count = self.count_state.get()[0]
+ else:
+ count = 0
+ count += len(list(rows))
+ self.count_state.update((count,))
+
+ import pandas as pd
+
+ yield pd.DataFrame(
+ {"id": [key[0]], "decimal_result": [12345]} # Integer to be coerced to decimal
+ )
+
+ def close(self):
+ pass
+
+ data = [("1", "a"), ("1", "b"), ("2", "c")]
+ df = self.spark.createDataFrame(data, ["id", "value"])
+
+ output_schema = StructType(
+ [
+ StructField("id", StringType(), True),
+ StructField("decimal_result", DecimalType(10, 2), True),
+ ]
+ )
+
+ with self.sql_conf(
+ {"spark.sql.execution.pythonUDF.pandas.intToDecimalCoercionEnabled": True}
+ ):
+ result = (
+ df.groupBy("id")
+ .transformWithStateInPandas(
+ statefulProcessor=IntToDecimalProcessor(),
+ outputStructType=output_schema,
+ outputMode="Update",
+ timeMode="None",
+ )
+ .collect()
+ )
+ self.assertTrue(len(result) > 0)
+
+ with self.sql_conf(
+ {"spark.sql.execution.pythonUDF.pandas.intToDecimalCoercionEnabled": False}
+ ):
+ with self.assertRaisesRegex(
+ Exception, "Exception thrown when converting pandas.Series"
+ ):
+ (
+ df.groupBy("id")
+ .transformWithStateInPandas(
+ statefulProcessor=IntToDecimalProcessor(),
+ outputStructType=output_schema,
+ outputMode="Update",
+ timeMode="None",
+ )
+ .collect()
+ )
+
@unittest.skipIf(
not have_pyarrow or os.environ.get("PYTHON_GIL", "?") == "0",
diff --git a/python/pyspark/sql/tests/pandas/test_pandas_udf.py b/python/pyspark/sql/tests/pandas/test_pandas_udf.py
index c2769bf882b6b..23fceb746114f 100644
--- a/python/pyspark/sql/tests/pandas/test_pandas_udf.py
+++ b/python/pyspark/sql/tests/pandas/test_pandas_udf.py
@@ -222,12 +222,20 @@ def foo(x):
with self.assertRaisesRegex(ValueError, "0-arg pandas_udfs.*not.*supported"):
pandas_udf(lambda: 1, LongType(), PandasUDFType.SCALAR)
+
with self.assertRaisesRegex(ValueError, "0-arg pandas_udfs.*not.*supported"):
@pandas_udf(LongType(), PandasUDFType.SCALAR)
def zero_with_type():
return 1
+ with self.assertRaisesRegex(ValueError, "0-arg pandas_udfs.*not.*supported"):
+
+ @pandas_udf(LongType(), PandasUDFType.SCALAR_ITER)
+ def zero_with_type():
+ yield 1
+ yield 2
+
with self.assertRaises(PySparkTypeError) as pe:
@pandas_udf(returnType=PandasUDFType.GROUPED_MAP)
@@ -343,6 +351,54 @@ def udf(column):
with self.sql_conf({"spark.sql.execution.pandas.convertToArrowArraySafely": False}):
df.withColumn("udf", udf("id")).collect()
+ def test_pandas_udf_int_to_decimal_coercion(self):
+ import pandas as pd
+ from decimal import Decimal
+
+ df = self.spark.range(0, 3)
+
+ @pandas_udf(returnType="decimal(10,2)")
+ def int_to_decimal_udf(column):
+ values = [123, 456, 789]
+ return pd.Series([values[int(val) % len(values)] for val in column])
+
+ with self.sql_conf(
+ {"spark.sql.execution.pythonUDF.pandas.intToDecimalCoercionEnabled": True}
+ ):
+ result = df.withColumn("decimal_val", int_to_decimal_udf("id")).collect()
+ self.assertEqual(result[0]["decimal_val"], 123.00)
+ self.assertEqual(result[1]["decimal_val"], 456.00)
+ self.assertEqual(result[2]["decimal_val"], 789.00)
+
+ with self.sql_conf(
+ {"spark.sql.execution.pythonUDF.pandas.intToDecimalCoercionEnabled": False}
+ ):
+ self.assertRaisesRegex(
+ PythonException,
+ "Exception thrown when converting pandas.Series",
+ df.withColumn("decimal_val", int_to_decimal_udf("id")).collect,
+ )
+
+ @pandas_udf(returnType="decimal(25,1)")
+ def high_precision_udf(column):
+ values = [1, 2, 3]
+ return pd.Series([values[int(val) % len(values)] for val in column])
+
+ for intToDecimalCoercionEnabled in [True, False]:
+ # arrow_cast is enabled by default for SQL_SCALAR_PANDAS_UDF and
+ # and SQL_SCALAR_PANDAS_ITER_UDF, arrow can do this cast safely.
+ # intToDecimalCoercionEnabled is not required for this case
+ with self.sql_conf(
+ {
+ "spark.sql.execution.pythonUDF.pandas.intToDecimalCoercionEnabled": intToDecimalCoercionEnabled # noqa: E501
+ }
+ ):
+ result = df.withColumn("decimal_val", high_precision_udf("id")).collect()
+ self.assertEqual(len(result), 3)
+ self.assertEqual(result[0]["decimal_val"], Decimal("1.0"))
+ self.assertEqual(result[1]["decimal_val"], Decimal("2.0"))
+ self.assertEqual(result[2]["decimal_val"], Decimal("3.0"))
+
def test_pandas_udf_timestamp_ntz(self):
# SPARK-36626: Test TimestampNTZ in pandas UDF
@pandas_udf(returnType="timestamp_ntz")
diff --git a/python/pyspark/sql/tests/pandas/test_pandas_udf_grouped_agg.py b/python/pyspark/sql/tests/pandas/test_pandas_udf_grouped_agg.py
index e22b8f9ccacc2..1059af59f4a84 100644
--- a/python/pyspark/sql/tests/pandas/test_pandas_udf_grouped_agg.py
+++ b/python/pyspark/sql/tests/pandas/test_pandas_udf_grouped_agg.py
@@ -718,6 +718,49 @@ def biased_sum(v, w=None):
aggregated, df.groupby("id").agg((sum(df.v) + sum(df.w)).alias("s"))
)
+ def test_arrow_cast_enabled_numeric_to_decimal(self):
+ import numpy as np
+ from decimal import Decimal
+
+ columns = [
+ "int8",
+ "int16",
+ "int32",
+ "uint8",
+ "uint16",
+ "uint32",
+ "float64",
+ ]
+
+ pdf = pd.DataFrame({key: np.arange(1, 2).astype(key) for key in columns})
+ df = self.spark.range(2).repartition(1)
+
+ for column in columns:
+ with self.subTest(column=column):
+
+ @pandas_udf("decimal(10,0)", PandasUDFType.GROUPED_AGG)
+ def test(series):
+ return pdf[column].iloc[0]
+
+ row = df.groupby("id").agg(test(df.id)).first()
+ res = row[1]
+ self.assertEqual(res, Decimal("1"))
+
+ def test_arrow_cast_enabled_str_to_numeric(self):
+ df = self.spark.range(2).repartition(1)
+
+ types = ["int", "long", "float", "double"]
+
+ for type_str in types:
+ with self.subTest(type=type_str):
+
+ @pandas_udf(type_str, PandasUDFType.GROUPED_AGG)
+ def test(series):
+ return 123
+
+ row = df.groupby("id").agg(test(df.id)).first()
+ self.assertEqual(row[1], 123)
+
class GroupedAggPandasUDFTests(GroupedAggPandasUDFTestsMixin, ReusedSQLTestCase):
pass
diff --git a/python/pyspark/sql/tests/pandas/test_pandas_udf_scalar.py b/python/pyspark/sql/tests/pandas/test_pandas_udf_scalar.py
index 0128ae4840703..e614d9039b61a 100644
--- a/python/pyspark/sql/tests/pandas/test_pandas_udf_scalar.py
+++ b/python/pyspark/sql/tests/pandas/test_pandas_udf_scalar.py
@@ -517,7 +517,10 @@ def test_vectorized_udf_struct_complex(self):
def _scalar_f(id):
return pd.DataFrame(
- {"ts": id.apply(lambda i: pd.Timestamp(i)), "arr": id.apply(lambda i: [i, i + 1])}
+ {
+ "ts": id.apply(lambda i: pd.Timestamp(i, unit="s")),
+ "arr": id.apply(lambda i: [i, i + 1]),
+ }
)
scalar_f = pandas_udf(_scalar_f, returnType=return_type)
@@ -532,7 +535,7 @@ def iter_f(it):
for i, row in enumerate(actual):
id, f = row
self.assertEqual(i, id)
- self.assertEqual(pd.Timestamp(i).to_pydatetime(), f[0])
+ self.assertEqual(pd.Timestamp(i, unit="s").to_pydatetime(), f[0])
self.assertListEqual([i, i + 1], f[1])
def test_vectorized_udf_struct_empty(self):
@@ -1872,6 +1875,36 @@ def test_udf(a, b=0):
with self.subTest(with_b=True, query_no=i):
assertDataFrameEqual(df, [Row(0), Row(101)])
+ def test_arrow_cast_enabled_numeric_to_decimal(self):
+ import numpy as np
+
+ columns = [
+ "int8",
+ "int16",
+ "int32",
+ "uint8",
+ "uint16",
+ "uint32",
+ "float64",
+ ]
+
+ pdf = pd.DataFrame({key: np.arange(1, 2).astype(key) for key in columns})
+ df = self.spark.range(2).repartition(1)
+
+ t = DecimalType(10, 0)
+ for column in columns:
+ with self.subTest(column=column):
+ v = pdf[column].iloc[:1]
+ row = df.select(pandas_udf(lambda _: v, t)(df.id)).first()
+ assert (row[0] == v).all()
+
+ def test_arrow_cast_enabled_str_to_numeric(self):
+ df = self.spark.range(2).repartition(1)
+ for t in [IntegerType(), LongType(), FloatType(), DoubleType()]:
+ with self.subTest(type=t):
+ row = df.select(pandas_udf(lambda _: pd.Series(["123"]), t)(df.id)).first()
+ assert row[0] == 123
+
class ScalarPandasUDFTests(ScalarPandasUDFTestsMixin, ReusedSQLTestCase):
@classmethod
diff --git a/python/pyspark/sql/tests/pandas/test_pandas_udf_typehints.py b/python/pyspark/sql/tests/pandas/test_pandas_udf_typehints.py
index 7a0fccc225725..bb00f8f5b69b7 100644
--- a/python/pyspark/sql/tests/pandas/test_pandas_udf_typehints.py
+++ b/python/pyspark/sql/tests/pandas/test_pandas_udf_typehints.py
@@ -377,6 +377,19 @@ def func(col: "Union[pd.Series, pd.DataFrame]", *, col2: "pd.DataFrame") -> "pd.
infer_eval_type(signature(func), get_type_hints(func)), PandasUDFType.SCALAR
)
+ @unittest.skipIf(not have_pyarrow, pyarrow_requirement_message)
+ def test_negative_with_arrow_udf(self):
+ import pyarrow as pa
+
+ with self.assertRaisesRegex(
+ Exception,
+ "Unsupported signature:.*pyarrow.lib.Array.",
+ ):
+
+ @pandas_udf("long")
+ def multiply_arrow(a: pa.Array, b: pa.Array) -> pa.Array:
+ return pa.compute.multiply(a, b)
+
if __name__ == "__main__":
from pyspark.sql.tests.pandas.test_pandas_udf_typehints import * # noqa: #401
diff --git a/python/pyspark/sql/tests/pandas/test_pandas_udf_window.py b/python/pyspark/sql/tests/pandas/test_pandas_udf_window.py
index 9b3673d80d224..2f534b811b345 100644
--- a/python/pyspark/sql/tests/pandas/test_pandas_udf_window.py
+++ b/python/pyspark/sql/tests/pandas/test_pandas_udf_window.py
@@ -17,6 +17,7 @@
import unittest
from typing import cast
+from decimal import Decimal
from pyspark.errors import AnalysisException, PythonException
from pyspark.sql.functions import (
@@ -33,6 +34,13 @@
PandasUDFType,
)
from pyspark.sql.window import Window
+from pyspark.sql.types import (
+ DecimalType,
+ IntegerType,
+ LongType,
+ FloatType,
+ DoubleType,
+)
from pyspark.testing.sqlutils import (
ReusedSQLTestCase,
have_pandas,
@@ -563,6 +571,43 @@ def weighted_mean(**kwargs):
)
).show()
+ def test_arrow_cast_numeric_to_decimal(self):
+ import numpy as np
+ import pandas as pd
+
+ columns = [
+ "int8",
+ "int16",
+ "int32",
+ "uint8",
+ "uint16",
+ "uint32",
+ "float64",
+ ]
+
+ pdf = pd.DataFrame({key: np.arange(1, 2).astype(key) for key in columns})
+ df = self.data
+ w = self.unbounded_window
+
+ t = DecimalType(10, 0)
+ for column in columns:
+ with self.subTest(column=column):
+ value = pdf[column].iloc[0]
+ mean_udf = pandas_udf(lambda v: value, t, PandasUDFType.GROUPED_AGG)
+ result = df.select(mean_udf(df["v"]).over(w)).first()[0]
+ assert result == Decimal("1.0")
+ assert type(result) == Decimal
+
+ def test_arrow_cast_str_to_numeric(self):
+ df = self.data
+ w = self.unbounded_window
+
+ for t in [IntegerType(), LongType(), FloatType(), DoubleType()]:
+ with self.subTest(type=t):
+ mean_udf = pandas_udf(lambda v: "123", t, PandasUDFType.GROUPED_AGG)
+ result = df.select(mean_udf(df["v"]).over(w)).first()[0]
+ assert result == 123
+
class WindowPandasUDFTests(WindowPandasUDFTestsMixin, ReusedSQLTestCase):
pass
diff --git a/python/pyspark/sql/tests/streaming/test_streaming_foreach_batch.py b/python/pyspark/sql/tests/streaming/test_streaming_foreach_batch.py
index 9db66aa252ee6..380f089d6a552 100644
--- a/python/pyspark/sql/tests/streaming/test_streaming_foreach_batch.py
+++ b/python/pyspark/sql/tests/streaming/test_streaming_foreach_batch.py
@@ -29,18 +29,18 @@ def test_streaming_foreach_batch(self):
q = None
def collectBatch(batch_df, batch_id):
- batch_df.write.format("parquet").saveAsTable("test_table")
+ batch_df.write.format("parquet").saveAsTable("test_table1")
try:
df = self.spark.readStream.format("text").load("python/test_support/sql/streaming")
q = df.writeStream.foreachBatch(collectBatch).start()
q.processAllAvailable()
- collected = self.spark.sql("select * from test_table").collect()
+ collected = self.spark.sql("select * from test_table1").collect()
self.assertTrue(len(collected), 2)
finally:
if q:
q.stop()
- self.spark.sql("DROP TABLE IF EXISTS test_table")
+ self.spark.sql("DROP TABLE IF EXISTS test_table1")
def test_streaming_foreach_batch_tempview(self):
q = None
@@ -52,18 +52,18 @@ def collectBatch(batch_df, batch_id):
# streaming query
assert len(batch_df.sparkSession.sql("SELECT * FROM updates").collect()) == 2
# Write a table to verify on the repl/client side.
- batch_df.write.format("parquet").saveAsTable("test_table")
+ batch_df.write.format("parquet").saveAsTable("test_table2")
try:
df = self.spark.readStream.format("text").load("python/test_support/sql/streaming")
q = df.writeStream.foreachBatch(collectBatch).start()
q.processAllAvailable()
- collected = self.spark.sql("SELECT * FROM test_table").collect()
+ collected = self.spark.sql("SELECT * FROM test_table2").collect()
self.assertTrue(len(collected[0]), 2)
finally:
if q:
q.stop()
- self.spark.sql("DROP TABLE IF EXISTS test_table")
+ self.spark.sql("DROP TABLE IF EXISTS test_table2")
def test_streaming_foreach_batch_propagates_python_errors(self):
from pyspark.errors import StreamingQueryException
@@ -204,6 +204,27 @@ def func(df: DataFrame, batch_id: int):
df = self.spark.read.format("text").load("python/test_support/sql/streaming")
self.assertEqual(sorted(df.collect()), sorted(actual.collect()))
+ def test_streaming_foreach_batch_external_column(self):
+ from pyspark.sql import functions as sf
+
+ table_name = "testTable_foreach_batch_external_column"
+ with self.table(table_name):
+ # Define 'col' outside the `func` below, so it'd have to be serialized.
+ col = sf.col("value")
+
+ def func(df: DataFrame, batch_id: int):
+ result_df = df.select(col.alias("result"))
+ result_df.write.mode("append").saveAsTable(table_name)
+
+ df = self.spark.readStream.format("text").load("python/test_support/sql/streaming")
+ q = df.writeStream.foreachBatch(func).start()
+ q.processAllAvailable()
+ q.stop()
+
+ collected = self.spark.sql("select * from " + table_name).collect()
+ results = [row["result"] for row in collected]
+ self.assertEqual(sorted(results), ["hello", "this"])
+
class StreamingTestsForeachBatch(StreamingTestsForeachBatchMixin, ReusedSQLTestCase):
pass
diff --git a/python/pyspark/sql/tests/streaming/test_streaming_listener.py b/python/pyspark/sql/tests/streaming/test_streaming_listener.py
index 934b1463a15cd..bb485dae6bdc2 100644
--- a/python/pyspark/sql/tests/streaming/test_streaming_listener.py
+++ b/python/pyspark/sql/tests/streaming/test_streaming_listener.py
@@ -400,9 +400,17 @@ def verify(test_listener):
# Check query terminated with exception
from pyspark.sql.functions import col, udf
+ start_event = None
+ progress_event = None
+ terminated_event = None
bad_udf = udf(lambda x: 1 / 0)
q = df.select(bad_udf(col("value"))).writeStream.format("noop").start()
- time.sleep(5)
+ wait_count = 0
+ while terminated_event is None:
+ time.sleep(0.5)
+ wait_count = wait_count + 1
+ if wait_count > 100:
+ self.fail("Not getting terminated event after 50 seconds")
q.stop()
self.spark.sparkContext._jsc.sc().listenerBus().waitUntilEmpty()
self.check_terminated_event(terminated_event, "ZeroDivisionError")
diff --git a/python/pyspark/sql/tests/test_collection.py b/python/pyspark/sql/tests/test_collection.py
index f59f4dc7b465a..5c923e0e7558e 100644
--- a/python/pyspark/sql/tests/test_collection.py
+++ b/python/pyspark/sql/tests/test_collection.py
@@ -15,9 +15,11 @@
# limitations under the License.
#
+import datetime
import unittest
from pyspark.sql.types import (
+ Row,
ArrayType,
StringType,
IntegerType,
@@ -363,6 +365,49 @@ def check_to_local_iterator_not_fully_consumed(self):
break
self.assertEqual(df.take(8), result)
+ def test_collect_time(self):
+ import pandas as pd
+
+ query = """
+ SELECT * FROM VALUES
+ (TIME '12:34:56', 'a'), (TIME '22:56:01', 'b'), (NULL, 'c')
+ AS tab(t, i)
+ """
+
+ df = self.spark.sql(query)
+
+ rows = df.collect()
+ self.assertEqual(
+ rows,
+ [
+ Row(t=datetime.time(12, 34, 56), i="a"),
+ Row(t=datetime.time(22, 56, 1), i="b"),
+ Row(t=None, i="c"),
+ ],
+ )
+
+ pdf = df.toPandas()
+ self.assertTrue(
+ pdf.equals(
+ pd.DataFrame(
+ {
+ "t": [datetime.time(12, 34, 56), datetime.time(22, 56, 1), None],
+ "i": ["a", "b", "c"],
+ }
+ )
+ )
+ )
+
+ tbl = df.toArrow()
+ self.assertEqual(
+ [t.as_py() for t in tbl.column("t")],
+ [datetime.time(12, 34, 56), datetime.time(22, 56, 1), None],
+ )
+ self.assertEqual(
+ [i.as_py() for i in tbl.column("i")],
+ ["a", "b", "c"],
+ )
+
class DataFrameCollectionTests(
DataFrameCollectionTestsMixin,
diff --git a/python/pyspark/sql/tests/test_column.py b/python/pyspark/sql/tests/test_column.py
index 5f1991973d27d..e2ebfd3c092c4 100644
--- a/python/pyspark/sql/tests/test_column.py
+++ b/python/pyspark/sql/tests/test_column.py
@@ -133,8 +133,8 @@ def test_access_column(self):
self.assertTrue(isinstance(df["key"], Column))
self.assertTrue(isinstance(df[0], Column))
self.assertRaises(IndexError, lambda: df[2])
- self.assertRaises(AnalysisException, lambda: df["bad_key"])
self.assertRaises(TypeError, lambda: df[{}])
+ self.assertRaises(AnalysisException, lambda: df.select(df["bad_key"]).schema)
def test_column_name_with_non_ascii(self):
columnName = "数量"
@@ -361,6 +361,9 @@ def test_lit_time_representation(self):
ts = datetime.datetime(2021, 3, 4, 12, 34, 56, 1234)
self.assertEqual(str(sf.lit(ts)), "Column<'2021-03-04 12:34:56.001234'>")
+ ts = datetime.time(12, 34, 56, 1234)
+ self.assertEqual(str(sf.lit(ts)), "Column<'12:34:56.001234'>")
+
@unittest.skipIf(not have_pandas, pandas_requirement_message)
def test_lit_delta_representation(self):
for delta in [
diff --git a/python/pyspark/sql/tests/test_conversion.py b/python/pyspark/sql/tests/test_conversion.py
new file mode 100644
index 0000000000000..2b18fe8d04d7a
--- /dev/null
+++ b/python/pyspark/sql/tests/test_conversion.py
@@ -0,0 +1,122 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+import unittest
+
+from pyspark.sql.conversion import ArrowTableToRowsConversion, LocalDataToArrowConversion
+from pyspark.sql.types import (
+ ArrayType,
+ BinaryType,
+ IntegerType,
+ MapType,
+ Row,
+ StringType,
+ StructType,
+)
+from pyspark.testing.objects import ExamplePoint, ExamplePointUDT
+from pyspark.testing.utils import have_pyarrow, pyarrow_requirement_message
+
+
+@unittest.skipIf(not have_pyarrow, pyarrow_requirement_message)
+class ConversionTests(unittest.TestCase):
+ def test_conversion(self):
+ data = [
+ (
+ i if i % 2 == 0 else None,
+ str(i),
+ i,
+ str(i).encode(),
+ [j if j % 2 == 0 else None for j in range(i)],
+ list(range(i)),
+ [str(j).encode() for j in range(i)],
+ {str(j): j if j % 2 == 0 else None for j in range(i)},
+ {str(j): j for j in range(i)},
+ {str(j): str(j).encode() for j in range(i)},
+ (i if i % 2 == 0 else None, str(i), i, str(i).encode()),
+ {"i": i if i % 2 == 0 else None, "s": str(i), "ii": i, "b": str(i).encode()},
+ ExamplePoint(float(i), float(i)),
+ )
+ for i in range(5)
+ ]
+ schema = (
+ StructType()
+ .add("i", IntegerType())
+ .add("s", StringType())
+ .add("ii", IntegerType(), nullable=False)
+ .add("b", BinaryType())
+ .add("arr_i", ArrayType(IntegerType()))
+ .add("arr_ii", ArrayType(IntegerType(), containsNull=False))
+ .add("arr_b", ArrayType(BinaryType()))
+ .add("map_i", MapType(StringType(), IntegerType()))
+ .add("map_ii", MapType(StringType(), IntegerType(), valueContainsNull=False))
+ .add("map_b", MapType(StringType(), BinaryType()))
+ .add(
+ "struct_t",
+ StructType()
+ .add("i", IntegerType())
+ .add("s", StringType())
+ .add("ii", IntegerType(), nullable=False)
+ .add("b", BinaryType()),
+ )
+ .add(
+ "struct_d",
+ StructType()
+ .add("i", IntegerType())
+ .add("s", StringType())
+ .add("ii", IntegerType(), nullable=False)
+ .add("b", BinaryType()),
+ )
+ .add("udt", ExamplePointUDT())
+ )
+
+ tbl = LocalDataToArrowConversion.convert(data, schema, use_large_var_types=False)
+ actual = ArrowTableToRowsConversion.convert(tbl, schema)
+
+ for a, e in zip(
+ actual,
+ [
+ Row(
+ i=i if i % 2 == 0 else None,
+ s=str(i),
+ ii=i,
+ b=str(i).encode(),
+ arr_i=[j if j % 2 == 0 else None for j in range(i)],
+ arr_ii=list(range(i)),
+ arr_b=[str(j).encode() for j in range(i)],
+ map_i={str(j): j if j % 2 == 0 else None for j in range(i)},
+ map_ii={str(j): j for j in range(i)},
+ map_b={str(j): str(j).encode() for j in range(i)},
+ struct_t=Row(i=i if i % 2 == 0 else None, s=str(i), ii=i, b=str(i).encode()),
+ struct_d=Row(i=i if i % 2 == 0 else None, s=str(i), ii=i, b=str(i).encode()),
+ udt=ExamplePoint(float(i), float(i)),
+ )
+ for i in range(5)
+ ],
+ ):
+ with self.subTest(expected=e):
+ self.assertEqual(a, e)
+
+
+if __name__ == "__main__":
+ from pyspark.sql.tests.test_conversion import * # noqa: F401
+
+ try:
+ import xmlrunner # type: ignore
+
+ testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2)
+ except ImportError:
+ testRunner = None
+ unittest.main(testRunner=testRunner, verbosity=2)
diff --git a/python/pyspark/sql/tests/test_creation.py b/python/pyspark/sql/tests/test_creation.py
index c6917aa234b41..fb78b4c146326 100644
--- a/python/pyspark/sql/tests/test_creation.py
+++ b/python/pyspark/sql/tests/test_creation.py
@@ -24,7 +24,12 @@
from pyspark.sql import Row
import pyspark.sql.functions as F
from pyspark.sql.types import (
+ DecimalType,
+ StructType,
+ StructField,
+ StringType,
DateType,
+ TimeType,
TimestampType,
TimestampNTZType,
)
@@ -32,6 +37,7 @@
PySparkTypeError,
PySparkValueError,
)
+from pyspark.testing import assertDataFrameEqual
from pyspark.testing.sqlutils import (
ReusedSQLTestCase,
have_pandas,
@@ -42,6 +48,21 @@
class DataFrameCreationTestsMixin:
+ def test_create_str_from_dict(self):
+ data = [
+ {"broker": {"teamId": 3398, "contactEmail": "abc.xyz@123.ca"}},
+ ]
+
+ for schema in [
+ StructType([StructField("broker", StringType())]),
+ "broker: string",
+ ]:
+ df = self.spark.createDataFrame(data, schema=schema)
+ self.assertEqual(
+ df.first().broker,
+ """{'teamId': 3398, 'contactEmail': 'abc.xyz@123.ca'}""",
+ )
+
def test_create_dataframe_from_array_of_long(self):
import array
@@ -49,6 +70,20 @@ def test_create_dataframe_from_array_of_long(self):
df = self.spark.createDataFrame(data)
self.assertEqual(df.first(), Row(longarray=[-9223372036854775808, 0, 9223372036854775807]))
+ def test_create_dataframe_from_datetime_time(self):
+ import datetime
+
+ df = self.spark.createDataFrame(
+ [
+ (datetime.time(1, 2, 3),),
+ (datetime.time(4, 5, 6),),
+ (datetime.time(7, 8, 9),),
+ ],
+ ["t"],
+ )
+ self.assertIsInstance(df.schema["t"].dataType, TimeType)
+ self.assertEqual(df.count(), 3)
+
@unittest.skipIf(not have_pandas, pandas_requirement_message) # type: ignore
def test_create_dataframe_from_pandas_with_timestamp(self):
import pandas as pd
@@ -126,6 +161,12 @@ def test_create_nan_decimal_dataframe(self):
[Row(value=None)],
)
+ def test_check_decimal_nan(self):
+ data = [Row(dec=Decimal("NaN"))]
+ schema = StructType([StructField("dec", DecimalType(), False)])
+ with self.assertRaises(PySparkValueError):
+ self.spark.createDataFrame(data=data, schema=schema)
+
def test_invalid_argument_create_dataframe(self):
with self.assertRaises(PySparkTypeError) as pe:
self.spark.createDataFrame([(1, 2)], schema=123)
@@ -210,6 +251,13 @@ def test_schema_inference_from_pandas_with_dict(self):
[Row(str_col="second", dict_col={"first": 0.7, "second": 0.3}, test=0.3)],
)
+ def test_empty_schema(self):
+ schema = StructType()
+ for data in [[], [Row()]]:
+ with self.subTest(data=data):
+ sdf = self.spark.createDataFrame(data, schema)
+ assertDataFrameEqual(sdf, data)
+
class DataFrameCreationTests(
DataFrameCreationTestsMixin,
diff --git a/python/pyspark/sql/tests/test_dataframe.py b/python/pyspark/sql/tests/test_dataframe.py
index 890ae56ffa529..75a553b62838e 100644
--- a/python/pyspark/sql/tests/test_dataframe.py
+++ b/python/pyspark/sql/tests/test_dataframe.py
@@ -26,7 +26,18 @@
from contextlib import redirect_stdout
from pyspark.sql import Row, functions, DataFrame
-from pyspark.sql.functions import col, lit, count, struct, date_format, to_date, array, explode
+from pyspark.sql.functions import (
+ col,
+ lit,
+ count,
+ struct,
+ date_format,
+ to_date,
+ array,
+ explode,
+ when,
+ concat,
+)
from pyspark.sql.types import (
StringType,
IntegerType,
@@ -189,6 +200,54 @@ def test_drop(self):
self.assertEqual(df.drop(col("name"), col("age")).columns, ["active"])
self.assertEqual(df.drop(col("name"), col("age"), col("random")).columns, ["active"])
+ def test_drop_notexistent_col(self):
+ df1 = self.spark.createDataFrame(
+ [("a", "b", "c")],
+ schema="colA string, colB string, colC string",
+ )
+ df2 = self.spark.createDataFrame(
+ [("c", "d", "e")],
+ schema="colC string, colD string, colE string",
+ )
+ df3 = df1.join(df2, df1["colC"] == df2["colC"]).withColumn(
+ "colB",
+ when(df1["colB"] == "b", concat(df1["colB"].cast("string"), lit("x"))).otherwise(
+ df1["colB"]
+ ),
+ )
+ df4 = df3.drop(df1["colB"])
+
+ self.assertEqual(df4.columns, ["colA", "colB", "colC", "colC", "colD", "colE"])
+ self.assertEqual(df4.count(), 1)
+
+ def test_drop_col_from_different_dataframe(self):
+ df1 = self.spark.range(10)
+ df2 = df1.withColumn("v0", lit(0))
+
+ # drop df2["id"] from df2
+ self.assertEqual(df2.drop(df2["id"]).columns, ["v0"])
+
+ # drop df1["id"] from df2, which is semantically equal to df2["id"]
+ # note that df1.drop(df2["id"]) works in Classic, but not in Connect
+ self.assertEqual(df2.drop(df1["id"]).columns, ["v0"])
+
+ df3 = df2.select("*", lit(1).alias("v1"))
+
+ # drop df3["id"] from df3
+ self.assertEqual(df3.drop(df3["id"]).columns, ["v0", "v1"])
+
+ # drop df2["id"] from df3, which is semantically equal to df3["id"]
+ self.assertEqual(df3.drop(df2["id"]).columns, ["v0", "v1"])
+
+ # drop df1["id"] from df3, which is semantically equal to df3["id"]
+ self.assertEqual(df3.drop(df1["id"]).columns, ["v0", "v1"])
+
+ # drop df3["v0"] from df3
+ self.assertEqual(df3.drop(df3["v0"]).columns, ["id", "v1"])
+
+ # drop df2["v0"] from df3, which is semantically equal to df3["v0"]
+ self.assertEqual(df3.drop(df2["v0"]).columns, ["id", "v1"])
+
def test_drop_join(self):
left_df = self.spark.createDataFrame(
[(1, "a"), (2, "b"), (3, "c")],
diff --git a/python/pyspark/sql/tests/test_functions.py b/python/pyspark/sql/tests/test_functions.py
index e2b3e33756ba3..2fde3edc2486c 100644
--- a/python/pyspark/sql/tests/test_functions.py
+++ b/python/pyspark/sql/tests/test_functions.py
@@ -81,7 +81,10 @@ def test_function_parity(self):
missing_in_py = jvm_fn_set.difference(py_fn_set)
# Functions that we expect to be missing in python until they are added to pyspark
- expected_missing_in_py = set()
+ expected_missing_in_py = set(
+ # TODO(SPARK-53108): Implement the time_diff function in Python
+ ["time_diff"]
+ )
self.assertEqual(
expected_missing_in_py, missing_in_py, "Missing functions in pyspark not as expected"
@@ -400,6 +403,19 @@ def test_rand_functions(self):
rndn2 = df.select("key", F.randn(0)).collect()
self.assertEqual(sorted(rndn1), sorted(rndn2))
+ def test_time_trunc(self):
+ # SPARK-53110: test the time_trunc function.
+ df = self.spark.range(1).select(
+ F.lit("minute").alias("unit"), F.lit(datetime.time(1, 2, 3)).alias("time")
+ )
+ result = datetime.time(1, 2, 0)
+ row_from_col = df.select(F.time_trunc(df.unit, df.time)).first()
+ self.assertIsInstance(row_from_col[0], datetime.time)
+ self.assertEqual(row_from_col[0], result)
+ row_from_name = df.select(F.time_trunc("unit", "time")).first()
+ self.assertIsInstance(row_from_name[0], datetime.time)
+ self.assertEqual(row_from_name[0], result)
+
def test_try_parse_url(self):
df = self.spark.createDataFrame(
[("https://spark.apache.org/path?query=1", "QUERY", "query")],
@@ -574,6 +590,30 @@ def test_dayname(self):
row = df.select(F.dayname(df.date)).first()
self.assertEqual(row[0], "Mon")
+ def test_hour(self):
+ # SPARK-52892: test the hour function with time.
+ df = self.spark.range(1).select(F.lit(datetime.time(12, 34, 56)).alias("time"))
+ row_from_col = df.select(F.hour(df.time)).first()
+ self.assertEqual(row_from_col[0], 12)
+ row_from_name = df.select(F.hour("time")).first()
+ self.assertEqual(row_from_name[0], 12)
+
+ def test_minute(self):
+ # SPARK-52893: test the minute function with time.
+ df = self.spark.range(1).select(F.lit(datetime.time(12, 34, 56)).alias("time"))
+ row_from_col = df.select(F.minute(df.time)).first()
+ self.assertEqual(row_from_col[0], 34)
+ row_from_name = df.select(F.minute("time")).first()
+ self.assertEqual(row_from_name[0], 34)
+
+ def test_second(self):
+ # SPARK-52894: test the second function with time.
+ df = self.spark.range(1).select(F.lit(datetime.time(12, 34, 56)).alias("time"))
+ row_from_col = df.select(F.second(df.time)).first()
+ self.assertEqual(row_from_col[0], 56)
+ row_from_name = df.select(F.second("time")).first()
+ self.assertEqual(row_from_name[0], 56)
+
# Test added for SPARK-37738; change Python API to accept both col & int as input
def test_date_add_function(self):
dt = datetime.date(2021, 12, 27)
@@ -628,6 +668,17 @@ def test_add_months_function(self):
)
)
+ def test_make_time(self):
+ # SPARK-52888: test the make_time function.
+ df = self.spark.createDataFrame([(1, 2, 3)], ["hour", "minute", "second"])
+ result = datetime.time(1, 2, 3)
+ row_from_col = df.select(F.make_time(df.hour, df.minute, df.second)).first()
+ self.assertIsInstance(row_from_col[0], datetime.time)
+ self.assertEqual(row_from_col[0], result)
+ row_from_name = df.select(F.make_time("hour", "minute", "second")).first()
+ self.assertIsInstance(row_from_name[0], datetime.time)
+ self.assertEqual(row_from_name[0], result)
+
def test_make_date(self):
# SPARK-36554: expose make_date expression
df = self.spark.createDataFrame([(2020, 6, 26)], ["Y", "M", "D"])
@@ -1275,6 +1326,11 @@ def test_datetime_functions(self):
parse_result = df.select(F.to_date(F.col("dateCol"))).first()
self.assertEqual(datetime.date(2017, 1, 22), parse_result["to_date(dateCol)"])
+ def test_try_datetime_functions(self):
+ df = self.spark.range(1).selectExpr("'2017-01-22' as dateCol")
+ parse_result = df.select(F.try_to_date(F.col("dateCol")).alias("tryToDateCol")).first()
+ self.assertEqual(datetime.date(2017, 1, 22), parse_result["tryToDateCol"])
+
def test_assert_true(self):
self.check_assert_true(SparkRuntimeException)
@@ -1344,6 +1400,11 @@ def test_shiftrightunsigned(self):
)
).collect()
+ def test_lit_time(self):
+ t = datetime.time(12, 34, 56)
+ actual = self.spark.range(1).select(F.lit(t)).first()[0]
+ self.assertEqual(actual, t)
+
def test_lit_day_time_interval(self):
td = datetime.timedelta(days=1, hours=12, milliseconds=123)
actual = self.spark.range(1).select(F.lit(td)).first()[0]
@@ -1614,6 +1675,35 @@ def test_try_parse_json(self):
self.assertEqual("""{"a":1}""", actual[0]["var"])
self.assertEqual(None, actual[1]["var"])
+ def test_try_to_time(self):
+ # SPARK-52891: test the try_to_time function.
+ df = self.spark.createDataFrame([("10:30:00", "HH:mm:ss")], ["time", "format"])
+ result = datetime.time(10, 30, 0)
+ # Test without format.
+ row_from_col_no_format = df.select(F.try_to_time(df.time)).first()
+ self.assertIsInstance(row_from_col_no_format[0], datetime.time)
+ self.assertEqual(row_from_col_no_format[0], result)
+ row_from_name_no_format = df.select(F.try_to_time("time")).first()
+ self.assertIsInstance(row_from_name_no_format[0], datetime.time)
+ self.assertEqual(row_from_name_no_format[0], result)
+ # Test with format.
+ row_from_col_with_format = df.select(F.try_to_time(df.time, df.format)).first()
+ self.assertIsInstance(row_from_col_with_format[0], datetime.time)
+ self.assertEqual(row_from_col_with_format[0], result)
+ row_from_name_with_format = df.select(F.try_to_time("time", "format")).first()
+ self.assertIsInstance(row_from_name_with_format[0], datetime.time)
+ self.assertEqual(row_from_name_with_format[0], result)
+ # Test with malformed time.
+ df = self.spark.createDataFrame([("malformed", "HH:mm:ss")], ["time", "format"])
+ row_from_col_no_format_malformed = df.select(F.try_to_time(df.time)).first()
+ self.assertIsNone(row_from_col_no_format_malformed[0])
+ row_from_name_no_format_malformed = df.select(F.try_to_time("time")).first()
+ self.assertIsNone(row_from_name_no_format_malformed[0])
+ row_from_col_with_format_malformed = df.select(F.try_to_time(df.time, df.format)).first()
+ self.assertIsNone(row_from_col_with_format_malformed[0])
+ row_from_name_with_format_malformed = df.select(F.try_to_time("time", "format")).first()
+ self.assertIsNone(row_from_name_with_format_malformed[0])
+
def test_to_variant_object(self):
df = self.spark.createDataFrame([(1, {"a": 1})], "i int, v struct")
actual = df.select(
@@ -1723,6 +1813,25 @@ def test_bucket(self):
messageParameters={"arg_name": "numBuckets", "arg_type": "str"},
)
+ def test_to_time(self):
+ # SPARK-52890: test the to_time function.
+ df = self.spark.createDataFrame([("10:30:00", "HH:mm:ss")], ["time", "format"])
+ result = datetime.time(10, 30, 0)
+ # Test without format.
+ row_from_col_no_format = df.select(F.to_time(df.time)).first()
+ self.assertIsInstance(row_from_col_no_format[0], datetime.time)
+ self.assertEqual(row_from_col_no_format[0], result)
+ row_from_name_no_format = df.select(F.to_time("time")).first()
+ self.assertIsInstance(row_from_name_no_format[0], datetime.time)
+ self.assertEqual(row_from_name_no_format[0], result)
+ # Test with format.
+ row_from_col_with_format = df.select(F.to_time(df.time, df.format)).first()
+ self.assertIsInstance(row_from_col_with_format[0], datetime.time)
+ self.assertEqual(row_from_col_with_format[0], result)
+ row_from_name_with_format = df.select(F.to_time("time", "format")).first()
+ self.assertIsInstance(row_from_name_with_format[0], datetime.time)
+ self.assertEqual(row_from_name_with_format[0], result)
+
def test_to_timestamp_ltz(self):
df = self.spark.createDataFrame([("2016-12-31",)], ["e"])
df = df.select(F.to_timestamp_ltz(df.e, F.lit("yyyy-MM-dd")).alias("r"))
@@ -1765,6 +1874,16 @@ def test_non_deterministic_with_seed(self):
for i in range(3):
self.assertEqual(res[0][i * 2], res[0][i * 2 + 1])
+ def test_current_time(self):
+ # SPARK-52889: test the current_time function without precision.
+ df = self.spark.range(1).select(F.current_time())
+ self.assertIsInstance(df.first()[0], datetime.time)
+ self.assertEqual(df.schema.names[0], "current_time(6)")
+ # SPARK-52889: test the current_time function with precision.
+ df = self.spark.range(1).select(F.current_time(3))
+ self.assertIsInstance(df.first()[0], datetime.time)
+ self.assertEqual(df.schema.names[0], "current_time(3)")
+
def test_current_timestamp(self):
df = self.spark.range(1).select(F.current_timestamp())
self.assertIsInstance(df.first()[0], datetime.datetime)
diff --git a/python/pyspark/sql/tests/test_python_streaming_datasource.py b/python/pyspark/sql/tests/test_python_streaming_datasource.py
index fa14b37b57e62..9879231540f1d 100644
--- a/python/pyspark/sql/tests/test_python_streaming_datasource.py
+++ b/python/pyspark/sql/tests/test_python_streaming_datasource.py
@@ -24,6 +24,7 @@
DataSourceStreamReader,
InputPartition,
DataSourceStreamWriter,
+ DataSourceStreamArrowWriter,
SimpleDataSourceStreamReader,
WriterCommitMessage,
)
@@ -295,6 +296,156 @@ def test_stream_writer(self):
output_dir.cleanup()
checkpoint_dir.cleanup()
+ def test_stream_arrow_writer(self):
+ """Test DataSourceStreamArrowWriter with Arrow RecordBatch format."""
+ import tempfile
+ import shutil
+ import json
+ import os
+ import pyarrow as pa
+ from dataclasses import dataclass
+
+ @dataclass
+ class ArrowCommitMessage(WriterCommitMessage):
+ partition_id: int
+ batch_count: int
+ total_rows: int
+
+ class TestStreamArrowWriter(DataSourceStreamArrowWriter):
+ def __init__(self, options):
+ self.options = options
+ self.path = self.options.get("path")
+ assert self.path is not None
+
+ def write(self, iterator):
+ from pyspark import TaskContext
+
+ context = TaskContext.get()
+ partition_id = context.partitionId()
+ batch_count = 0
+ total_rows = 0
+
+ for batch in iterator:
+ assert isinstance(batch, pa.RecordBatch)
+ batch_count += 1
+ total_rows += batch.num_rows
+
+ # Convert to pandas and write to temp JSON file
+ df = batch.to_pandas()
+
+ filename = f"partition_{partition_id}_batch_{batch_count}.json"
+ filepath = os.path.join(self.path, filename)
+
+ # Actually write the JSON file
+ df.to_json(filepath, orient="records")
+
+ commit_msg = ArrowCommitMessage(
+ partition_id=partition_id, batch_count=batch_count, total_rows=total_rows
+ )
+ return commit_msg
+
+ def commit(self, messages, batchId):
+ """Write commit metadata for successful batch."""
+ total_batches = sum(m.batch_count for m in messages if m)
+ total_rows = sum(m.total_rows for m in messages if m)
+
+ status = {
+ "batch_id": batchId,
+ "num_partitions": len([m for m in messages if m]),
+ "total_batches": total_batches,
+ "total_rows": total_rows,
+ }
+
+ with open(os.path.join(self.path, f"commit_{batchId}.json"), "w") as f:
+ json.dump(status, f)
+
+ def abort(self, messages, batchId):
+ """Handle batch failure."""
+ with open(os.path.join(self.path, f"abort_{batchId}.txt"), "w") as f:
+ f.write(f"Batch {batchId} aborted")
+
+ class TestDataSource(DataSource):
+ @classmethod
+ def name(cls):
+ return "TestArrowStreamWriter"
+
+ def schema(self):
+ return "id INT, name STRING, value DOUBLE"
+
+ def streamWriter(self, schema, overwrite):
+ return TestStreamArrowWriter(self.options)
+
+ # Create temporary directory for test
+ temp_dir = tempfile.mkdtemp()
+ try:
+ # Register the data source
+ self.spark.dataSource.register(TestDataSource)
+
+ # Create test data
+ df = (
+ self.spark.readStream.format("rate")
+ .option("rowsPerSecond", 10)
+ .option("numPartitions", 3)
+ .load()
+ .selectExpr("value as id", "concat('name_', value) as name", "value * 2.5 as value")
+ )
+
+ # Write using streaming with Arrow writer
+ query = (
+ df.writeStream.format("TestArrowStreamWriter")
+ .option("path", temp_dir)
+ .option("checkpointLocation", os.path.join(temp_dir, "checkpoint"))
+ .trigger(processingTime="1 seconds")
+ .start()
+ )
+
+ # Wait a bit for data to be processed, then stop
+ time.sleep(6) # Allow a few batches to run
+ query.stop()
+ query.awaitTermination()
+
+ # Since we're writing actual JSON files, verify commit metadata and written files
+ commit_files = [f for f in os.listdir(temp_dir) if f.startswith("commit_")]
+ self.assertTrue(len(commit_files) > 0, "No commit files were created")
+
+ # Read and verify commit metadata - check all commits for any with data
+ total_committed_rows = 0
+ total_committed_batches = 0
+
+ for commit_file in commit_files:
+ with open(os.path.join(temp_dir, commit_file), "r") as f:
+ commit_data = json.load(f)
+ total_committed_rows += commit_data.get("total_rows", 0)
+ total_committed_batches += commit_data.get("total_batches", 0)
+
+ # We should have both committed data AND JSON files written
+ json_files = [
+ f
+ for f in os.listdir(temp_dir)
+ if f.startswith("partition_") and f.endswith(".json")
+ ]
+
+ # Verify that we have both committed data AND JSON files
+ has_committed_data = total_committed_rows > 0
+ has_json_files = len(json_files) > 0
+
+ self.assertTrue(
+ has_committed_data, f"Expected committed data but got {total_committed_rows} rows"
+ )
+ self.assertTrue(
+ has_json_files, f"Expected JSON files but found {len(json_files)} files"
+ )
+
+ # Verify JSON files contain valid data
+ for json_file in json_files:
+ with open(os.path.join(temp_dir, json_file), "r") as f:
+ data = json.load(f)
+ self.assertTrue(len(data) > 0, f"JSON file {json_file} is empty")
+
+ finally:
+ # Clean up
+ shutil.rmtree(temp_dir, ignore_errors=True)
+
class PythonStreamingDataSourceTests(BasePythonStreamingDataSourceTestsMixin, ReusedSQLTestCase):
pass
diff --git a/python/pyspark/sql/tests/test_sql.py b/python/pyspark/sql/tests/test_sql.py
index bf50bbc11ac33..e60ad183d1474 100644
--- a/python/pyspark/sql/tests/test_sql.py
+++ b/python/pyspark/sql/tests/test_sql.py
@@ -168,6 +168,12 @@ def test_nested_dataframe(self):
self.assertEqual(df3.take(1), [Row(id=4)])
self.assertEqual(df3.tail(1), [Row(id=9)])
+ def test_lit_time(self):
+ import datetime
+
+ actual = self.spark.sql("select TIME '12:34:56'").first()[0]
+ self.assertEqual(actual, datetime.time(12, 34, 56))
+
class SQLTests(SQLTestsMixin, ReusedSQLTestCase):
pass
diff --git a/python/pyspark/sql/tests/test_stat.py b/python/pyspark/sql/tests/test_stat.py
index 804e2484536f8..fe1746dbd894d 100644
--- a/python/pyspark/sql/tests/test_stat.py
+++ b/python/pyspark/sql/tests/test_stat.py
@@ -435,12 +435,12 @@ def test_unpivot(self):
self.assertEqual(
actual.collect(),
[
- Row(var="int", value=10.0),
- Row(var="double", value=1.0),
- Row(var="int", value=20.0),
- Row(var="double", value=2.0),
- Row(var="int", value=30.0),
- Row(var="double", value=3.0),
+ Row(var="int", val=10.0),
+ Row(var="double", val=1.0),
+ Row(var="int", val=20.0),
+ Row(var="double", val=2.0),
+ Row(var="int", val=30.0),
+ Row(var="double", val=3.0),
],
)
@@ -455,12 +455,12 @@ def test_unpivot(self):
self.assertEqual(
actual.collect(),
[
- Row(id=1, var="int", value=10.0),
- Row(id=1, var="double", value=1.0),
- Row(id=2, var="int", value=20.0),
- Row(id=2, var="double", value=2.0),
- Row(id=3, var="int", value=30.0),
- Row(id=3, var="double", value=3.0),
+ Row(id=1, var="int", val=10.0),
+ Row(id=1, var="double", val=1.0),
+ Row(id=2, var="int", val=20.0),
+ Row(id=2, var="double", val=2.0),
+ Row(id=3, var="int", val=30.0),
+ Row(id=3, var="double", val=3.0),
],
)
@@ -475,12 +475,12 @@ def test_unpivot(self):
self.assertEqual(
actual.collect(),
[
- Row(id=1, double=1.0, var="int", value=10.0),
- Row(id=1, double=1.0, var="double", value=1.0),
- Row(id=2, double=2.0, var="int", value=20.0),
- Row(id=2, double=2.0, var="double", value=2.0),
- Row(id=3, double=3.0, var="int", value=30.0),
- Row(id=3, double=3.0, var="double", value=3.0),
+ Row(id=1, double=1.0, var="int", val=10.0),
+ Row(id=1, double=1.0, var="double", val=1.0),
+ Row(id=2, double=2.0, var="int", val=20.0),
+ Row(id=2, double=2.0, var="double", val=2.0),
+ Row(id=3, double=3.0, var="int", val=30.0),
+ Row(id=3, double=3.0, var="double", val=3.0),
],
)
@@ -491,15 +491,15 @@ def test_unpivot(self):
self.assertEqual(
actual.collect(),
[
- Row(var="id", value=1.0),
- Row(var="int", value=10.0),
- Row(var="double", value=1.0),
- Row(var="id", value=2.0),
- Row(var="int", value=20.0),
- Row(var="double", value=2.0),
- Row(var="id", value=3.0),
- Row(var="int", value=30.0),
- Row(var="double", value=3.0),
+ Row(var="id", val=1.0),
+ Row(var="int", val=10.0),
+ Row(var="double", val=1.0),
+ Row(var="id", val=2.0),
+ Row(var="int", val=20.0),
+ Row(var="double", val=2.0),
+ Row(var="id", val=3.0),
+ Row(var="int", val=30.0),
+ Row(var="double", val=3.0),
],
)
@@ -514,12 +514,12 @@ def test_unpivot(self):
self.assertEqual(
actual.collect(),
[
- Row(id=1, var="int", value=10.0),
- Row(id=1, var="double", value=1.0),
- Row(id=2, var="int", value=20.0),
- Row(id=2, var="double", value=2.0),
- Row(id=3, var="int", value=30.0),
- Row(id=3, var="double", value=3.0),
+ Row(id=1, var="int", val=10.0),
+ Row(id=1, var="double", val=1.0),
+ Row(id=2, var="int", val=20.0),
+ Row(id=2, var="double", val=2.0),
+ Row(id=3, var="int", val=30.0),
+ Row(id=3, var="double", val=3.0),
],
)
diff --git a/python/pyspark/sql/tests/test_types.py b/python/pyspark/sql/tests/test_types.py
index 185198766b794..319ff92dd362d 100644
--- a/python/pyspark/sql/tests/test_types.py
+++ b/python/pyspark/sql/tests/test_types.py
@@ -42,6 +42,7 @@
IntegerType,
FloatType,
DateType,
+ TimeType,
TimestampType,
TimestampNTZType,
DayTimeIntervalType,
@@ -183,6 +184,7 @@ def __init__(self):
"a",
datetime.date(1970, 1, 1),
datetime.datetime(1970, 1, 1, 0, 0),
+ datetime.time(hour=1, minute=2, second=3),
datetime.timedelta(microseconds=123456678),
1.0,
array.array("d", [1]),
@@ -206,6 +208,7 @@ def __init__(self):
"string",
"date",
"timestamp",
+ "time(6)",
"interval day to second",
"double",
"array",
@@ -229,6 +232,7 @@ def __init__(self):
"a",
datetime.date(1970, 1, 1),
datetime.datetime(1970, 1, 1, 0, 0),
+ datetime.time(hour=1, minute=2, second=3),
datetime.timedelta(microseconds=123456678),
1.0,
[1.0],
@@ -525,7 +529,7 @@ def test_create_dataframe_from_objects(self):
self.assertEqual(df.first(), Row(key=1, value="1"))
def test_apply_schema(self):
- from datetime import date, datetime, timedelta
+ from datetime import date, time, datetime, timedelta
rdd = self.sc.parallelize(
[
@@ -537,6 +541,7 @@ def test_apply_schema(self):
2147483647,
1.0,
date(2010, 1, 1),
+ time(23, 23, 59, 999999),
datetime(2010, 1, 1, 1, 1, 1),
timedelta(days=1),
{"a": 1},
@@ -555,6 +560,7 @@ def test_apply_schema(self):
StructField("int1", IntegerType(), False),
StructField("float1", FloatType(), False),
StructField("date1", DateType(), False),
+ StructField("time", TimeType(), False),
StructField("time1", TimestampType(), False),
StructField("daytime1", DayTimeIntervalType(), False),
StructField("map1", MapType(StringType(), IntegerType(), False), False),
@@ -573,6 +579,7 @@ def test_apply_schema(self):
x.int1,
x.float1,
x.date1,
+ x.time,
x.time1,
x.daytime1,
x.map1["a"],
@@ -589,6 +596,7 @@ def test_apply_schema(self):
2147483647,
1.0,
date(2010, 1, 1),
+ time(23, 23, 59, 999999),
datetime(2010, 1, 1, 1, 1, 1),
timedelta(days=1),
1,
@@ -642,6 +650,17 @@ def test_schema_with_collations_json_ser_de(self):
from pyspark.sql.types import _parse_datatype_json_string
unicode_collation = "UNICODE"
+ utf8_lcase_collation = "UTF8_LCASE"
+
+ standalone_string = StringType(unicode_collation)
+
+ standalone_array = ArrayType(StringType(unicode_collation))
+
+ standalone_map = MapType(StringType(utf8_lcase_collation), StringType(unicode_collation))
+
+ standalone_nested = ArrayType(
+ MapType(StringType(utf8_lcase_collation), ArrayType(StringType(unicode_collation)))
+ )
simple_struct = StructType([StructField("c1", StringType(unicode_collation))])
@@ -713,6 +732,10 @@ def test_schema_with_collations_json_ser_de(self):
)
schemas = [
+ standalone_string,
+ standalone_array,
+ standalone_map,
+ standalone_nested,
simple_struct,
nested_struct,
array_in_schema,
@@ -1241,6 +1264,7 @@ def test_parse_datatype_json_string(self):
IntegerType(),
LongType(),
DateType(),
+ TimeType(5),
TimestampType(),
TimestampNTZType(),
NullType(),
@@ -1291,6 +1315,8 @@ def test_parse_datatype_string(self):
_parse_datatype_string("a INT, c DOUBLE"),
)
self.assertEqual(VariantType(), _parse_datatype_string("variant"))
+ self.assertEqual(TimeType(5), _parse_datatype_string("time(5)"))
+ self.assertEqual(TimeType(), _parse_datatype_string("time( 6 )"))
def test_tree_string(self):
schema1 = DataType.fromDDL("c1 INT, c2 STRUCT>")
@@ -1543,6 +1569,7 @@ def test_tree_string_for_builtin_types(self):
.add("bin", BinaryType())
.add("bool", BooleanType())
.add("date", DateType())
+ .add("time", TimeType())
.add("ts", TimestampType())
.add("ts_ntz", TimestampNTZType())
.add("dec", DecimalType(10, 2))
@@ -1578,6 +1605,7 @@ def test_tree_string_for_builtin_types(self):
" |-- bin: binary (nullable = true)",
" |-- bool: boolean (nullable = true)",
" |-- date: date (nullable = true)",
+ " |-- time: time(6) (nullable = true)",
" |-- ts: timestamp (nullable = true)",
" |-- ts_ntz: timestamp_ntz (nullable = true)",
" |-- dec: decimal(10,2) (nullable = true)",
@@ -1925,6 +1953,7 @@ def test_repr(self):
BinaryType(),
BooleanType(),
DateType(),
+ TimeType(),
TimestampType(),
DecimalType(),
DoubleType(),
@@ -2332,8 +2361,10 @@ def test_to_ddl(self):
schema = StructType().add("a", ArrayType(DoubleType()), False).add("b", DateType())
self.assertEqual(schema.toDDL(), "a ARRAY NOT NULL,b DATE")
- schema = StructType().add("a", TimestampType()).add("b", TimestampNTZType())
- self.assertEqual(schema.toDDL(), "a TIMESTAMP,b TIMESTAMP_NTZ")
+ schema = (
+ StructType().add("a", TimestampType()).add("b", TimestampNTZType()).add("c", TimeType())
+ )
+ self.assertEqual(schema.toDDL(), "a TIMESTAMP,b TIMESTAMP_NTZ,c TIME(6)")
def test_from_ddl(self):
self.assertEqual(DataType.fromDDL("long"), LongType())
@@ -2349,6 +2380,10 @@ def test_from_ddl(self):
DataType.fromDDL("a int, v variant"),
StructType([StructField("a", IntegerType()), StructField("v", VariantType())]),
)
+ self.assertEqual(
+ DataType.fromDDL("a time(6)"),
+ StructType([StructField("a", TimeType(6))]),
+ )
# Ensures that changing the implementation of `DataType.fromDDL` in PR #47253 does not change
# `fromDDL`'s behavior.
@@ -2602,8 +2637,9 @@ def __init__(self, **kwargs):
(decimal.Decimal("1.0"), DecimalType()),
# Binary
(bytearray([1, 2]), BinaryType()),
- # Date/Timestamp
+ # Date/Time/Timestamp
(datetime.date(2000, 1, 2), DateType()),
+ (datetime.time(1, 0, 0), TimeType()),
(datetime.datetime(2000, 1, 2, 3, 4), DateType()),
(datetime.datetime(2000, 1, 2, 3, 4), TimestampType()),
# Array
@@ -2666,8 +2702,9 @@ def __init__(self, **kwargs):
("1.0", DecimalType(), TypeError),
# Binary
(1, BinaryType(), TypeError),
- # Date/Timestamp
+ # Date/Time/Timestamp
("2000-01-02", DateType(), TypeError),
+ ("23:59:59", TimeType(), TypeError),
(946811040, TimestampType(), TypeError),
# Array
(["1", None], ArrayType(StringType(), containsNull=False), ValueError),
diff --git a/python/pyspark/sql/tests/test_udf.py b/python/pyspark/sql/tests/test_udf.py
index bd4db5306cb75..9d130a7f525ac 100644
--- a/python/pyspark/sql/tests/test_udf.py
+++ b/python/pyspark/sql/tests/test_udf.py
@@ -1379,6 +1379,73 @@ def add1(x):
result = empty_df.select(add1("id"))
self.assertEqual(result.collect(), [])
+ def test_udf_with_collated_string_types(self):
+ @udf("string collate fr")
+ def my_udf(input_val):
+ return "%s - %s" % (type(input_val), input_val)
+
+ string_types = [
+ StringType(),
+ StringType("UTF8_BINARY"),
+ StringType("UTF8_LCASE"),
+ StringType("UNICODE"),
+ ]
+ data = [("hello",)]
+ expected = " - hello"
+
+ for string_type in string_types:
+ schema = StructType([StructField("input_col", string_type, True)])
+ df = self.spark.createDataFrame(data, schema=schema)
+ df_result = df.select(my_udf(df.input_col).alias("result"))
+ row = df_result.collect()[0][0]
+ self.assertEqual(row, expected)
+ result_type = df_result.schema["result"].dataType
+ self.assertEqual(result_type, StringType("fr"))
+
+ def test_udf_with_char_varchar_return_type(self):
+ (char_type, char_value) = ("char(10)", "a")
+ (varchar_type, varchar_value) = ("varchar(8)", "a")
+ (array_with_char_type, array_with_char_type_value) = ("array", ["a", "b"])
+ (array_with_varchar_type, array_with_varchar_value) = ("array", ["a", "b"])
+ (map_type, map_value) = (f"map<{char_type}, {varchar_type}>", {"a": "b"})
+ (struct_type, struct_value) = (
+ f"struct",
+ {"f1": "a", "f2": "b"},
+ )
+
+ pairs = [
+ (char_type, char_value),
+ (varchar_type, varchar_value),
+ (array_with_char_type, array_with_char_type_value),
+ (array_with_varchar_type, array_with_varchar_value),
+ (map_type, map_value),
+ (struct_type, struct_value),
+ (
+ f"struct",
+ f"{{'f1': {array_with_char_type_value}, 'f2': {array_with_varchar_value}, "
+ f"'f3': {map_value}}}",
+ ),
+ (
+ f"map<{array_with_char_type}, {array_with_varchar_type}>",
+ f"{{{array_with_char_type_value}: {array_with_varchar_value}}}",
+ ),
+ (f"array<{struct_type}>", [struct_value, struct_value]),
+ ]
+
+ for return_type, return_value in pairs:
+ with self.assertRaisesRegex(
+ Exception,
+ "(Please use a different output data type for your UDF or DataFrame|"
+ "Invalid return type with Arrow-optimized Python UDF)",
+ ):
+
+ @udf(return_type)
+ def my_udf():
+ return return_value
+
+ self.spark.range(1).select(my_udf().alias("result")).show()
+
class UDFTests(BaseUDFTestsMixin, ReusedSQLTestCase):
@classmethod
diff --git a/python/pyspark/sql/tests/test_udf_combinations.py b/python/pyspark/sql/tests/test_udf_combinations.py
new file mode 100644
index 0000000000000..8111bb79d3c75
--- /dev/null
+++ b/python/pyspark/sql/tests/test_udf_combinations.py
@@ -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.
+#
+
+from typing import Iterator
+import itertools
+import unittest
+
+from pyspark.sql.functions import udf, arrow_udf, pandas_udf
+from pyspark.testing.sqlutils import ReusedSQLTestCase
+from pyspark.testing.utils import (
+ have_pandas,
+ have_pyarrow,
+ pandas_requirement_message,
+ pyarrow_requirement_message,
+)
+
+
+@unittest.skipIf(
+ not have_pandas or not have_pyarrow,
+ pandas_requirement_message or pyarrow_requirement_message,
+)
+class UDFCombinationsTestsMixin:
+ @property
+ def python_udf_add1(self):
+ @udf("long")
+ def py_add1(v):
+ assert isinstance(v, int)
+ return v + 1
+
+ return py_add1
+
+ @property
+ def arrow_opt_python_udf_add1(self):
+ @udf("long")
+ def py_arrow_opt_add1(v, useArrow=True):
+ assert isinstance(v, int)
+ return v + 1
+
+ return py_arrow_opt_add1
+
+ @property
+ def pandas_udf_add1(self):
+ import pandas as pd
+
+ @pandas_udf("long")
+ def pandas_add1(s):
+ assert isinstance(s, pd.Series)
+ return s + 1
+
+ return pandas_add1
+
+ @property
+ def pandas_iter_udf_add1(self):
+ import pandas as pd
+
+ @pandas_udf("long")
+ def pandas_iter_add1(it: Iterator[pd.Series]) -> Iterator[pd.Series]:
+ for s in it:
+ assert isinstance(s, pd.Series)
+ yield s + 1
+
+ return pandas_iter_add1
+
+ @property
+ def arrow_udf_add1(self):
+ import pyarrow as pa
+
+ @arrow_udf("long")
+ def arrow_add1(a):
+ assert isinstance(a, pa.Array)
+ return pa.compute.add(a, 1)
+
+ return arrow_add1
+
+ @property
+ def arrow_iter_udf_add1(self):
+ import pyarrow as pa
+
+ @arrow_udf("long")
+ def arrow_iter_add1(it: Iterator[pa.Array]) -> Iterator[pa.Array]:
+ for a in it:
+ assert isinstance(a, pa.Array)
+ yield pa.compute.add(a, 1)
+
+ return arrow_iter_add1
+
+ def all_scalar_functions(self):
+ return [
+ self.python_udf_add1,
+ self.arrow_opt_python_udf_add1,
+ self.pandas_udf_add1,
+ self.pandas_iter_udf_add1,
+ self.arrow_udf_add1,
+ self.arrow_iter_udf_add1,
+ ]
+
+ def test_combination_2(self):
+ df = self.spark.range(10)
+
+ expected = df.selectExpr("id + 2 AS res").collect()
+
+ combs = itertools.combinations(self.all_scalar_functions(), 2)
+ for f1, f2 in combs:
+ with self.subTest(
+ udf1=f1.__name__,
+ udf2=f2.__name__,
+ ):
+ result = df.select(f1(f2("id")).alias("res"))
+ self.assertEqual(expected, result.collect())
+
+ def test_combination_3(self):
+ df = self.spark.range(10)
+
+ expected = df.selectExpr("id + 3 AS res").collect()
+
+ combs = itertools.combinations(self.all_scalar_functions(), 3)
+ for f1, f2, f3 in combs:
+ with self.subTest(
+ udf1=f1.__name__,
+ udf2=f2.__name__,
+ udf3=f3.__name__,
+ ):
+ result = df.select(f1(f2(f3("id"))).alias("res"))
+ self.assertEqual(expected, result.collect())
+
+ def test_combination_4(self):
+ df = self.spark.range(10)
+
+ expected = df.selectExpr("id + 4 AS res").collect()
+
+ combs = itertools.combinations(self.all_scalar_functions(), 4)
+ for f1, f2, f3, f4 in combs:
+ with self.subTest(
+ udf1=f1.__name__,
+ udf2=f2.__name__,
+ udf3=f3.__name__,
+ udf4=f4.__name__,
+ ):
+ result = df.select(f1(f2(f3(f4("id")))).alias("res"))
+ self.assertEqual(expected, result.collect())
+
+ def test_combination_5(self):
+ df = self.spark.range(10)
+
+ expected = df.selectExpr("id + 5 AS res").collect()
+
+ combs = itertools.combinations(self.all_scalar_functions(), 5)
+ for f1, f2, f3, f4, f5 in combs:
+ with self.subTest(
+ udf1=f1.__name__,
+ udf2=f2.__name__,
+ udf3=f3.__name__,
+ udf4=f4.__name__,
+ udf5=f5.__name__,
+ ):
+ result = df.select(f1(f2(f3(f4(f5("id"))))).alias("res"))
+ self.assertEqual(expected, result.collect())
+
+ def test_combination_6(self):
+ df = self.spark.range(10)
+
+ expected = df.selectExpr("id + 6 AS res").collect()
+
+ combs = itertools.combinations(self.all_scalar_functions(), 6)
+ for f1, f2, f3, f4, f5, f6 in combs:
+ with self.subTest(
+ udf1=f1.__name__,
+ udf2=f2.__name__,
+ udf3=f3.__name__,
+ udf4=f4.__name__,
+ udf5=f5.__name__,
+ udf6=f6.__name__,
+ ):
+ result = df.select(f1(f2(f3(f4(f5(f6("id")))))).alias("res"))
+ self.assertEqual(expected, result.collect())
+
+
+class UDFCombinationsTests(UDFCombinationsTestsMixin, ReusedSQLTestCase):
+ @classmethod
+ def setUpClass(cls):
+ ReusedSQLTestCase.setUpClass()
+ cls.spark.conf.set("spark.sql.execution.pythonUDF.arrow.enabled", "false")
+
+
+if __name__ == "__main__":
+ from pyspark.sql.tests.test_udf_combinations import * # noqa: F401
+
+ try:
+ import xmlrunner # type: ignore
+
+ testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2)
+ except ImportError:
+ testRunner = None
+ unittest.main(testRunner=testRunner, verbosity=2)
diff --git a/python/pyspark/sql/tests/test_udf_profiler.py b/python/pyspark/sql/tests/test_udf_profiler.py
index 8c45149605c72..de35532285df6 100644
--- a/python/pyspark/sql/tests/test_udf_profiler.py
+++ b/python/pyspark/sql/tests/test_udf_profiler.py
@@ -28,7 +28,7 @@
from pyspark import SparkConf
from pyspark.errors import PySparkValueError
from pyspark.sql import SparkSession
-from pyspark.sql.functions import col, pandas_udf, udf
+from pyspark.sql.functions import col, arrow_udf, pandas_udf, udf
from pyspark.sql.window import Window
from pyspark.profiler import UDFBasicProfiler
from pyspark.testing.sqlutils import ReusedSQLTestCase
@@ -127,6 +127,16 @@ def iter_to_iter(batch_ser: Iterator[pd.Series]) -> Iterator[pd.Series]:
self.spark.range(10).select(iter_to_iter("id")).collect()
+ def exec_arrow_udf_iter_to_iter(self):
+ import pyarrow as pa
+
+ @arrow_udf("int")
+ def iter_to_iter(iter: Iterator[pa.Array]) -> Iterator[pa.Array]:
+ for s in iter:
+ yield pa.compute.add(s, 1)
+
+ self.spark.range(10).select(iter_to_iter("id")).collect()
+
# Unsupported
def exec_map(self):
import pandas as pd
@@ -149,6 +159,15 @@ def test_unsupported(self):
"Profiling UDFs with iterators input/output is not supported" in str(user_warns[0])
)
+ with warnings.catch_warnings(record=True) as warns:
+ warnings.simplefilter("always")
+ self.exec_arrow_udf_iter_to_iter()
+ user_warns = [warn.message for warn in warns if isinstance(warn.message, UserWarning)]
+ self.assertTrue(len(user_warns) > 0)
+ self.assertTrue(
+ "Profiling UDFs with iterators input/output is not supported" in str(user_warns[0])
+ )
+
with warnings.catch_warnings(record=True) as warns:
warnings.simplefilter("always")
self.exec_map()
@@ -278,6 +297,29 @@ def add2(x):
for id in self.profile_results:
self.assert_udf_profile_present(udf_id=id, expected_line_count_prefix=2)
+ @unittest.skipIf(not have_pyarrow, pyarrow_requirement_message)
+ def test_perf_profiler_arrow_udf(self):
+ import pyarrow as pa
+
+ @arrow_udf("long")
+ def add1(x):
+ return pa.compute.add(x, 1)
+
+ @arrow_udf("long")
+ def add2(x):
+ return pa.compute.add(x, 2)
+
+ with self.sql_conf({"spark.sql.pyspark.udf.profiler": "perf"}):
+ df = self.spark.range(10, numPartitions=2).select(
+ add1("id"), add2("id"), add1("id"), add2(col("id") + 1)
+ )
+ df.collect()
+
+ self.assertEqual(3, len(self.profile_results), str(self.profile_results.keys()))
+
+ for id in self.profile_results:
+ self.assert_udf_profile_present(udf_id=id, expected_line_count_prefix=2)
+
@unittest.skipIf(
not have_pandas or not have_pyarrow,
cast(str, pandas_requirement_message or pyarrow_requirement_message),
@@ -305,6 +347,30 @@ def add2(iter: Iterator[pd.Series]) -> Iterator[pd.Series]:
for id in self.profile_results:
self.assert_udf_profile_present(udf_id=id, expected_line_count_prefix=2)
+ @unittest.skipIf(not have_pyarrow, pyarrow_requirement_message)
+ def test_perf_profiler_arrow_udf_iterator_not_supported(self):
+ import pyarrow as pa
+
+ @arrow_udf("long")
+ def add1(x):
+ return pa.compute.add(x, 1)
+
+ @arrow_udf("long")
+ def add2(iter: Iterator[pa.Array]) -> Iterator[pa.Array]:
+ for s in iter:
+ yield pa.compute.add(s, 2)
+
+ with self.sql_conf({"spark.sql.pyspark.udf.profiler": "perf"}):
+ df = self.spark.range(10, numPartitions=2).select(
+ add1("id"), add2("id"), add1("id"), add2(col("id") + 1)
+ )
+ df.collect()
+
+ self.assertEqual(1, len(self.profile_results), str(self.profile_results.keys()))
+
+ for id in self.profile_results:
+ self.assert_udf_profile_present(udf_id=id, expected_line_count_prefix=2)
+
@unittest.skipIf(
not have_pandas or not have_pyarrow,
cast(str, pandas_requirement_message or pyarrow_requirement_message),
@@ -346,6 +412,27 @@ def mean_udf(v: pd.Series) -> float:
for id in self.profile_results:
self.assert_udf_profile_present(udf_id=id, expected_line_count_prefix=5)
+ @unittest.skipIf(not have_pyarrow, pyarrow_requirement_message)
+ def test_perf_profiler_arrow_udf_window(self):
+ import pyarrow as pa
+
+ @arrow_udf("double")
+ def mean_udf(v: pa.Array) -> float:
+ return pa.compute.mean(v)
+
+ df = self.spark.createDataFrame(
+ [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)], ("id", "v")
+ )
+ w = Window.partitionBy("id").orderBy("v").rowsBetween(-1, 0)
+
+ with self.sql_conf({"spark.sql.pyspark.udf.profiler": "perf"}):
+ df.withColumn("mean_v", mean_udf("v").over(w)).show()
+
+ self.assertEqual(1, len(self.profile_results), str(self.profile_results.keys()))
+
+ for id in self.profile_results:
+ self.assert_udf_profile_present(udf_id=id, expected_line_count_prefix=5)
+
@unittest.skipIf(
not have_pandas or not have_pyarrow,
cast(str, pandas_requirement_message or pyarrow_requirement_message),
@@ -369,6 +456,25 @@ def min_udf(v: pd.Series) -> float:
for id in self.profile_results:
self.assert_udf_profile_present(udf_id=id, expected_line_count_prefix=2)
+ @unittest.skipIf(not have_pyarrow, pyarrow_requirement_message)
+ def test_perf_profiler_arrow_udf_agg(self):
+ import pyarrow as pa
+
+ @arrow_udf("double")
+ def min_udf(v: pa.Array) -> float:
+ return pa.compute.min(v)
+
+ with self.sql_conf({"spark.sql.pyspark.udf.profiler": "perf"}):
+ df = self.spark.createDataFrame(
+ [(2, "Alice"), (3, "Alice"), (5, "Bob"), (10, "Bob")], ["age", "name"]
+ )
+ df.groupBy(df.name).agg(min_udf(df.age)).show()
+
+ self.assertEqual(1, len(self.profile_results), str(self.profile_results.keys()))
+
+ for id in self.profile_results:
+ self.assert_udf_profile_present(udf_id=id, expected_line_count_prefix=2)
+
@unittest.skipIf(
not have_pandas or not have_pyarrow,
cast(str, pandas_requirement_message or pyarrow_requirement_message),
diff --git a/python/pyspark/sql/tests/test_udtf.py b/python/pyspark/sql/tests/test_udtf.py
index b5536ddc7b5d0..b006ac6c14d4a 100644
--- a/python/pyspark/sql/tests/test_udtf.py
+++ b/python/pyspark/sql/tests/test_udtf.py
@@ -63,6 +63,7 @@
VariantVal,
)
from pyspark.testing import assertDataFrameEqual, assertSchemaEqual
+from pyspark.testing.objects import ExamplePoint, ExamplePointUDT
from pyspark.testing.sqlutils import (
have_pandas,
have_pyarrow,
@@ -169,11 +170,58 @@ def eval(self, a: int, b: int) -> Iterator:
self.spark.udtf.register("testUDTF", TestUDTF)
+ for i, df in enumerate(
+ [
+ self.spark.sql("values (0, 1), (1, 2) t(a, b)").lateralJoin(
+ TestUDTF(col("a").outer(), col("b").outer())
+ ),
+ self.spark.sql("values (0, 1), (1, 2) t(a, b)").lateralJoin(
+ TestUDTF(a=col("a").outer(), b=col("b").outer())
+ ),
+ self.spark.sql("values (0, 1), (1, 2) t(a, b)").lateralJoin(
+ TestUDTF(b=col("b").outer(), a=col("a").outer())
+ ),
+ ]
+ ):
+ with self.subTest(query_no=i):
+ assertDataFrameEqual(
+ df,
+ self.spark.sql(
+ "SELECT * FROM values (0, 1), (1, 2) t(a, b), LATERAL testUDTF(a, b)"
+ ),
+ )
+
+ @udtf(returnType="a: int")
+ class TestUDTF:
+ def eval(self):
+ yield 1,
+ yield 2,
+
assertDataFrameEqual(
- self.spark.sql("values (0, 1), (1, 2) t(a, b)").lateralJoin(
- TestUDTF(col("a").outer(), col("b").outer())
- ),
- self.spark.sql("SELECT * FROM values (0, 1), (1, 2) t(a, b), LATERAL testUDTF(a, b)"),
+ self.spark.range(3, numPartitions=1).lateralJoin(TestUDTF()),
+ [
+ Row(id=0, a=1),
+ Row(id=0, a=2),
+ Row(id=1, a=1),
+ Row(id=1, a=2),
+ Row(id=2, a=1),
+ Row(id=2, a=2),
+ ],
+ )
+
+ @udtf(returnType="a: int")
+ class TestUDTF:
+ def eval(self, i: int):
+ for n in range(i):
+ yield n,
+
+ assertDataFrameEqual(
+ self.spark.range(3, numPartitions=1).lateralJoin(TestUDTF(col("id").outer())),
+ [
+ Row(id=1, a=0),
+ Row(id=2, a=0),
+ Row(id=2, a=1),
+ ],
)
def test_udtf_eval_with_return_stmt(self):
@@ -202,6 +250,62 @@ def eval(self, a: int):
with self.assertRaisesRegex(PythonException, "UDTF_INVALID_OUTPUT_ROW_TYPE"):
TestUDTF(lit(1)).collect()
+ @udtf(returnType="a: int")
+ class TestUDTF:
+ def eval(self, a: int):
+ return [a]
+
+ with self.assertRaisesRegex(PythonException, "UDTF_INVALID_OUTPUT_ROW_TYPE"):
+ TestUDTF(lit(1)).collect()
+
+ @udtf(returnType=StructType().add("point", ExamplePointUDT()))
+ class TestUDTF:
+ def eval(self, x: float, y: float):
+ yield ExamplePoint(x=x * 10, y=y * 10)
+
+ with self.assertRaisesRegex(PythonException, "UDTF_INVALID_OUTPUT_ROW_TYPE"):
+ TestUDTF(lit(1.0), lit(2.0)).collect()
+
+ def test_udtf_eval_returning_tuple_with_struct_type(self):
+ @udtf(returnType="a: struct")
+ class TestUDTF:
+ def eval(self, a: int):
+ yield (a, a + 1),
+
+ assertDataFrameEqual(TestUDTF(lit(1)), [Row(a=Row(b=1, c=2))])
+
+ @udtf(returnType="a: struct")
+ class TestUDTF:
+ def eval(self, a: int):
+ yield a, a + 1
+
+ with self.assertRaisesRegex(PythonException, "UDTF_RETURN_SCHEMA_MISMATCH"):
+ TestUDTF(lit(1)).collect()
+
+ def test_udtf_eval_returning_udt(self):
+ @udtf(returnType=StructType().add("point", ExamplePointUDT()))
+ class TestUDTF:
+ def eval(self, x: float, y: float):
+ yield ExamplePoint(x=x * 10, y=y * 10),
+
+ assertDataFrameEqual(
+ TestUDTF(lit(1.0), lit(2.0)), [Row(point=ExamplePoint(x=10.0, y=20.0))]
+ )
+
+ def test_udtf_eval_taking_udt(self):
+ @udtf(returnType="x: double, y: double")
+ class TestUDTF:
+ def eval(self, point: ExamplePoint):
+ yield point.x * 10, point.y * 10
+
+ df = self.spark.createDataFrame(
+ [(ExamplePoint(x=1.0, y=2.0),)], schema=StructType().add("point", ExamplePointUDT())
+ )
+ assertDataFrameEqual(
+ df.lateralJoin(TestUDTF(col("point").outer())),
+ [Row(point=ExamplePoint(x=1.0, y=2.0), x=10.0, y=20.0)],
+ )
+
def test_udtf_with_invalid_return_value(self):
@udtf(returnType="x: int")
class TestUDTF:
@@ -351,15 +455,13 @@ def terminate(self, a: int):
TestUDTF(lit(1)).show()
def test_udtf_with_wrong_num_output(self):
- err_msg = "(UDTF_ARROW_TYPE_CONVERSION_ERROR|UDTF_RETURN_SCHEMA_MISMATCH)"
-
# Output less columns than specified return schema
@udtf(returnType="a: int, b: int")
class TestUDTF:
def eval(self, a: int):
yield a,
- with self.assertRaisesRegex(PythonException, err_msg):
+ with self.assertRaisesRegex(PythonException, "UDTF_RETURN_SCHEMA_MISMATCH"):
TestUDTF(lit(1)).collect()
# Output more columns than specified return schema
@@ -368,7 +470,7 @@ class TestUDTF:
def eval(self, a: int):
yield a, a + 1
- with self.assertRaisesRegex(PythonException, err_msg):
+ with self.assertRaisesRegex(PythonException, "UDTF_RETURN_SCHEMA_MISMATCH"):
TestUDTF(lit(1)).collect()
def test_udtf_with_empty_output_schema_and_non_empty_output(self):
@@ -377,9 +479,7 @@ class TestUDTF:
def eval(self):
yield 1,
- with self.assertRaisesRegex(
- PythonException, "(UDTF_RETURN_SCHEMA_MISMATCH|UDTF_ARROW_TYPE_CONVERSION_ERROR)"
- ):
+ with self.assertRaisesRegex(PythonException, "UDTF_RETURN_SCHEMA_MISMATCH"):
TestUDTF().collect()
def test_udtf_with_non_empty_output_schema_and_empty_output(self):
@@ -388,9 +488,7 @@ class TestUDTF:
def eval(self):
yield tuple()
- with self.assertRaisesRegex(
- PythonException, "(UDTF_RETURN_SCHEMA_MISMATCH|UDTF_ARROW_TYPE_CONVERSION_ERROR)"
- ):
+ with self.assertRaisesRegex(PythonException, "UDTF_RETURN_SCHEMA_MISMATCH"):
TestUDTF().collect()
def test_udtf_init(self):
@@ -545,8 +643,6 @@ def terminate(self):
TestUDTF(lit(1)).collect()
def test_udtf_terminate_with_wrong_num_output(self):
- err_msg = "(UDTF_RETURN_SCHEMA_MISMATCH|UDTF_ARROW_TYPE_CONVERSION_ERROR)"
-
@udtf(returnType="a: int, b: int")
class TestUDTF:
def eval(self, a: int):
@@ -555,7 +651,7 @@ def eval(self, a: int):
def terminate(self):
yield 1, 2, 3
- with self.assertRaisesRegex(PythonException, err_msg):
+ with self.assertRaisesRegex(PythonException, "UDTF_RETURN_SCHEMA_MISMATCH"):
TestUDTF(lit(1)).show()
@udtf(returnType="a: int, b: int")
@@ -566,7 +662,7 @@ def eval(self, a: int):
def terminate(self):
yield 1,
- with self.assertRaisesRegex(PythonException, err_msg):
+ with self.assertRaisesRegex(PythonException, "UDTF_RETURN_SCHEMA_MISMATCH"):
TestUDTF(lit(1)).show()
def test_udtf_determinism(self):
@@ -2036,6 +2132,25 @@ def eval(self, a, b):
with self.subTest(query_no=i):
assertDataFrameEqual(df, [Row(a=10)])
+ def test_udtf_with_named_table_arguments(self):
+ @udtf(returnType="a: int")
+ class TestUDTF:
+ def eval(self, a, b):
+ yield a.id,
+
+ self.spark.udtf.register("test_udtf", TestUDTF)
+
+ for i, df in enumerate(
+ [
+ self.spark.sql("SELECT * FROM test_udtf(a => TABLE(FROM range(3)), b => 'x')"),
+ self.spark.sql("SELECT * FROM test_udtf(b => 'x', a => TABLE(FROM range(3)))"),
+ TestUDTF(a=self.spark.range(3).asTable(), b=lit("x")),
+ TestUDTF(b=lit("x"), a=self.spark.range(3).asTable()),
+ ]
+ ):
+ with self.subTest(query_no=i):
+ assertDataFrameEqual(df, [Row(a=i) for i in range(3)])
+
def test_udtf_with_named_arguments_negative(self):
@udtf(returnType="a: int")
class TestUDTF:
@@ -2088,6 +2203,25 @@ def eval(self, **kwargs):
with self.assertRaisesRegex(AnalysisException, "UNEXPECTED_POSITIONAL_ARGUMENT"):
self.spark.sql("SELECT * FROM test_udtf(a => 10, 'x')").show()
+ def test_udtf_with_table_argument_and_kwargs(self):
+ @udtf(returnType="a: int, b: string")
+ class TestUDTF:
+ def eval(self, **kwargs):
+ yield kwargs["a"].id, kwargs["b"]
+
+ self.spark.udtf.register("test_udtf", TestUDTF)
+
+ for i, df in enumerate(
+ [
+ self.spark.sql("SELECT * FROM test_udtf(a => TABLE(FROM range(3)), b => 'x')"),
+ self.spark.sql("SELECT * FROM test_udtf(b => 'x', a => TABLE(FROM range(3)))"),
+ TestUDTF(a=self.spark.range(3).asTable(), b=lit("x")),
+ TestUDTF(b=lit("x"), a=self.spark.range(3).asTable()),
+ ]
+ ):
+ with self.subTest(query_no=i):
+ assertDataFrameEqual(df, [Row(a=i, b="x") for i in range(3)])
+
def test_udtf_with_analyze_kwargs(self):
@udtf
class TestUDTF:
@@ -2122,6 +2256,38 @@ def eval(self, **kwargs):
with self.subTest(query_no=i):
assertDataFrameEqual(df, [Row(a=10, b="x")])
+ def test_udtf_with_table_argument_and_analyze_kwargs(self):
+ @udtf
+ class TestUDTF:
+ @staticmethod
+ def analyze(**kwargs: AnalyzeArgument) -> AnalyzeResult:
+ assert isinstance(kwargs["a"].dataType, StructType)
+ assert kwargs["a"].isTable is True
+ assert isinstance(kwargs["b"].dataType, StringType)
+ assert kwargs["b"].value == "x"
+ assert not kwargs["b"].isTable
+ return AnalyzeResult(
+ StructType(
+ [StructField(key, arg.dataType) for key, arg in sorted(kwargs.items())]
+ )
+ )
+
+ def eval(self, **kwargs):
+ yield tuple(value for _, value in sorted(kwargs.items()))
+
+ self.spark.udtf.register("test_udtf", TestUDTF)
+
+ for i, df in enumerate(
+ [
+ self.spark.sql("SELECT * FROM test_udtf(a => TABLE(FROM range(3)), b => 'x')"),
+ self.spark.sql("SELECT * FROM test_udtf(b => 'x', a => TABLE(FROM range(3)))"),
+ TestUDTF(a=self.spark.range(3).asTable(), b=lit("x")),
+ TestUDTF(b=lit("x"), a=self.spark.range(3).asTable()),
+ ]
+ ):
+ with self.subTest(query_no=i):
+ assertDataFrameEqual(df, [Row(a=Row(id=i), b="x") for i in range(3)])
+
def test_udtf_with_named_arguments_lateral_join(self):
@udtf
class TestUDTF:
@@ -2841,6 +3007,43 @@ def eval(self):
err_type=Exception,
)
+ def test_udtf_with_collated_string_types(self):
+ @udtf(
+ returnType="out1 string, out2 string collate UTF8_BINARY, "
+ "out3 string collate UTF8_LCASE, out4 string collate UNICODE"
+ )
+ class MyUDTF:
+ def eval(self, v1, v2, v3, v4):
+ yield (v1 + "1", v2 + "2", v3 + "3", v4 + "4")
+
+ schema = StructType(
+ [
+ StructField("col1", StringType(), True),
+ StructField("col2", StringType("UTF8_BINARY"), True),
+ StructField("col3", StringType("UTF8_LCASE"), True),
+ StructField("col4", StringType("UNICODE"), True),
+ ]
+ )
+ df = self.spark.createDataFrame([("hello",) * 4], schema=schema)
+
+ result_df = df.lateralJoin(
+ MyUDTF(
+ col("col1").outer(), col("col2").outer(), col("col3").outer(), col("col4").outer()
+ )
+ ).select("out1", "out2", "out3", "out4")
+
+ expected_row = ("hello1", "hello2", "hello3", "hello4")
+ self.assertEqual(result_df.collect()[0], expected_row)
+
+ expected_output_types = [
+ StringType(),
+ StringType("UTF8_BINARY"),
+ StringType("UTF8_LCASE"),
+ StringType("UNICODE"),
+ ]
+ for idx, field in enumerate(result_df.schema.fields):
+ self.assertEqual(field.dataType, expected_output_types[idx])
+
class UDTFTests(BaseUDTFTestsMixin, ReusedSQLTestCase):
@classmethod
@@ -2905,6 +3108,13 @@ def eval(self, a: int):
# When arrow is enabled, it can handle non-tuple return value.
assertDataFrameEqual(TestUDTF(lit(1)), [Row(a=1)])
+ @udtf(returnType="a: int")
+ class TestUDTF:
+ def eval(self, a: int):
+ return (a,)
+
+ assertDataFrameEqual(TestUDTF(lit(1)), [Row(a=1)])
+
@udtf(returnType="a: int")
class TestUDTF:
def eval(self, a: int):
@@ -2912,6 +3122,27 @@ def eval(self, a: int):
assertDataFrameEqual(TestUDTF(lit(1)), [Row(a=1)])
+ @udtf(returnType=StructType().add("udt", ExamplePointUDT()))
+ class TestUDTF:
+ def eval(self, x: float, y: float):
+ yield ExamplePoint(x=x * 10, y=y * 10)
+
+ assertDataFrameEqual(TestUDTF(lit(1.0), lit(2.0)), [Row(udt=ExamplePoint(x=10.0, y=20.0))])
+
+ def test_udtf_use_large_var_types(self):
+ for use_large_var_types in [True, False]:
+ with self.subTest(use_large_var_types=use_large_var_types):
+ with self.sql_conf(
+ {"spark.sql.execution.arrow.useLargeVarTypes": use_large_var_types}
+ ):
+
+ @udtf(returnType="a: string")
+ class TestUDTF:
+ def eval(self, a: int):
+ yield str(a)
+
+ assertDataFrameEqual(TestUDTF(lit(1)), [Row(a="1")])
+
def test_numeric_output_type_casting(self):
class TestUDTF:
def eval(self):
@@ -3158,23 +3389,6 @@ def tearDownClass(cls):
class UDTFArrowTestsMixin(LegacyUDTFArrowTestsMixin):
- def test_udtf_eval_returning_non_tuple(self):
- @udtf(returnType="a: int")
- class TestUDTF:
- def eval(self, a: int):
- yield a
-
- with self.assertRaisesRegex(PythonException, "UDTF_ARROW_TYPE_CONVERSION_ERROR"):
- TestUDTF(lit(1)).collect()
-
- @udtf(returnType="a: int")
- class TestUDTF:
- def eval(self, a: int):
- return [a]
-
- with self.assertRaisesRegex(PythonException, "UDTF_ARROW_TYPE_CONVERSION_ERROR"):
- TestUDTF(lit(1)).collect()
-
def test_numeric_output_type_casting(self):
class TestUDTF:
def eval(self):
diff --git a/python/pyspark/sql/tests/udf_type_tests/README.md b/python/pyspark/sql/tests/udf_type_tests/README.md
new file mode 100644
index 0000000000000..74d1933c49519
--- /dev/null
+++ b/python/pyspark/sql/tests/udf_type_tests/README.md
@@ -0,0 +1,13 @@
+These tests capture input/output type interfaces between python udfs and the engine. This internal documentation, not user-facing documentation. Please consider the type behavior "experimental", unless we specify otherwise. Parts of the type handling might change in the future.
+
+# Return type tests
+These generate tables with the returned 'Python Value' and the 'SQL Type' output type of the UDF. The 'SQL Type' fields are DDL formatted strings, which can be used as `returnType`s.
+- Note: The values inside the table are generated by `repr`. X' means it throws an exception during the conversion.
+- Note: Python 3.11.9, Pandas 2.2.3 and PyArrow 17.0.0 are used.
+
+# Input type tests
+These generate tables with 'Spark Type' and 'Spark Value', representing the engine-side input data. The UDF input data is captured in the 'Python type' and 'Python value' columns.
+
+# When this test fails:
+- Look at the diff in the test output
+- To regenerate golden files, simply delete the existing golden file and re-run the test.
\ No newline at end of file
diff --git a/python/pyspark/sql/tests/udf_type_tests/__init__.py b/python/pyspark/sql/tests/udf_type_tests/__init__.py
new file mode 100644
index 0000000000000..cce3acad34a49
--- /dev/null
+++ b/python/pyspark/sql/tests/udf_type_tests/__init__.py
@@ -0,0 +1,16 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
diff --git a/python/pyspark/sql/tests/udf_type_tests/golden_pandas_udf_input_types.txt b/python/pyspark/sql/tests/udf_type_tests/golden_pandas_udf_input_types.txt
new file mode 100644
index 0000000000000..d21e7f2eb24a1
--- /dev/null
+++ b/python/pyspark/sql/tests/udf_type_tests/golden_pandas_udf_input_types.txt
@@ -0,0 +1,43 @@
++--------------------------------------------------------------------------------------+--------------------------------------------------------------------------------------+--------------------------------------------------------------------------------------+--------------------------------------------------------------------------------------+--------------------------------------------------------------------------------------+
+|Test Case |Spark Type |Spark Value |Python Type |Python Value |
++--------------------------------------------------------------------------------------+--------------------------------------------------------------------------------------+--------------------------------------------------------------------------------------+--------------------------------------------------------------------------------------+--------------------------------------------------------------------------------------+
+|byte_values |tinyint |[-128, 127, 0] |['int8', 'int8', 'int8'] |[-128, 127, 0] |
+|byte_null |tinyint |[None, 42] |['float64', 'float64'] |[None, 42] |
+|short_values |smallint |[-32768, 32767, 0] |['int16', 'int16', 'int16'] |[-32768, 32767, 0] |
+|short_null |smallint |[None, 123] |['float64', 'float64'] |[None, 123] |
+|int_values |int |[-2147483648, 2147483647, 0] |['int32', 'int32', 'int32'] |[-2147483648, 2147483647, 0] |
+|int_null |int |[None, 456] |['float64', 'float64'] |[None, 456] |
+|long_values |bigint |[-9223372036854775808, 9223372036854775807, 0] |['int64', 'int64', 'int64'] |[-9223372036854775808, 9223372036854775807, 0] |
+|long_null |bigint |[None, 789] |['float64', 'float64'] |[None, 789] |
+|float_values |float |[0.0, 1.0, 3.140000104904175] |['float32', 'float32', 'float32'] |[0.0, 1.0, 3.140000104904175] |
+|float_null |float |[None, 3.140000104904175] |['float32', 'float32'] |[None, 3.140000104904175] |
+|double_values |double |[0.0, 1.0, 0.3333333333333333] |['float64', 'float64', 'float64'] |[0.0, 1.0, 0.3333333333333333] |
+|double_null |double |[None, 2.71] |['float64', 'float64'] |[None, 2.71] |
+|decimal_values |decimal(3,2) |[Decimal('5.35'), Decimal('1.23')] |['object', 'object'] |[Decimal('5.35'), Decimal('1.23')] |
+|decimal_null |decimal(3,2) |[None, Decimal('9.99')] |['object', 'object'] |[None, Decimal('9.99')] |
+|string_values |string |['abc', '', 'hello'] |['object', 'object', 'object'] |['abc', '', 'hello'] |
+|string_null |string |[None, 'test'] |['object', 'object'] |[None, 'test'] |
+|binary_values |binary |[bytearray(b'abc'), bytearray(b''), bytearray(b'ABC')] |['object', 'object', 'object'] |[bytearray(b'abc'), bytearray(b''), bytearray(b'ABC')] |
+|binary_null |binary |[None, bytearray(b'test')] |['object', 'object'] |[None, bytearray(b'test')] |
+|boolean_values |boolean |[True, False] |['bool', 'bool'] |[True, False] |
+|boolean_null |boolean |[None, True] |['object', 'object'] |[None, True] |
+|date_values |date |[datetime.date(2020, 2, 2), datetime.date(1970, 1, 1)] |['object', 'object'] |[datetime.date(2020, 2, 2), datetime.date(1970, 1, 1)] |
+|date_null |date |[None, datetime.date(2023, 1, 1)] |['object', 'object'] |[None, datetime.date(2023, 1, 1)] |
+|timestamp_values |timestamp |[datetime.datetime(2020, 2, 2, 12, 15, 16, 123000)] |['datetime64[ns]'] |[datetime.datetime(2020, 2, 2, 12, 15, 16, 123000)] |
+|timestamp_null |timestamp |[None, datetime.datetime(2023, 1, 1, 12, 0)] |['datetime64[ns]', 'datetime64[ns]'] |[None, datetime.datetime(2023, 1, 1, 12, 0)] |
+|array_int_values |array |[[1, 2, 3], [], [1, None, 3]] |['object', 'object', 'object'] |[[1, 2, 3], [], [1, None, 3]] |
+|array_int_null |array |[None, [4, 5, 6]] |['object', 'object'] |[None, [4, 5, 6]] |
+|map_str_int_values |map |[{'world': 2, 'hello': 1}, {}] |['object', 'object'] |[{'world': 2, 'hello': 1}, {}] |
+|map_str_int_null |map |[None, {'test': 123}] |['object', 'object'] |[None, {'test': 123}] |
+|struct_int_str_values |struct |[Row(a1=1, a2='hello'), Row(a1=2, a2='world')] |['DataFrame', 'DataFrame'] |[Row(a1=1, a2='hello'), Row(a1=2, a2='world')] |
+|struct_int_str_null |struct |[None, Row(a1=99, a2='test')] |['DataFrame', 'DataFrame'] |[Row(a1=None, a2=None), Row(a1=99, a2='test')] |
+|array_array_int |array> |[[[1, 2, 3]], [[1], [2, 3]]] |['object', 'object'] |[[[1, 2, 3]], [[1], [2, 3]]] |
+|array_map_str_int |array
|""".stripMargin
- Files.asCharSink(new File(customConf, "core-site.xml"), StandardCharsets.UTF_8).write(coreSite)
+ Files.writeString(new File(customConf, "core-site.xml").toPath, coreSite)
val result = File.createTempFile("result", null, tempDir)
val finalState = runSpark(false,
@@ -353,7 +352,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
test("running Spark in yarn-cluster mode displays driver log links") {
val log4jConf = new File(tempDir, "log4j2.properties")
val logOutFile = new File(tempDir, "logs")
- Files.asCharSink(log4jConf, StandardCharsets.UTF_8).write(
+ Files.writeString(log4jConf.toPath,
s"""rootLogger.level = debug
|rootLogger.appenderRef.file.ref = file
|appender.file.type = File
@@ -367,8 +366,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
val confDir = new File(tempDir, "conf")
confDir.mkdir()
val javaOptsFile = new File(confDir, "java-opts")
- Files.asCharSink(javaOptsFile, StandardCharsets.UTF_8)
- .write(s"-Dlog4j.configurationFile=file://$log4jConf\n")
+ Files.writeString(javaOptsFile.toPath, s"-Dlog4j.configurationFile=file://$log4jConf\n")
val result = File.createTempFile("result", null, tempDir)
val finalState = runSpark(clientMode = false,
@@ -377,7 +375,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
extraEnv = Map("SPARK_CONF_DIR" -> confDir.getAbsolutePath),
extraConf = Map(CLIENT_INCLUDE_DRIVER_LOGS_LINK.key -> true.toString))
checkResult(finalState, result)
- val logOutput = Files.asCharSource(logOutFile, StandardCharsets.UTF_8).read()
+ val logOutput = Files.readString(logOutFile.toPath)
val logFilePattern = raw"""(?s).+\sDriver Logs \(\): https?://.+/(\?\S+)?\s.+"""
logOutput should fullyMatch regex logFilePattern.replace("", "stdout")
logOutput should fullyMatch regex logFilePattern.replace("", "stderr")
@@ -432,7 +430,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
script: String = TEST_PYFILE): Unit = {
assume(isPythonAvailable)
val primaryPyFile = new File(tempDir, "test.py")
- Files.asCharSink(primaryPyFile, StandardCharsets.UTF_8).write(script)
+ Files.writeString(primaryPyFile.toPath, script)
// When running tests, let's not assume the user has built the assembly module, which also
// creates the pyspark archive. Instead, let's use PYSPARK_ARCHIVES_PATH to point at the
@@ -460,7 +458,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
subdir
}
val pyModule = new File(moduleDir, "mod1.py")
- Files.asCharSink(pyModule, StandardCharsets.UTF_8).write(TEST_PYMODULE)
+ Files.writeString(pyModule.toPath, TEST_PYMODULE)
val mod2Archive = TestUtils.createJarWithFiles(Map("mod2.py" -> TEST_PYMODULE), moduleDir)
val pyFiles = Seq(pyModule.getAbsolutePath(), mod2Archive.getPath()).mkString(",")
@@ -507,7 +505,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite {
def createEmptyIvySettingsFile: File = {
val emptyIvySettings = File.createTempFile("ivy", ".xml")
- Files.asCharSink(emptyIvySettings, StandardCharsets.UTF_8).write("")
+ Files.writeString(emptyIvySettings.toPath, "")
emptyIvySettings
}
@@ -619,7 +617,7 @@ private object YarnClusterDriverUseSparkHadoopUtilConf extends Logging with Matc
}
result = "success"
} finally {
- Files.asCharSink(status, StandardCharsets.UTF_8).write(result)
+ Files.writeString(status.toPath, result)
sc.stop()
}
}
@@ -722,7 +720,7 @@ private object YarnClusterDriver extends Logging with Matchers {
assert(driverAttributes === expectationAttributes)
}
} finally {
- Files.asCharSink(status, StandardCharsets.UTF_8).write(result)
+ Files.writeString(status.toPath, result)
sc.stop()
}
}
@@ -764,14 +762,13 @@ private object YarnClasspathTest extends Logging {
var result = "failure"
try {
val ccl = Thread.currentThread().getContextClassLoader()
- val resource = ccl.getResourceAsStream("test.resource")
- val bytes = ByteStreams.toByteArray(resource)
+ val bytes = ccl.getResourceAsStream("test.resource").readAllBytes()
result = new String(bytes, 0, bytes.length, StandardCharsets.UTF_8)
} catch {
case t: Throwable =>
error(s"loading test.resource to $resultPath", t)
} finally {
- Files.asCharSink(new File(resultPath), StandardCharsets.UTF_8).write(result)
+ Files.writeString(new File(resultPath).toPath, result)
}
}
@@ -815,7 +812,7 @@ private object YarnAddJarTest extends Logging {
result = "success"
}
} finally {
- Files.asCharSink(new File(resultPath), StandardCharsets.UTF_8).write(result)
+ Files.writeString(new File(resultPath).toPath, result)
sc.stop()
}
}
@@ -860,7 +857,7 @@ private object ExecutorEnvTestApp {
executorEnvs.get(k).contains(v)
}
- Files.asCharSink(new File(status), StandardCharsets.UTF_8).write(result.toString)
+ Files.writeString(new File(status).toPath, result.toString)
sc.stop()
}
@@ -875,7 +872,7 @@ private class PyConnectDepChecker(python: String, libPath: Seq[String]) {
lazy val isSparkConnectJarAvailable: Boolean = {
val filePath = s"$sparkHome/assembly/target/$scalaDir/jars/" +
s"spark-connect_$scalaVersion-$SPARK_VERSION.jar"
- java.nio.file.Files.exists(Paths.get(filePath))
+ Files.exists(Paths.get(filePath))
}
lazy val isConnectPythonPackagesAvailable: Boolean = Try {
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala
index f8d69c0ae568e..2c567b3fa243d 100644
--- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala
@@ -18,10 +18,8 @@
package org.apache.spark.deploy.yarn
import java.io.File
-import java.nio.charset.StandardCharsets
+import java.nio.file.Files
-import com.google.common.io.Files
-import org.apache.commons.io.FileUtils
import org.apache.hadoop.yarn.conf.YarnConfiguration
import org.scalatest.matchers.must.Matchers
import org.scalatest.matchers.should.Matchers._
@@ -34,6 +32,7 @@ import org.apache.spark.network.shuffle.ShuffleTestAccessor
import org.apache.spark.network.shuffledb.DBBackend
import org.apache.spark.network.yarn.{YarnShuffleService, YarnTestAccessor}
import org.apache.spark.tags.{ExtendedLevelDBTest, ExtendedYarnTest}
+import org.apache.spark.util.Utils
/**
* Integration test for the external shuffle service with a yarn mini-cluster
@@ -172,16 +171,16 @@ private object YarnExternalShuffleDriver extends Logging with Matchers {
val dbBackend = DBBackend.byName(dbBackendName)
logWarning(s"Use ${dbBackend.name()} as the implementation of " +
s"${SHUFFLE_SERVICE_DB_BACKEND.key}")
- FileUtils.copyDirectory(registeredExecFile, execStateCopy)
+ Utils.copyDirectory(registeredExecFile, execStateCopy)
assert(!ShuffleTestAccessor
.reloadRegisteredExecutors(dbBackend, execStateCopy).isEmpty)
}
} finally {
sc.stop()
if (execStateCopy != null) {
- FileUtils.deleteDirectory(execStateCopy)
+ Utils.deleteRecursively(execStateCopy)
}
- Files.asCharSink(status, StandardCharsets.UTF_8).write(result)
+ Files.writeString(status.toPath, result)
}
}
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala
index 562ce3b5bdbaf..a6b9caae8d369 100644
--- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala
@@ -18,9 +18,8 @@
package org.apache.spark.deploy.yarn
import java.io.{File, IOException}
-import java.nio.charset.StandardCharsets
+import java.nio.file.Files
-import com.google.common.io.{ByteStreams, Files}
import org.apache.hadoop.yarn.api.records.ApplicationAccessType
import org.apache.hadoop.yarn.conf.YarnConfiguration
import org.scalatest.matchers.must.Matchers
@@ -54,12 +53,12 @@ class YarnSparkHadoopUtilSuite extends SparkFunSuite with Matchers with ResetSys
val args = Array("arg1", "${arg.2}", "\"arg3\"", "'arg4'", "$arg5", "\\arg6")
try {
val argLine = args.map(a => YarnSparkHadoopUtil.escapeForShell(a)).mkString(" ")
- Files.write(("bash -c \"echo " + argLine + "\"").getBytes(StandardCharsets.UTF_8), scriptFile)
+ Files.writeString(scriptFile.toPath, "bash -c \"echo " + argLine + "\"")
scriptFile.setExecutable(true)
val proc = Runtime.getRuntime().exec(Array(scriptFile.getAbsolutePath()))
- val out = new String(ByteStreams.toByteArray(proc.getInputStream())).trim()
- val err = new String(ByteStreams.toByteArray(proc.getErrorStream()))
+ val out = Utils.toString(proc.getInputStream()).trim()
+ val err = Utils.toString(proc.getErrorStream())
val exitCode = proc.waitFor()
exitCode should be (0)
out should be (args.mkString(" "))
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala
index 56d7b7ff6a09e..72d684e15fc62 100644
--- a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala
@@ -129,12 +129,12 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers {
reduceId: Int,
blockId: String): AppShufflePartitionInfo = {
val dataFile = ShuffleTestAccessor.getMergedShuffleDataFile(mergeManager, partitionId, reduceId)
- dataFile.getParentFile.mkdirs()
+ Utils.createDirectory(dataFile.getParentFile)
val indexFile =
ShuffleTestAccessor.getMergedShuffleIndexFile(mergeManager, partitionId, reduceId)
- indexFile.getParentFile.mkdirs()
+ Utils.createDirectory(indexFile.getParentFile)
val metaFile = ShuffleTestAccessor.getMergedShuffleMetaFile(mergeManager, partitionId, reduceId)
- metaFile.getParentFile.mkdirs()
+ Utils.createDirectory(metaFile.getParentFile)
val partitionInfo = ShuffleTestAccessor.getOrCreateAppShufflePartitionInfo(
mergeManager, partitionId, reduceId, blockId)
@@ -616,35 +616,35 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers {
val mergeManager1DB = ShuffleTestAccessor.mergeManagerDB(mergeManager1)
ShuffleTestAccessor.recoveryFile(mergeManager1) should be (mergeMgrFile)
- ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1).size() equals 0
- ShuffleTestAccessor.reloadAppShuffleInfo(
- mergeManager1, mergeManager1DB).size() equals 0
+ assert(ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1).size() equals 0)
+ assert(ShuffleTestAccessor.reloadAppShuffleInfo(
+ mergeManager1, mergeManager1DB).size() equals 0)
mergeManager1.registerExecutor(app1Id.toString, mergedShuffleInfo1)
var appShuffleInfo = ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1)
- appShuffleInfo.size() equals 1
+ assert(appShuffleInfo.size() equals 1)
appShuffleInfo.get(app1Id.toString).getAppPathsInfo should be (appPathsInfo1)
var appShuffleInfoAfterReload =
ShuffleTestAccessor.reloadAppShuffleInfo(mergeManager1, mergeManager1DB)
- appShuffleInfoAfterReload.size() equals 1
+ assert(appShuffleInfoAfterReload.size() equals 1)
appShuffleInfoAfterReload.get(app1Id.toString).getAppPathsInfo should be (appPathsInfo1)
mergeManager1.registerExecutor(app2Attempt1Id.toString, mergedShuffleInfo2Attempt1)
appShuffleInfo = ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1)
- appShuffleInfo.size() equals 2
+ assert(appShuffleInfo.size() equals 2)
appShuffleInfo.get(app1Id.toString).getAppPathsInfo should be (appPathsInfo1)
appShuffleInfo.get(
app2Attempt1Id.toString).getAppPathsInfo should be (appPathsInfo2Attempt1)
appShuffleInfoAfterReload =
ShuffleTestAccessor.reloadAppShuffleInfo(mergeManager1, mergeManager1DB)
- appShuffleInfoAfterReload.size() equals 2
+ assert(appShuffleInfoAfterReload.size() equals 2)
appShuffleInfoAfterReload.get(app1Id.toString).getAppPathsInfo should be (appPathsInfo1)
appShuffleInfoAfterReload.get(
app2Attempt1Id.toString).getAppPathsInfo should be (appPathsInfo2Attempt1)
mergeManager1.registerExecutor(app3IdNoAttemptId.toString, mergedShuffleInfo3NoAttemptId)
appShuffleInfo = ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1)
- appShuffleInfo.size() equals 3
+ assert(appShuffleInfo.size() equals 3)
appShuffleInfo.get(app1Id.toString).getAppPathsInfo should be (appPathsInfo1)
appShuffleInfo.get(
app2Attempt1Id.toString).getAppPathsInfo should be (appPathsInfo2Attempt1)
@@ -652,7 +652,7 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers {
app3IdNoAttemptId.toString).getAppPathsInfo should be (appPathsInfo3NoAttempt)
appShuffleInfoAfterReload =
ShuffleTestAccessor.reloadAppShuffleInfo(mergeManager1, mergeManager1DB)
- appShuffleInfoAfterReload.size() equals 3
+ assert(appShuffleInfoAfterReload.size() equals 3)
appShuffleInfoAfterReload.get(app1Id.toString).getAppPathsInfo should be (appPathsInfo1)
appShuffleInfoAfterReload.get(
app2Attempt1Id.toString).getAppPathsInfo should be (appPathsInfo2Attempt1)
@@ -661,7 +661,7 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers {
mergeManager1.registerExecutor(app2Attempt2Id.toString, mergedShuffleInfo2Attempt2)
appShuffleInfo = ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1)
- appShuffleInfo.size() equals 3
+ assert(appShuffleInfo.size() equals 3)
appShuffleInfo.get(app1Id.toString).getAppPathsInfo should be (appPathsInfo1)
appShuffleInfo.get(
app2Attempt2Id.toString).getAppPathsInfo should be (appPathsInfo2Attempt2)
@@ -669,7 +669,7 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers {
app3IdNoAttemptId.toString).getAppPathsInfo should be (appPathsInfo3NoAttempt)
appShuffleInfoAfterReload =
ShuffleTestAccessor.reloadAppShuffleInfo(mergeManager1, mergeManager1DB)
- appShuffleInfoAfterReload.size() equals 3
+ assert(appShuffleInfoAfterReload.size() equals 3)
appShuffleInfoAfterReload.get(app1Id.toString).getAppPathsInfo should be (appPathsInfo1)
appShuffleInfoAfterReload.get(
app2Attempt2Id.toString).getAppPathsInfo should be (appPathsInfo2Attempt2)
@@ -678,14 +678,14 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers {
mergeManager1.applicationRemoved(app2Attempt2Id.toString, true)
appShuffleInfo = ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1)
- appShuffleInfo.size() equals 2
+ assert(appShuffleInfo.size() equals 2)
appShuffleInfo.get(app1Id.toString).getAppPathsInfo should be (appPathsInfo1)
assert(!appShuffleInfo.containsKey(app2Attempt2Id.toString))
appShuffleInfo.get(
app3IdNoAttemptId.toString).getAppPathsInfo should be (appPathsInfo3NoAttempt)
appShuffleInfoAfterReload =
ShuffleTestAccessor.reloadAppShuffleInfo(mergeManager1, mergeManager1DB)
- appShuffleInfoAfterReload.size() equals 2
+ assert(appShuffleInfoAfterReload.size() equals 2)
appShuffleInfoAfterReload.get(app1Id.toString).getAppPathsInfo should be (appPathsInfo1)
assert(!appShuffleInfoAfterReload.containsKey(app2Attempt2Id.toString))
appShuffleInfoAfterReload.get(
@@ -725,9 +725,9 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers {
val mergeManager1DB = ShuffleTestAccessor.mergeManagerDB(mergeManager1)
ShuffleTestAccessor.recoveryFile(mergeManager1) should be (mergeMgrFile)
- ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1).size() equals 0
- ShuffleTestAccessor.reloadAppShuffleInfo(
- mergeManager1, mergeManager1DB).size() equals 0
+ assert(ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1).size() equals 0)
+ assert(ShuffleTestAccessor.reloadAppShuffleInfo(
+ mergeManager1, mergeManager1DB).size() equals 0)
mergeManager1.registerExecutor(app1Id.toString, mergedShuffleInfo1)
mergeManager1.registerExecutor(app2Attempt1Id.toString, mergedShuffleInfo2Attempt1)
@@ -737,7 +737,7 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers {
prepareAppShufflePartition(mergeManager1, partitionId2, 2, "4")
var appShuffleInfo = ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1)
- appShuffleInfo.size() equals 2
+ assert(appShuffleInfo.size() equals 2)
appShuffleInfo.get(app1Id.toString).getAppPathsInfo should be (appPathsInfo1)
appShuffleInfo.get(
app2Attempt1Id.toString).getAppPathsInfo should be (appPathsInfo2Attempt1)
@@ -745,7 +745,7 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers {
assert(!appShuffleInfo.get(app2Attempt1Id.toString).getShuffles.get(2).isFinalized)
var appShuffleInfoAfterReload =
ShuffleTestAccessor.reloadAppShuffleInfo(mergeManager1, mergeManager1DB)
- appShuffleInfoAfterReload.size() equals 2
+ assert(appShuffleInfoAfterReload.size() equals 2)
appShuffleInfoAfterReload.get(app1Id.toString).getAppPathsInfo should be (appPathsInfo1)
appShuffleInfoAfterReload.get(
app2Attempt1Id.toString).getAppPathsInfo should be (appPathsInfo2Attempt1)
@@ -765,12 +765,12 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers {
mergeManager1.applicationRemoved(app1Id.toString, true)
appShuffleInfo = ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1)
- appShuffleInfo.size() equals 1
+ assert(appShuffleInfo.size() equals 1)
assert(!appShuffleInfo.containsKey(app1Id.toString))
assert(appShuffleInfo.get(app2Attempt1Id.toString).getShuffles.get(2).isFinalized)
appShuffleInfoAfterReload =
ShuffleTestAccessor.reloadAppShuffleInfo(mergeManager1, mergeManager1DB)
- appShuffleInfoAfterReload.size() equals 1
+ assert(appShuffleInfoAfterReload.size() equals 1)
assert(!appShuffleInfoAfterReload.containsKey(app1Id.toString))
assert(appShuffleInfoAfterReload.get(app2Attempt1Id.toString).getShuffles.get(2).isFinalized)
@@ -844,7 +844,7 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers {
prepareAppShufflePartition(mergeManager1, partitionId2, 2, "4")
var appShuffleInfo = ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1)
- appShuffleInfo.size() equals 2
+ assert(appShuffleInfo.size() equals 2)
appShuffleInfo.get(app1Id.toString).getAppPathsInfo should be (appPathsInfo1)
appShuffleInfo.get(
app2Id.toString).getAppPathsInfo should be (appPathsInfo2Attempt1)
@@ -867,20 +867,20 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers {
mergeManager1.applicationRemoved(app1Id.toString, true)
appShuffleInfo = ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1)
- appShuffleInfo.size() equals 1
+ assert(appShuffleInfo.size() equals 1)
assert(!appShuffleInfo.containsKey(app1Id.toString))
assert(appShuffleInfo.get(app2Id.toString).getShuffles.get(2).isFinalized)
// Clear the AppsShuffleInfo hashmap and reload the hashmap from DB
appShuffleInfoAfterReload =
ShuffleTestAccessor.reloadAppShuffleInfo(mergeManager1, mergeManager1DB)
- appShuffleInfoAfterReload.size() equals 1
+ assert(appShuffleInfoAfterReload.size() equals 1)
assert(!appShuffleInfoAfterReload.containsKey(app1Id.toString))
assert(appShuffleInfoAfterReload.get(app2Id.toString).getShuffles.get(2).isFinalized)
// Register application app1Id again and reload the DB again
mergeManager1.registerExecutor(app1Id.toString, mergedShuffleInfo1)
appShuffleInfo = ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1)
- appShuffleInfo.size() equals 2
+ assert(appShuffleInfo.size() equals 2)
appShuffleInfo.get(app1Id.toString).getAppPathsInfo should be (appPathsInfo1)
assert(appShuffleInfo.get(app1Id.toString).getShuffles.isEmpty)
assert(appShuffleInfo.get(app2Id.toString).getShuffles.get(2).isFinalized)
@@ -924,7 +924,7 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers {
prepareAppShufflePartition(mergeManager1, partitionId1, 2, "4")
var appShuffleInfo = ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1)
- appShuffleInfo.size() equals 1
+ assert(appShuffleInfo.size() equals 1)
appShuffleInfo.get(
app1Id.toString).getAppPathsInfo should be (appPathsInfo1Attempt1)
assert(!appShuffleInfo.get(app1Id.toString).getShuffles.get(2).isFinalized)
@@ -938,7 +938,7 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers {
prepareAppShufflePartition(mergeManager1, partitionId2, 2, "4")
appShuffleInfo = ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1)
- appShuffleInfo.size() equals 1
+ assert(appShuffleInfo.size() equals 1)
appShuffleInfo.get(
app1Id.toString).getAppPathsInfo should be (appPathsInfo1Attempt2)
assert(!appShuffleInfo.get(app1Id.toString).getShuffles.get(2).isFinalized)
@@ -973,7 +973,7 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers {
val mergeManager3 = s3.shuffleMergeManager.asInstanceOf[RemoteBlockPushResolver]
val mergeManager3DB = ShuffleTestAccessor.mergeManagerDB(mergeManager3)
appShuffleInfo = ShuffleTestAccessor.getAppsShuffleInfo(mergeManager3)
- appShuffleInfo.size() equals 1
+ assert(appShuffleInfo.size() equals 1)
appShuffleInfo.get(
app1Id.toString).getAppPathsInfo should be (appPathsInfo1Attempt2)
assert(appShuffleInfo.get(app1Id.toString).getShuffles.get(2).isFinalized)
@@ -1014,7 +1014,7 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers {
mergeManager1.registerExecutor(app1Id.toString, mergedShuffleInfo1Attempt2)
val appShuffleInfo = ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1)
- appShuffleInfo.size() equals 1
+ assert(appShuffleInfo.size() equals 1)
appShuffleInfo.get(
app1Id.toString).getAppPathsInfo should be (appPathsInfo1Attempt2)
diff --git a/scalastyle-config.xml b/scalastyle-config.xml
index ca3d507ed3a44..f9b260e14c6f9 100644
--- a/scalastyle-config.xml
+++ b/scalastyle-config.xml
@@ -46,7 +46,7 @@ This file is divided into 3 sections:
-
+
+
+ \bFiles\.createParentDirs\b
+ Use createParentDirs of SparkFileUtils or Utils instead.
+
+
+
+ \bFiles\.equal\b
+ Use contentEquals of SparkFileUtils or Utils instead.
+
+
+
+ \bFiles\.toByteArray\b
+ Use java.nio.file.Files.readAllBytes instead.
+
+
+
+ \bFiles\.asByteSource\b
+ Use java.nio.file.Files.newInputStream instead.
+
+
+
+ \bFileUtils\.getTempDirectory\b
+ Use System.getProperty instead.
+
+
+
+ \bFileUtils\.readLines\b
+ Use Files.readAllLines instead.
+
+
+
+ \bFiles\.readLines\b
+ Use Files.readAllLines instead.
+
+
+
+ \bFileUtils\.readFileToString\b
+ Use Files.readString instead.
+
+
+
+ \bFiles\.asCharSource\b
+ Use Files.readString instead.
+
+
+
+ \bFileUtils\.write\b
+ Use Files.writeString instead.
+
+
+
+ \bFiles\.asCharSink\b
+ Use Files.writeString instead.
+
+
+
+ \bFileUtils\.writeLines\b
+ Use Files.write instead.
+
+
+
+ \bFileUtils\.cleanDirectory\b
+ Use cleanDirectory of JavaUtils/SparkFileUtils/Utils
+
+
+
+ \bFileUtils\.deleteDirectory\b
+ Use deleteRecursively of JavaUtils/SparkFileUtils/Utils
+
+
+
+ \bFileUtils\.forceDelete\b
+ Use deleteRecursively of JavaUtils/SparkFileUtils/Utils
+
+
+
+ \bFileUtils\.forceDeleteOnExit\b
+ Use forceDeleteOnExit of JavaUtils/SparkFileUtils/Utils instead.
+
+
+
+ \bFileUtils\.deleteQuietly\b
+ Use deleteQuietly of JavaUtils/SparkFileUtils/Utils
+
+
+
+ \bFileUtils\.readFileToByteArray\b
+ Use java.nio.file.Files.readAllBytes
+
+
+
+ \bFileUtils\.sizeOf(Directory)?\b
+ Use sizeOf of JavaUtils or Utils instead.
+
+
+
+ \bFileUtils\.moveFile\b
+ Use copyFile of JavaUtils/SparkFileUtils/Utils instead.
+
+
+
+ \bFileUtils\.copyURLToFile\b
+ Use copyURLToFile of JavaUtils instead.
+
+
+
+ \bFileUtils\.copyFile\b
+ Use copyFile of SparkFileUtils or Utils instead.
+
+
+
+ \bFileUtils\.copyFileToDirectory\b
+ Use copyFileToDirectory of SparkFileUtils or Utils instead.
+
+
+
+ \bFileUtils\.copyDirectory\b
+ Use copyDirectory of JavaUtils/SparkFileUtils/Utils instead.
+
+
+
+ \bFileUtils\.moveDirectory\b
+ Use copyDirectory of SparkFileUtils or Utils instead.
+
+
+
+ \bFileUtils\.contentEquals\b
+ Use contentEquals of SparkFileUtils or Utils instead.
+
+
org\.apache\.commons\.lang\.Use Commons Lang 3 classes (package org.apache.commons.lang3.*) instead
of Commons Lang 2 (package org.apache.commons.lang.*)
+
+ \bFileUtils\.getFile\b
+ Use getFile of SparkFileUtil or Utils instead.
+
+
+
+ \bFileUtils\.touch\b
+ Use touch of SparkFileUtil or Utils instead.
+
+
+
+ \bFiles\.touch\b
+ Use touch of SparkFileUtil or Utils instead.
+
+
+
+ \bFileUtils\.writeStringToFile\b
+ Use java.nio.file.Files.writeString instead.
+
+
+
+ \bFileUtils\.listFiles\b
+ Use listFiles of SparkFileUtil or Utils instead.
+
+
+
+ org\.apache\.commons\.codec\.binary\.Base64\b
+ Use java.util.Base64 instead
+
+
+
+ org\.apache\.commons\.lang3\..*JavaVersion
+ Use JEP 223 API (java.lang.Runtime.Version) instead of
+ Commons Lang 3 JavaVersion (org.apache.commons.lang3.JavaVersion)
+
+
+
+ org\.apache\.commons\.lang3\.tuple
+ Use org.apache.spark.util.Pair instead
+
+
+
+ org\.apache\.commons\.lang3\.builder\.ToStringBuilder
+ Use String concatenation instead
+
+
+
+ \bStringUtils\.(left|right)Pad\b
+ Use (left|right)Pad of SparkStringUtils or Utils instead
+
+
+
+ \bStringUtils\.split\b
+ Use Utils.stringToSeq instead
+
+
+
+ \bStringUtils\.is(Not)?(Blank|Empty)\b
+ Use Utils.is(Not)?(Blank|Empty) instead
+
+
+
+ \bExceptionUtils\.getRootCause\b
+ Use getRootCause of SparkErrorUtils or Utils instead
+
+
+
+ \bExceptionUtils\.getStackTrace\b
+ Use stackTraceToString of JavaUtils/SparkFileUtils/Utils instead.
+
+
+
+ org\.apache\.commons\.lang3\.Strings\b
+ Use Java String methods instead
+
+
+
+ \bStringUtils\.strip\b
+ Use Utils.strip method instead
+
+
+
+ \bHex\.encodeHexString\b
+ Use java.util.HexFormat instead
+
+
+
+ org\.apache\.commons\.io\.FileUtils\b
+ Use Java API or Spark's JavaUtils/SparkSystemUtils/Utils instead
+
+
+
+ org\.apache\.commons\.lang3\.StringUtils\b
+ Use Java String or Spark's Utils/JavaUtils methods instead
+
+
+
+ org\.apache\.commons\.lang3\.SystemUtils\b
+ Use SparkSystemUtils or Utils instead
+
+
+
+ org\.apache\.commons\.text\.StringSubstitutor\b
+ Use org.apache.spark.StringSubstitutor instead
+
+
+
+ \bStringUtils\.abbreviate\b
+ Use Utils.abbreviate method instead
+
+
+
+ \bStringUtils\.substring\b
+ Use Java String.substring instead.
+
+
- UriBuilder\.fromUri
+ \bUriBuilder\.fromUri\bUse Utils.getUriBuilder instead.
@@ -300,7 +546,7 @@ This file is divided into 3 sections:
- FileSystem.get\([a-zA-Z_$][a-zA-Z_$0-9]*\)
+ \bFileSystem\.get\([a-zA-Z_$][a-zA-Z_$0-9]*\)
- byteCountToDisplaySize
+ \bbyteCountToDisplaySize\bUse Utils.bytesToString instead of byteCountToDisplaySize for consistency.
@@ -470,4 +716,124 @@ This file is divided into 3 sections:
buildConf\("spark.databricks.Use Apache Spark config namespace.
+
+
+ com\.google\.common\.base\.Strings\b
+ Use Java built-in methods or SparkStringUtils instead
+
+
+
+ org\.apache\.hadoop\.io\.IOUtils\b
+ Use org.apache.spark.util.Utils instead.
+
+
+
+ Charset\.defaultCharset
+ Use StandardCharsets.UTF_8 instead.
+
+
+
+ \bIOUtils\.toByteArray\b
+ Use Java readAllBytes instead.
+
+
+
+ \bIOUtils\.closeQuietly\b
+ Use closeQuietly of SparkErrorUtils or Utils instead.
+
+
+
+ \bIOUtils\.copy\b
+ Use Java transferTo instead.
+
+
+
+ \bIOUtils\.toString\b
+ Use toString of SparkStreamUtils or Utils instead.
+
+
+
+ \bCharStreams\.toString\b
+ Use toString of SparkStreamUtils or Utils instead.
+
+
+
+ \bIOUtils\.write\b
+ Use Java `write` instead.
+
+
+
+ \bByteStreams\.read\b
+ Use Java readNBytes instead.
+
+
+
+ \bByteStreams\.copy\b
+ Use Java transferTo instead.
+
+
+
+ \bByteStreams\.skipFully\b
+ Use Java `skipNBytes` instead.
+
+
+
+ \bByteStreams\.readFully\b
+ Use readFully of JavaUtils/SparkStreamUtils/Utils instead.
+
+
+
+ \bByteStreams\.nullOutputStream\b
+ Use OutputStream.nullOutputStream instead.
+
+
+
+ \bImmutableMap\.copyOf\b
+ Use Map.copyOf instead.
+
+
+
+ \bImmutableSet\.of\b
+ Use java.util.Set.of instead.
+
+
+
+ org\.apache\.commons\.collections4\.MapUtils\b
+ Use org.apache.spark.util.collection.Utils instead.
+
+
+
+ com\.google\.common\.io\.Files\b
+ Use Java API or Spark's JavaUtils/SparkFileUtils/Utils instead.
+
+
+
+ com\.google\.common\.base\.Objects\b
+ Use Java APIs (like java.util.Objects) instead.
+
+
+
+ com\.google\.common\.base\.Joiner\b
+ Use Java APIs (like String.join/StringJoiner) instead.
+
+
+
+ com\.google\.common\.io\.BaseEncoding\b
+ Use Java APIs (like java.util.Base64) instead.
+
+
+
+ \bThrowables\.getStackTraceAsString\b
+ Use stackTraceToString of JavaUtils/SparkFileUtils/Utils instead.
+
+
+
+ \bPreconditions\.checkNotNull\b
+ Use requireNonNull of java.util.Objects instead.
+
+
+
+ \bInts\.checkedCast\b
+ Use JavaUtils.checkedCast instead.
+
diff --git a/sql/api/pom.xml b/sql/api/pom.xml
index 86a8b0adaff95..184d39c4b8ea1 100644
--- a/sql/api/pom.xml
+++ b/sql/api/pom.xml
@@ -64,6 +64,10 @@
${project.version}compile
+
+ org.apache.commons
+ commons-lang3
+ org.json4sjson4s-jackson_${scala.binary.version}
diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4
index c698f2696eb41..e402067926f2a 100644
--- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4
+++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4
@@ -507,6 +507,7 @@ WHILE: 'WHILE';
WINDOW: 'WINDOW';
WITH: 'WITH';
WITHIN: 'WITHIN';
+WITHOUT: 'WITHOUT';
YEAR: 'YEAR';
YEARS: 'YEARS';
ZONE: 'ZONE';
@@ -547,13 +548,13 @@ HENT_END: '*/';
QUESTION: '?';
STRING_LITERAL
- : '\'' ( ~('\''|'\\') | ('\\' .) )* '\''
+ : '\'' ( ~('\''|'\\') | ('\\' .) | ('\'' '\'') )* '\''
| 'R\'' (~'\'')* '\''
| 'R"'(~'"')* '"'
;
DOUBLEQUOTED_STRING
- :'"' ( ~('"'|'\\') | ('\\' .) )* '"'
+ :'"' ( ~('"'|'\\') | '""' | ('\\' .) )* '"'
;
// NOTE: If you move a numeric literal, you should modify `ParserUtils.toExprAlias()`
diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4
index 08f222b2f4124..e63a229a32078 100644
--- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4
+++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4
@@ -298,7 +298,9 @@ statement
RETURN (query | expression) #createUserDefinedFunction
| DROP TEMPORARY? FUNCTION (IF EXISTS)? identifierReference #dropFunction
| DECLARE (OR REPLACE)? variable?
- identifierReference dataType? variableDefaultExpression? #createVariable
+ identifierReferences+=identifierReference
+ (COMMA identifierReferences+=identifierReference)*
+ dataType? variableDefaultExpression? #createVariable
| DROP TEMPORARY variable (IF EXISTS)? identifierReference #dropVariable
| EXPLAIN (LOGICAL | FORMATTED | EXTENDED | CODEGEN | COST)?
(statement|setResetStatement) #explain
@@ -374,8 +376,8 @@ createPipelineDatasetHeader
;
streamRelationPrimary
- : STREAM multipartIdentifier tableAlias optionsClause? #streamTableName
- | STREAM LEFT_PAREN multipartIdentifier RIGHT_PAREN tableAlias optionsClause? #streamTableName
+ : STREAM multipartIdentifier optionsClause? tableAlias #streamTableName
+ | STREAM LEFT_PAREN multipartIdentifier RIGHT_PAREN optionsClause? tableAlias #streamTableName
;
setResetStatement
@@ -1208,6 +1210,7 @@ primaryExpression
| constant #constantDefault
| ASTERISK exceptClause? #star
| qualifiedName DOT ASTERISK exceptClause? #star
+ | col=primaryExpression COLON path=semiStructuredExtractionPath #semiStructuredExtract
| LEFT_PAREN namedExpression (COMMA namedExpression)+ RIGHT_PAREN #rowConstructor
| LEFT_PAREN query RIGHT_PAREN #subqueryExpression
| functionName LEFT_PAREN (setQuantifier? argument+=functionArgument
@@ -1230,6 +1233,32 @@ primaryExpression
FROM position=valueExpression (FOR length=valueExpression)? RIGHT_PAREN #overlay
;
+semiStructuredExtractionPath
+ : jsonPathFirstPart (jsonPathParts)*
+ ;
+
+jsonPathIdentifier
+ : identifier
+ | BACKQUOTED_IDENTIFIER
+ ;
+
+jsonPathBracketedIdentifier
+ : LEFT_BRACKET stringLit RIGHT_BRACKET
+ ;
+
+jsonPathFirstPart
+ : jsonPathIdentifier
+ | jsonPathBracketedIdentifier
+ | LEFT_BRACKET INTEGER_VALUE RIGHT_BRACKET
+ ;
+
+jsonPathParts
+ : DOT jsonPathIdentifier
+ | jsonPathBracketedIdentifier
+ | LEFT_BRACKET INTEGER_VALUE RIGHT_BRACKET
+ | LEFT_BRACKET identifier RIGHT_BRACKET
+ ;
+
literalType
: DATE
| TIME
@@ -1313,7 +1342,20 @@ collateClause
: COLLATE collationName=multipartIdentifier
;
-type
+nonTrivialPrimitiveType
+ : STRING collateClause?
+ | (CHARACTER | CHAR) (LEFT_PAREN length=INTEGER_VALUE RIGHT_PAREN)?
+ | VARCHAR (LEFT_PAREN length=INTEGER_VALUE RIGHT_PAREN)?
+ | (DECIMAL | DEC | NUMERIC)
+ (LEFT_PAREN precision=INTEGER_VALUE (COMMA scale=INTEGER_VALUE)? RIGHT_PAREN)?
+ | INTERVAL
+ (fromYearMonth=(YEAR | MONTH) (TO to=MONTH)? |
+ fromDayTime=(DAY | HOUR | MINUTE | SECOND) (TO to=(HOUR | MINUTE | SECOND))?)?
+ | TIMESTAMP (WITHOUT TIME ZONE)?
+ | TIME (LEFT_PAREN precision=INTEGER_VALUE RIGHT_PAREN)? (WITHOUT TIME ZONE)?
+ ;
+
+trivialPrimitiveType
: BOOLEAN
| TINYINT | BYTE
| SMALLINT | SHORT
@@ -1322,29 +1364,23 @@ type
| FLOAT | REAL
| DOUBLE
| DATE
- | TIME
- | TIMESTAMP | TIMESTAMP_NTZ | TIMESTAMP_LTZ
- | STRING collateClause?
- | CHARACTER | CHAR
- | VARCHAR
+ | TIMESTAMP_LTZ | TIMESTAMP_NTZ
| BINARY
- | DECIMAL | DEC | NUMERIC
| VOID
- | INTERVAL
| VARIANT
- | ARRAY | STRUCT | MAP
- | unsupportedType=identifier
+ ;
+
+primitiveType
+ : nonTrivialPrimitiveType
+ | trivialPrimitiveType
+ | unsupportedType=identifier (LEFT_PAREN INTEGER_VALUE(COMMA INTEGER_VALUE)* RIGHT_PAREN)?
;
dataType
- : complex=ARRAY LT dataType GT #complexDataType
- | complex=MAP LT dataType COMMA dataType GT #complexDataType
- | complex=STRUCT (LT complexColTypeList? GT | NEQ) #complexDataType
- | INTERVAL from=(YEAR | MONTH) (TO to=MONTH)? #yearMonthIntervalDataType
- | INTERVAL from=(DAY | HOUR | MINUTE | SECOND)
- (TO to=(HOUR | MINUTE | SECOND))? #dayTimeIntervalDataType
- | type (LEFT_PAREN INTEGER_VALUE
- (COMMA INTEGER_VALUE)* RIGHT_PAREN)? #primitiveDataType
+ : complex=ARRAY (LT dataType GT)? #complexDataType
+ | complex=MAP (LT dataType COMMA dataType GT)? #complexDataType
+ | complex=STRUCT ((LT complexColTypeList? GT) | NEQ)? #complexDataType
+ | primitiveType #primitiveDataType
;
qualifiedColTypeWithPositionList
@@ -2013,6 +2049,7 @@ ansiNonReserved
| WEEKS
| WHILE
| WINDOW
+ | WITHOUT
| YEAR
| YEARS
| ZONE
@@ -2423,6 +2460,7 @@ nonReserved
| WINDOW
| WITH
| WITHIN
+ | WITHOUT
| YEAR
| YEARS
| ZONE
diff --git a/sql/api/src/main/java/org/apache/spark/sql/connector/catalog/IdentifierImpl.java b/sql/api/src/main/java/org/apache/spark/sql/connector/catalog/IdentifierImpl.java
index ba883b8042c4d..d209a540823c2 100644
--- a/sql/api/src/main/java/org/apache/spark/sql/connector/catalog/IdentifierImpl.java
+++ b/sql/api/src/main/java/org/apache/spark/sql/connector/catalog/IdentifierImpl.java
@@ -17,8 +17,6 @@
package org.apache.spark.sql.connector.catalog;
-import org.apache.arrow.util.Preconditions;
-
import java.util.Arrays;
import java.util.Objects;
import java.util.StringJoiner;
@@ -36,10 +34,8 @@ class IdentifierImpl implements Identifier {
private String name;
IdentifierImpl(String[] namespace, String name) {
- Preconditions.checkNotNull(namespace, "Identifier namespace cannot be null");
- Preconditions.checkNotNull(name, "Identifier name cannot be null");
- this.namespace = namespace;
- this.name = name;
+ this.namespace = Objects.requireNonNull(namespace, "Identifier namespace cannot be null");
+ this.name = Objects.requireNonNull(name, "Identifier name cannot be null");
}
@Override
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/Artifact.scala b/sql/api/src/main/scala/org/apache/spark/sql/Artifact.scala
index 7e020df06fe47..40c7e532c27db 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/Artifact.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/Artifact.scala
@@ -21,12 +21,10 @@ import java.io.{ByteArrayInputStream, InputStream, PrintStream}
import java.net.URI
import java.nio.file.{Files, Path, Paths}
-import org.apache.commons.lang3.StringUtils
-
import org.apache.spark.sql.Artifact.LocalData
import org.apache.spark.sql.util.ArtifactUtils
+import org.apache.spark.util.{MavenUtils, SparkStringUtils}
import org.apache.spark.util.ArrayImplicits._
-import org.apache.spark.util.MavenUtils
private[sql] class Artifact private (val path: Path, val storage: LocalData) {
require(!path.isAbsolute, s"Bad path: $path")
@@ -104,7 +102,7 @@ private[sql] object Artifact {
val (transitive, exclusions, repos) = MavenUtils.parseQueryParams(uri)
val exclusionsList: Seq[String] =
- if (!StringUtils.isBlank(exclusions)) {
+ if (!SparkStringUtils.isBlank(exclusions)) {
exclusions.split(",").toImmutableArraySeq
} else {
Nil
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/Column.scala b/sql/api/src/main/scala/org/apache/spark/sql/Column.scala
index 88d597fdfbb73..316b629df4c14 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/Column.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/Column.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql
import scala.jdk.CollectionConverters._
import org.apache.spark.annotation.Stable
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{LEFT_EXPR, RIGHT_EXPR}
import org.apache.spark.sql.catalyst.parser.DataTypeParser
import org.apache.spark.sql.catalyst.trees.CurrentOrigin.withOrigin
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/Encoders.scala b/sql/api/src/main/scala/org/apache/spark/sql/Encoders.scala
index 94a627fd17a64..cb1402e1b0f4a 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/Encoders.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/Encoders.scala
@@ -368,4 +368,11 @@ object Encoders {
*/
def scalaBoolean: Encoder[Boolean] = PrimitiveBooleanEncoder
+ /**
+ * An encoder for UserDefinedType.
+ * @since 4.1.0
+ */
+ def udt[T >: Null](tpe: UserDefinedType[T]): Encoder[T] = {
+ UDTEncoder(tpe)
+ }
}
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/AgnosticEncoder.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/AgnosticEncoder.scala
index 1dd939131ab96..0c5295176608f 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/AgnosticEncoder.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/AgnosticEncoder.scala
@@ -182,6 +182,12 @@ object AgnosticEncoders {
override def clsTag: ClassTag[E] = ClassTag(udt.userClass)
}
+ object UDTEncoder {
+ def apply[E >: Null](udt: UserDefinedType[E]): UDTEncoder[E] = {
+ new UDTEncoder(udt, udt.getClass.asInstanceOf[Class[_ <: UserDefinedType[_]]])
+ }
+ }
+
// Enums are special leafs because we need to capture the class.
protected abstract class EnumEncoder[E] extends AgnosticEncoder[E] {
override def isPrimitive: Boolean = false
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
index d5692bb85c4e9..620278c66d21d 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
@@ -22,7 +22,7 @@ import scala.reflect.classTag
import org.apache.spark.sql.{AnalysisException, Row}
import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{BinaryEncoder, BoxedBooleanEncoder, BoxedByteEncoder, BoxedDoubleEncoder, BoxedFloatEncoder, BoxedIntEncoder, BoxedLongEncoder, BoxedShortEncoder, CalendarIntervalEncoder, CharEncoder, DateEncoder, DayTimeIntervalEncoder, EncoderField, InstantEncoder, IterableEncoder, JavaDecimalEncoder, LocalDateEncoder, LocalDateTimeEncoder, LocalTimeEncoder, MapEncoder, NullEncoder, RowEncoder => AgnosticRowEncoder, StringEncoder, TimestampEncoder, UDTEncoder, VarcharEncoder, VariantEncoder, YearMonthIntervalEncoder}
-import org.apache.spark.sql.errors.{DataTypeErrorsBase, ExecutionErrors}
+import org.apache.spark.sql.errors.DataTypeErrorsBase
import org.apache.spark.sql.internal.SqlApiConf
import org.apache.spark.sql.types._
import org.apache.spark.util.ArrayImplicits._
@@ -99,16 +99,7 @@ object RowEncoder extends DataTypeErrorsBase {
case p: PythonUserDefinedType =>
// TODO check if this works.
encoderForDataType(p.sqlType, lenient)
- case udt: UserDefinedType[_] =>
- val annotation = udt.userClass.getAnnotation(classOf[SQLUserDefinedType])
- val udtClass: Class[_] = if (annotation != null) {
- annotation.udt()
- } else {
- UDTRegistration.getUDTFor(udt.userClass.getName).getOrElse {
- throw ExecutionErrors.userDefinedTypeNotAnnotatedAndRegisteredError(udt)
- }
- }
- UDTEncoder(udt, udtClass.asInstanceOf[Class[_ <: UserDefinedType[_]]])
+ case udt: UserDefinedType[_] => UDTEncoder(udt, udt.getClass)
case ArrayType(elementType, containsNull) =>
IterableEncoder(
classTag[mutable.ArraySeq[_]],
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala
index bf9a250d6499e..beb7061a841a8 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala
@@ -69,54 +69,85 @@ class DataTypeAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] {
* Resolve/create a primitive type.
*/
override def visitPrimitiveDataType(ctx: PrimitiveDataTypeContext): DataType = withOrigin(ctx) {
- val typeCtx = ctx.`type`
- (typeCtx.start.getType, ctx.INTEGER_VALUE().asScala.toList) match {
- case (BOOLEAN, Nil) => BooleanType
- case (TINYINT | BYTE, Nil) => ByteType
- case (SMALLINT | SHORT, Nil) => ShortType
- case (INT | INTEGER, Nil) => IntegerType
- case (BIGINT | LONG, Nil) => LongType
- case (FLOAT | REAL, Nil) => FloatType
- case (DOUBLE, Nil) => DoubleType
- case (DATE, Nil) => DateType
- case (TIME, Nil) => TimeType(TimeType.MICROS_PRECISION)
- case (TIME, precision :: Nil) => TimeType(precision.getText.toInt)
- case (TIMESTAMP, Nil) => SqlApiConf.get.timestampType
- case (TIMESTAMP_NTZ, Nil) => TimestampNTZType
- case (TIMESTAMP_LTZ, Nil) => TimestampType
- case (STRING, Nil) =>
- typeCtx.children.asScala.toSeq match {
- case Seq(_) => StringType
- case Seq(_, ctx: CollateClauseContext) =>
- val collationNameParts = visitCollateClause(ctx).toArray
- val collationId = CollationFactory.collationNameToId(
- CollationFactory.resolveFullyQualifiedName(collationNameParts))
- StringType(collationId)
- }
- case (CHARACTER | CHAR, length :: Nil) => CharType(length.getText.toInt)
- case (VARCHAR, length :: Nil) => VarcharType(length.getText.toInt)
- case (BINARY, Nil) => BinaryType
- case (DECIMAL | DEC | NUMERIC, Nil) => DecimalType.USER_DEFAULT
- case (DECIMAL | DEC | NUMERIC, precision :: Nil) =>
- DecimalType(precision.getText.toInt, 0)
- case (DECIMAL | DEC | NUMERIC, precision :: scale :: Nil) =>
- DecimalType(precision.getText.toInt, scale.getText.toInt)
- case (VOID, Nil) => NullType
- case (INTERVAL, Nil) => CalendarIntervalType
- case (VARIANT, Nil) => VariantType
- case (CHARACTER | CHAR | VARCHAR, Nil) =>
- throw QueryParsingErrors.charTypeMissingLengthError(ctx.`type`.getText, ctx)
- case (ARRAY | STRUCT | MAP, Nil) =>
- throw QueryParsingErrors.nestedTypeMissingElementTypeError(ctx.`type`.getText, ctx)
- case (_, params) =>
- val badType = ctx.`type`.getText
- val dtStr = if (params.nonEmpty) s"$badType(${params.mkString(",")})" else badType
- throw QueryParsingErrors.dataTypeUnsupportedError(dtStr, ctx)
+ val typeCtx = ctx.primitiveType
+ if (typeCtx.nonTrivialPrimitiveType != null) {
+ // This is a primitive type with parameters, e.g. VARCHAR(10), DECIMAL(10, 2), etc.
+ val currentCtx = typeCtx.nonTrivialPrimitiveType
+ currentCtx.start.getType match {
+ case STRING =>
+ currentCtx.children.asScala.toSeq match {
+ case Seq(_) => StringType
+ case Seq(_, ctx: CollateClauseContext) =>
+ val collationNameParts = visitCollateClause(ctx).toArray
+ val collationId = CollationFactory.collationNameToId(
+ CollationFactory.resolveFullyQualifiedName(collationNameParts))
+ StringType(collationId)
+ }
+ case CHARACTER | CHAR =>
+ if (currentCtx.length == null) {
+ throw QueryParsingErrors.charVarcharTypeMissingLengthError(typeCtx.getText, ctx)
+ } else CharType(currentCtx.length.getText.toInt)
+ case VARCHAR =>
+ if (currentCtx.length == null) {
+ throw QueryParsingErrors.charVarcharTypeMissingLengthError(typeCtx.getText, ctx)
+ } else VarcharType(currentCtx.length.getText.toInt)
+ case DECIMAL | DEC | NUMERIC =>
+ if (currentCtx.precision == null) {
+ DecimalType.USER_DEFAULT
+ } else if (currentCtx.scale == null) {
+ DecimalType(currentCtx.precision.getText.toInt, 0)
+ } else {
+ DecimalType(currentCtx.precision.getText.toInt, currentCtx.scale.getText.toInt)
+ }
+ case INTERVAL =>
+ if (currentCtx.fromDayTime != null) {
+ visitDayTimeIntervalDataType(currentCtx)
+ } else if (currentCtx.fromYearMonth != null) {
+ visitYearMonthIntervalDataType(currentCtx)
+ } else {
+ CalendarIntervalType
+ }
+ case TIMESTAMP =>
+ if (currentCtx.WITHOUT() == null) {
+ SqlApiConf.get.timestampType
+ } else TimestampNTZType
+ case TIME =>
+ val precision = if (currentCtx.precision == null) {
+ TimeType.DEFAULT_PRECISION
+ } else {
+ currentCtx.precision.getText.toInt
+ }
+ TimeType(precision)
+ }
+ } else if (typeCtx.trivialPrimitiveType != null) {
+ // This is a primitive type without parameters, e.g. BOOLEAN, TINYINT, etc.
+ typeCtx.trivialPrimitiveType.start.getType match {
+ case BOOLEAN => BooleanType
+ case TINYINT | BYTE => ByteType
+ case SMALLINT | SHORT => ShortType
+ case INT | INTEGER => IntegerType
+ case BIGINT | LONG => LongType
+ case FLOAT | REAL => FloatType
+ case DOUBLE => DoubleType
+ case DATE => DateType
+ case TIMESTAMP_LTZ => TimestampType
+ case TIMESTAMP_NTZ => TimestampNTZType
+ case BINARY => BinaryType
+ case VOID => NullType
+ case VARIANT => VariantType
+ }
+ } else {
+ val badType = typeCtx.unsupportedType.getText
+ val params = typeCtx.INTEGER_VALUE().asScala.toList
+ val dtStr =
+ if (params.nonEmpty) s"$badType(${params.mkString(",")})"
+ else badType
+ throw QueryParsingErrors.dataTypeUnsupportedError(dtStr, ctx)
}
}
- override def visitYearMonthIntervalDataType(ctx: YearMonthIntervalDataTypeContext): DataType = {
- val startStr = ctx.from.getText.toLowerCase(Locale.ROOT)
+ private def visitYearMonthIntervalDataType(ctx: NonTrivialPrimitiveTypeContext): DataType = {
+ val startStr = ctx.fromYearMonth.getText.toLowerCase(Locale.ROOT)
val start = YearMonthIntervalType.stringToField(startStr)
if (ctx.to != null) {
val endStr = ctx.to.getText.toLowerCase(Locale.ROOT)
@@ -130,8 +161,8 @@ class DataTypeAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] {
}
}
- override def visitDayTimeIntervalDataType(ctx: DayTimeIntervalDataTypeContext): DataType = {
- val startStr = ctx.from.getText.toLowerCase(Locale.ROOT)
+ private def visitDayTimeIntervalDataType(ctx: NonTrivialPrimitiveTypeContext): DataType = {
+ val startStr = ctx.fromDayTime.getText.toLowerCase(Locale.ROOT)
val start = DayTimeIntervalType.stringToField(startStr)
if (ctx.to != null) {
val endStr = ctx.to.getText.toLowerCase(Locale.ROOT)
@@ -149,6 +180,9 @@ class DataTypeAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] {
* Create a complex DataType. Arrays, Maps and Structures are supported.
*/
override def visitComplexDataType(ctx: ComplexDataTypeContext): DataType = withOrigin(ctx) {
+ if (ctx.LT() == null && ctx.NEQ() == null) {
+ throw QueryParsingErrors.nestedTypeMissingElementTypeError(ctx.getText, ctx)
+ }
ctx.complex.getType match {
case SqlBaseParser.ARRAY =>
ArrayType(typedVisit(ctx.dataType(0)))
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala
index 28fccd2092b34..f149fd5f165f7 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala
@@ -16,15 +16,18 @@
*/
package org.apache.spark.sql.catalyst.parser
+import java.util.concurrent.atomic.AtomicReference
+
import scala.jdk.CollectionConverters._
import org.antlr.v4.runtime._
-import org.antlr.v4.runtime.atn.PredictionMode
+import org.antlr.v4.runtime.atn.{ATN, ParserATNSimulator, PredictionContextCache, PredictionMode}
+import org.antlr.v4.runtime.dfa.DFA
import org.antlr.v4.runtime.misc.{Interval, ParseCancellationException}
import org.antlr.v4.runtime.tree.TerminalNodeImpl
import org.apache.spark.{QueryContext, SparkException, SparkThrowable, SparkThrowableHelper}
-import org.apache.spark.internal.Logging
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, Origin, SQLQueryContext, WithOrigin}
import org.apache.spark.sql.catalyst.util.SparkParserUtils
@@ -62,6 +65,7 @@ abstract class AbstractParser extends DataTypeParserInterface with Logging {
val tokenStream = new CommonTokenStream(lexer)
val parser = new SqlBaseParser(tokenStream)
+ if (conf.manageParserCaches) AbstractParser.installCaches(parser)
parser.addParseListener(PostProcessor)
parser.addParseListener(UnclosedCommentProcessor(command, tokenStream))
parser.removeErrorListeners()
@@ -102,6 +106,18 @@ abstract class AbstractParser extends DataTypeParserInterface with Logging {
errorClass = e.getCondition,
messageParameters = e.getMessageParameters.asScala.toMap,
queryContext = e.getQueryContext)
+ } finally {
+ // Antlr4 uses caches to make parsing faster but its caches are unbounded and never purged,
+ // which can cause OOMs when parsing a huge number of SQL queries. Clearing these caches too
+ // often will slow down parsing and cause performance regressions, but will prevent OOMs
+ // caused by the parser cache. We use a heuristic and clear the cache if the number of states
+ // in the DFA cache has exceeded the threshold
+ // configured by `spark.sql.parser.parserDfaCacheFlushThreshold`. These states generally
+ // represent the bulk of the memory consumed by the parser, and the size of a single state
+ // is approximately `BYTES_PER_DFA_STATE` bytes.
+ //
+ // Negative values mean we should never clear the cache
+ AbstractParser.maybeClearParserCaches(parser, conf)
}
}
@@ -439,3 +455,109 @@ case class UnclosedCommentProcessor(command: String, tokenStream: CommonTokenStr
object DataTypeParser extends AbstractParser {
override protected def astBuilder: DataTypeAstBuilder = new DataTypeAstBuilder
}
+
+object AbstractParser extends Logging {
+ // Approximation based on experiments. Used to estimate the size of the DFA cache for the
+ // `parserDfaCacheFlushRatio` threshold.
+ final val BYTES_PER_DFA_STATE = 9700
+
+ private val DRIVER_MEMORY = Runtime.getRuntime.maxMemory()
+
+ private case class AntlrCaches(atn: ATN) {
+ private[parser] val predictionContextCache: PredictionContextCache =
+ new PredictionContextCache
+ private[parser] val decisionToDFACache: Array[DFA] = AntlrCaches.makeDecisionToDFACache(atn)
+
+ def installManagedParserCaches(parser: SqlBaseParser): Unit = {
+ parser.setInterpreter(
+ new ParserATNSimulator(parser, atn, decisionToDFACache, predictionContextCache))
+ }
+ }
+
+ private object AntlrCaches {
+ private def makeDecisionToDFACache(atn: ATN): Array[DFA] = {
+ val decisionToDFA = new Array[DFA](atn.getNumberOfDecisions)
+ for (i <- 0 until atn.getNumberOfDecisions) {
+ decisionToDFA(i) = new DFA(atn.getDecisionState(i), i)
+ }
+ decisionToDFA
+ }
+ }
+
+ private val parserCaches = new AtomicReference[AntlrCaches](AntlrCaches(SqlBaseParser._ATN))
+
+ private var numDFACacheStates: Long = 0
+ def getDFACacheNumStates: Long = numDFACacheStates
+
+ /**
+ * Returns the number of DFA states in the DFA cache.
+ *
+ * DFA states empirically consume about `BYTES_PER_DFA_STATE` bytes of memory each.
+ */
+ private def computeDFACacheNumStates: Long = {
+ parserCaches.get().decisionToDFACache.map(_.states.size).sum
+ }
+
+ /**
+ * Install the managed parser caches into the given parser. Configuring the parser to use the
+ * managed `AntlrCaches` enables us to manage the size of the cache and clear it when required
+ * as the parser caches are unbounded by default.
+ *
+ * This method should be called before parsing any input.
+ */
+ private[parser] def installCaches(parser: SqlBaseParser): Unit = {
+ parserCaches.get().installManagedParserCaches(parser)
+ }
+
+ /**
+ * Drop the existing parser caches and create a new one.
+ *
+ * ANTLR retains caches in its parser that are never released. This speeds up parsing of future
+ * input, but it can consume a lot of memory depending on the input seen so far.
+ *
+ * This method provides a mechanism to free the retained caches, which can be useful after
+ * parsing very large SQL inputs, especially if those large inputs are unlikely to be similar to
+ * future inputs seen by the driver.
+ */
+ private[parser] def clearParserCaches(parser: SqlBaseParser): Unit = {
+ parserCaches.set(AntlrCaches(SqlBaseParser._ATN))
+ logInfo(log"ANTLR parser caches cleared")
+ numDFACacheStates = 0
+ installCaches(parser)
+ }
+
+ /**
+ * Check cache size and config values to determine if we should clear the parser caches. Also
+ * logs the current cache size and the delta since the last check. This method should be called
+ * after parsing each input.
+ */
+ private[parser] def maybeClearParserCaches(parser: SqlBaseParser, conf: SqlApiConf): Unit = {
+ if (!conf.manageParserCaches) {
+ return
+ }
+
+ val numDFACacheStatesCurrent: Long = computeDFACacheNumStates
+ val numDFACacheStatesDelta = numDFACacheStatesCurrent - numDFACacheStates
+ numDFACacheStates = numDFACacheStatesCurrent
+ logInfo(
+ log"EXPERIMENTAL: Query cached " +
+ log"${MDC(LogKeys.ANTLR_DFA_CACHE_DELTA, numDFACacheStatesDelta)} " +
+ log"DFA states in the parser. Total cached DFA states: " +
+ log"${MDC(LogKeys.ANTLR_DFA_CACHE_SIZE, numDFACacheStatesCurrent)}." +
+ log"Driver memory: ${MDC(LogKeys.DRIVER_JVM_MEMORY, DRIVER_MEMORY)}.")
+
+ val staticThresholdExceeded = 0 <= conf.parserDfaCacheFlushThreshold &&
+ conf.parserDfaCacheFlushThreshold <= numDFACacheStatesCurrent
+
+ val estCacheBytes: Long = numDFACacheStatesCurrent * BYTES_PER_DFA_STATE
+ if (estCacheBytes < 0) {
+ logWarning(log"Estimated cache size is negative, likely due to an integer overflow.")
+ }
+ val dynamicThresholdExceeded = 0 <= conf.parserDfaCacheFlushRatio &&
+ conf.parserDfaCacheFlushRatio * DRIVER_MEMORY / 100 <= estCacheBytes
+
+ if (staticThresholdExceeded || dynamicThresholdExceeded) {
+ AbstractParser.clearParserCaches(parser)
+ }
+ }
+}
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkDateTimeUtils.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkDateTimeUtils.scala
index 6a51799e1132d..9ce9d14ed3161 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkDateTimeUtils.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkDateTimeUtils.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.util
import java.lang.invoke.{MethodHandles, MethodType}
import java.sql.{Date, Timestamp}
import java.time.{Instant, LocalDate, LocalDateTime, LocalTime, ZonedDateTime, ZoneId, ZoneOffset}
-import java.time.temporal.ChronoField.MICRO_OF_DAY
+import java.time.temporal.ChronoField.NANO_OF_DAY
import java.util.TimeZone
import java.util.concurrent.TimeUnit.{MICROSECONDS, NANOSECONDS}
import java.util.regex.Pattern
@@ -83,6 +83,12 @@ trait SparkDateTimeUtils {
case ldt: LocalDateTime => localDateTimeToMicros(ldt)
}
+ /**
+ * Converts the time to microseconds since midnight. In Spark time values have nanoseconds
+ * precision, so this conversion is lossy.
+ */
+ def nanosToMicros(nanos: Long): Long = Math.floorDiv(nanos, MICROS_PER_MILLIS)
+
/**
* Converts the timestamp to milliseconds since epoch. In Spark timestamp values have
* microseconds precision, so this conversion is lossy.
@@ -101,6 +107,11 @@ trait SparkDateTimeUtils {
Math.multiplyExact(millis, MICROS_PER_MILLIS)
}
+ /**
+ * Converts microseconds since the midnight to nanoseconds.
+ */
+ def microsToNanos(micros: Long): Long = Math.multiplyExact(micros, NANOS_PER_MICROS)
+
// See issue SPARK-35679
// min second cause overflow in instant to micro
private val MIN_SECONDS = Math.floorDiv(Long.MinValue, MICROS_PER_SECOND)
@@ -134,42 +145,42 @@ trait SparkDateTimeUtils {
}
/**
- * Gets the number of microseconds since midnight using the given time zone.
+ * Gets the number of nanoseconds since midnight using the given time zone.
*/
- def instantToMicrosOfDay(instant: Instant, timezone: String): Long = {
- instantToMicrosOfDay(instant, getZoneId(timezone))
+ def instantToNanosOfDay(instant: Instant, timezone: String): Long = {
+ instantToNanosOfDay(instant, getZoneId(timezone))
}
/**
- * Gets the number of microseconds since midnight using the given time zone.
+ * Gets the number of nanoseconds since midnight using the given time zone.
*/
- def instantToMicrosOfDay(instant: Instant, zoneId: ZoneId): Long = {
+ def instantToNanosOfDay(instant: Instant, zoneId: ZoneId): Long = {
val localDateTime = LocalDateTime.ofInstant(instant, zoneId)
- localDateTime.toLocalTime.getLong(MICRO_OF_DAY)
+ localDateTime.toLocalTime.getLong(NANO_OF_DAY)
}
/**
- * Truncates a time value (in microseconds) to the specified fractional precision `p`.
+ * Truncates a time value (in nanoseconds) to the specified fractional precision `p`.
*
* For example, if `p = 3`, we keep millisecond resolution and discard any digits beyond the
- * thousand-microsecond place. So a value like `123456` microseconds (12:34:56.123456) becomes
+ * thousand-nanosecond place. So a value like `123456` microseconds (12:34:56.123456) becomes
* `123000` microseconds (12:34:56.123).
*
- * @param micros
- * The original time in microseconds.
+ * @param nanos
+ * The original time in nanoseconds.
* @param p
* The fractional second precision (range 0 to 6).
* @return
- * The truncated microsecond value, preserving only `p` fractional digits.
+ * The truncated nanosecond value, preserving only `p` fractional digits.
*/
- def truncateTimeMicrosToPrecision(micros: Long, p: Int): Long = {
+ def truncateTimeToPrecision(nanos: Long, p: Int): Long = {
assert(
- p >= TimeType.MIN_PRECISION && p <= TimeType.MICROS_PRECISION,
+ TimeType.MIN_PRECISION <= p && p <= TimeType.MAX_PRECISION,
s"Fractional second precision $p out" +
- s" of range [${TimeType.MIN_PRECISION}..${TimeType.MICROS_PRECISION}].")
- val scale = TimeType.MICROS_PRECISION - p
+ s" of range [${TimeType.MIN_PRECISION}..${TimeType.MAX_PRECISION}].")
+ val scale = TimeType.NANOS_PRECISION - p
val factor = math.pow(10, scale).toLong
- (micros / factor) * factor
+ (nanos / factor) * factor
}
/**
@@ -225,17 +236,15 @@ trait SparkDateTimeUtils {
}
/**
- * Converts the local time to the number of microseconds within the day, from 0 to (24 * 60 * 60
- * * 1000000) - 1.
+ * Converts the local time to the number of nanoseconds within the day, from 0 to (24 * 60 * 60
+ * * 1000 * 1000 * 1000) - 1.
*/
- def localTimeToMicros(localTime: LocalTime): Long = localTime.getLong(MICRO_OF_DAY)
+ def localTimeToNanos(localTime: LocalTime): Long = localTime.getLong(NANO_OF_DAY)
/**
- * Converts the number of microseconds within the day to the local time.
+ * Converts the number of nanoseconds within the day to the local time.
*/
- def microsToLocalTime(micros: Long): LocalTime = {
- LocalTime.ofNanoOfDay(Math.multiplyExact(micros, NANOS_PER_MICROS))
- }
+ def nanosToLocalTime(nanos: Long): LocalTime = LocalTime.ofNanoOfDay(nanos)
/**
* Converts a local date at the default JVM time zone to the number of days since 1970-01-01 in
@@ -708,15 +717,63 @@ trait SparkDateTimeUtils {
*/
def stringToTime(s: UTF8String): Option[Long] = {
try {
- val (segments, zoneIdOpt, justTime) = parseTimestampString(s)
+ // Check for the AM/PM suffix.
+ val trimmed = s.trimRight
+ val numChars = trimmed.numChars()
+ var (isAM, isPM, hasSuffix) = (false, false, false)
+ if (numChars > 2) {
+ val lc = trimmed.getChar(numChars - 1)
+ if (lc == 'M' || lc == 'm') {
+ val slc = trimmed.getChar(numChars - 2)
+ isAM = slc == 'A' || slc == 'a'
+ isPM = slc == 'P' || slc == 'p'
+ hasSuffix = isAM || isPM
+ }
+ }
+ val timeString = if (hasSuffix) {
+ trimmed.substring(0, numChars - 2)
+ } else {
+ trimmed
+ }
+
+ val (segments, zoneIdOpt, justTime) = parseTimestampString(timeString)
+
// If the input string can't be parsed as a time, or it contains not only
// the time part or has time zone information, return None.
if (segments.isEmpty || !justTime || zoneIdOpt.isDefined) {
return None
}
- val nanoseconds = MICROSECONDS.toNanos(segments(6))
- val localTime = LocalTime.of(segments(3), segments(4), segments(5), nanoseconds.toInt)
- Some(localTimeToMicros(localTime))
+
+ // Unpack the segments.
+ var (hr, min, sec, ms) = (segments(3), segments(4), segments(5), segments(6))
+
+ // Handle AM/PM conversion in separate cases.
+ if (!hasSuffix) {
+ // For 24-hour format, validate hour range: 0-23.
+ if (hr < 0 || hr > 23) {
+ return None
+ }
+ } else {
+ // For 12-hour format, validate hour range: 1-12.
+ if (hr < 1 || hr > 12) {
+ return None
+ }
+ // For 12-hour format, convert to 24-hour format.
+ if (isAM) {
+ // AM: 12:xx:xx becomes 00:xx:xx, 1-11:xx:xx stays the same.
+ if (hr == 12) {
+ hr = 0
+ }
+ } else {
+ // PM: 12:xx:xx stays 12:xx:xx, 1-11:xx:xx becomes 13-23:xx:xx.
+ if (hr != 12) {
+ hr += 12
+ }
+ }
+ }
+
+ val localTime = LocalTime.of(hr, min, sec, MICROSECONDS.toNanos(ms).toInt)
+ Some(localTimeToNanos(localTime))
} catch {
case NonFatal(_) => None
}
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkParserUtils.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkParserUtils.scala
index 9c9e623e03395..52788262ce169 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkParserUtils.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkParserUtils.scala
@@ -26,8 +26,19 @@ import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, Origin}
trait SparkParserUtils {
- /** Unescape backslash-escaped string enclosed by quotes. */
- def unescapeSQLString(b: String): String = {
+ /**
+ * Unescape escaped string enclosed by quotes, with support for:
+ * 1. Double-quote escaping (`""`, `''`)
+ * 2. Traditional backslash escaping (\n, \t, \", etc.)
+ *
+ * @param b
+ * The input string
+ * @param ignoreQuoteQuote
+ * If true, consecutive quotes (`''` or `""`) are treated as string concatenation and will be
+ * removed directly (e.g., `'a''b'` → `ab`). If false, they are treated as escape sequences
+ * (e.g., `'a''b'` → `a'b`). Default is false (standard SQL escaping).
+ */
+ def unescapeSQLString(b: String, ignoreQuoteQuote: Boolean = false): String = {
def appendEscapedChar(n: Char, sb: JStringBuilder): Unit = {
n match {
case '0' => sb.append('\u0000')
@@ -71,10 +82,20 @@ trait SparkParserUtils {
firstChar == 'r' || firstChar == 'R'
}
+ val isDoubleQuotedString = {
+ b.charAt(0) == '"'
+ }
+
+ val isSingleQuotedString = {
+ b.charAt(0) == '\''
+ }
+
if (isRawString) {
// Skip the 'r' or 'R' and the first and last quotations enclosing the string literal.
b.substring(2, b.length - 1)
- } else if (b.indexOf('\\') == -1) {
+ } else if (b.indexOf('\\') == -1 &&
+ (!isDoubleQuotedString || b.indexOf("\"\"") == -1) &&
+ (!isSingleQuotedString || b.indexOf("''") == -1)) {
// Fast path for the common case where the string has no escaped characters,
// in which case we just skip the first and last quotations enclosing the string literal.
b.substring(1, b.length - 1)
@@ -85,7 +106,19 @@ trait SparkParserUtils {
val length = b.length - 1
while (i < length) {
val c = b.charAt(i)
- if (c != '\\' || i + 1 == length) {
+ // First check for double-quote escaping (`""`, `''`)
+ if (isDoubleQuotedString && c == '"' && i + 1 < length && b.charAt(i + 1) == '"') {
+ if (!ignoreQuoteQuote) {
+ sb.append('"')
+ }
+ i += 2
+ } else if (isSingleQuotedString && c == '\'' && i + 1 < length && b.charAt(
+ i + 1) == '\'') {
+ if (!ignoreQuoteQuote) {
+ sb.append('\'')
+ }
+ i += 2
+ } else if (c != '\\' || i + 1 == length) {
// Either a regular character or a backslash at the end of the string:
sb.append(c)
i += 1
@@ -138,6 +171,9 @@ trait SparkParserUtils {
/** Convert a string token into a string. */
def string(token: Token): String = unescapeSQLString(token.getText)
+ /** Convert a string token into a string and remove `""` and `''`. */
+ def stringIgnoreQuoteQuote(token: Token): String = unescapeSQLString(token.getText, true)
+
/** Convert a string node into a string. */
def string(node: TerminalNode): String = unescapeSQLString(node.getText)
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/StringConcat.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/StringConcat.scala
new file mode 100644
index 0000000000000..006c413d941cc
--- /dev/null
+++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/StringConcat.scala
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.catalyst.util
+
+import org.apache.spark.unsafe.array.ByteArrayUtils
+
+/**
+ * Concatenation of sequence of strings to final string with cheap append method and one memory
+ * allocation for the final string. Can also bound the final size of the string.
+ */
+class StringConcat(val maxLength: Int = ByteArrayUtils.MAX_ROUNDED_ARRAY_LENGTH) {
+ protected val strings = new java.util.ArrayList[String]
+ protected var length: Int = 0
+
+ def atLimit: Boolean = length >= maxLength
+
+ /**
+ * Appends a string and accumulates its length to allocate a string buffer for all appended
+ * strings once in the toString method. Returns true if the string still has room for further
+ * appends before it hits its max limit.
+ */
+ def append(s: String): Unit = {
+ if (s != null) {
+ val sLen = s.length
+ if (!atLimit) {
+ val available = maxLength - length
+ val stringToAppend = if (available >= sLen) s else s.substring(0, available)
+ strings.add(stringToAppend)
+ }
+
+ // Keeps the total length of appended strings. Note that we need to cap the length at
+ // `ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH`; otherwise, we will overflow
+ // length causing StringIndexOutOfBoundsException in the substring call above.
+ length = Math.min(length.toLong + sLen, ByteArrayUtils.MAX_ROUNDED_ARRAY_LENGTH).toInt
+ }
+ }
+
+ /**
+ * The method allocates memory for all appended strings, writes them to the memory and returns
+ * concatenated string.
+ */
+ override def toString: String = {
+ val finalLength = if (atLimit) maxLength else length
+ val result = new java.lang.StringBuilder(finalLength)
+ strings.forEach(s => result.append(s))
+ result.toString
+ }
+}
+
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala
deleted file mode 100644
index 486093225f063..0000000000000
--- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala
+++ /dev/null
@@ -1,141 +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.catalyst.util
-
-import java.util.HexFormat
-import java.util.concurrent.atomic.AtomicBoolean
-
-import org.apache.spark.internal.Logging
-import org.apache.spark.unsafe.array.ByteArrayUtils
-import org.apache.spark.util.ArrayImplicits._
-
-/**
- * Concatenation of sequence of strings to final string with cheap append method and one memory
- * allocation for the final string. Can also bound the final size of the string.
- */
-class StringConcat(val maxLength: Int = ByteArrayUtils.MAX_ROUNDED_ARRAY_LENGTH) {
- protected val strings = new java.util.ArrayList[String]
- protected var length: Int = 0
-
- def atLimit: Boolean = length >= maxLength
-
- /**
- * Appends a string and accumulates its length to allocate a string buffer for all appended
- * strings once in the toString method. Returns true if the string still has room for further
- * appends before it hits its max limit.
- */
- def append(s: String): Unit = {
- if (s != null) {
- val sLen = s.length
- if (!atLimit) {
- val available = maxLength - length
- val stringToAppend = if (available >= sLen) s else s.substring(0, available)
- strings.add(stringToAppend)
- }
-
- // Keeps the total length of appended strings. Note that we need to cap the length at
- // `ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH`; otherwise, we will overflow
- // length causing StringIndexOutOfBoundsException in the substring call above.
- length = Math.min(length.toLong + sLen, ByteArrayUtils.MAX_ROUNDED_ARRAY_LENGTH).toInt
- }
- }
-
- /**
- * The method allocates memory for all appended strings, writes them to the memory and returns
- * concatenated string.
- */
- override def toString: String = {
- val finalLength = if (atLimit) maxLength else length
- val result = new java.lang.StringBuilder(finalLength)
- strings.forEach(s => result.append(s))
- result.toString
- }
-}
-
-object SparkStringUtils extends Logging {
-
- /** Whether we have warned about plan string truncation yet. */
- private val truncationWarningPrinted = new AtomicBoolean(false)
-
- /**
- * Format a sequence with semantics similar to calling .mkString(). Any elements beyond
- * `maxFields` will be dropped and replaced by a "... N more fields" placeholder.
- *
- * @return
- * the trimmed and formatted string.
- */
- def truncatedString[T](
- seq: Seq[T],
- start: String,
- sep: String,
- end: String,
- maxFields: Int,
- customToString: Option[T => String] = None): String = {
- if (seq.length > maxFields) {
- if (truncationWarningPrinted.compareAndSet(false, true)) {
- logWarning(
- "Truncated the string representation of a plan since it was too large. This " +
- s"behavior can be adjusted by setting 'spark.sql.debug.maxToStringFields'.")
- }
- val numFields = math.max(0, maxFields)
- val restNum = seq.length - numFields
- val ending = (if (numFields == 0) "" else sep) +
- (if (restNum == 0) "" else s"... $restNum more fields") + end
- if (customToString.isDefined) {
- seq.take(numFields).map(customToString.get).mkString(start, sep, ending)
- } else {
- seq.take(numFields).mkString(start, sep, ending)
- }
- } else {
- if (customToString.isDefined) {
- seq.map(customToString.get).mkString(start, sep, end)
- } else {
- seq.mkString(start, sep, end)
- }
- }
- }
-
- /** Shorthand for calling truncatedString() without start or end strings. */
- def truncatedString[T](seq: Seq[T], sep: String, maxFields: Int): String = {
- truncatedString(seq, "", sep, "", maxFields)
- }
-
- private final lazy val SPACE_DELIMITED_UPPERCASE_HEX =
- HexFormat.of().withDelimiter(" ").withUpperCase()
-
- /**
- * Returns a pretty string of the byte array which prints each byte as a hex digit and add
- * spaces between them. For example, [1A C0].
- */
- def getHexString(bytes: Array[Byte]): String = {
- s"[${SPACE_DELIMITED_UPPERCASE_HEX.formatHex(bytes)}]"
- }
-
- def sideBySide(left: String, right: String): Seq[String] = {
- sideBySide(left.split("\n").toImmutableArraySeq, right.split("\n").toImmutableArraySeq)
- }
-
- def sideBySide(left: Seq[String], right: Seq[String]): Seq[String] = {
- val maxLeftSize = left.map(_.length).max
- val leftPadded = left ++ Seq.fill(math.max(right.size - left.size, 0))("")
- val rightPadded = right ++ Seq.fill(math.max(left.size - right.size, 0))("")
-
- leftPadded.zip(rightPadded).map { case (l, r) =>
- (if (l == r) " " else "!") + l + (" " * ((maxLeftSize - l.length) + 3)) + r
- }
- }
-}
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimeFormatter.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimeFormatter.scala
index 46afbc8aca196..d0438c6ff1b46 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimeFormatter.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimeFormatter.scala
@@ -25,11 +25,11 @@ import org.apache.spark.sql.catalyst.util.SparkDateTimeUtils._
import org.apache.spark.unsafe.types.UTF8String
sealed trait TimeFormatter extends Serializable {
- def parse(s: String): Long // returns microseconds since midnight
+ def parse(s: String): Long // returns nanoseconds since midnight
def format(localTime: LocalTime): String
- // Converts microseconds since the midnight to time string
- def format(micros: Long): String
+ // Converts nanoseconds since the midnight to time string
+ def format(nanos: Long): String
def validatePatternString(): Unit
}
@@ -47,15 +47,15 @@ class Iso8601TimeFormatter(pattern: String, locale: Locale, isParsing: Boolean)
override def parse(s: String): Long = {
val localTime = toLocalTime(formatter.parse(s))
- localTimeToMicros(localTime)
+ localTimeToNanos(localTime)
}
override def format(localTime: LocalTime): String = {
localTime.format(formatter)
}
- override def format(micros: Long): String = {
- format(microsToLocalTime(micros))
+ override def format(nanos: Long): String = {
+ format(nanosToLocalTime(nanos))
}
override def validatePatternString(): Unit = {
@@ -135,4 +135,8 @@ object TimeFormatter {
def apply(isParsing: Boolean): TimeFormatter = {
getFormatter(None, defaultLocale, isParsing)
}
+
+ def getFractionFormatter(): TimeFormatter = {
+ new FractionTimeFormatter()
+ }
}
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/errors/ExecutionErrors.scala b/sql/api/src/main/scala/org/apache/spark/sql/errors/ExecutionErrors.scala
index 8124b1a4ab197..1a4369b172f82 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/errors/ExecutionErrors.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/errors/ExecutionErrors.scala
@@ -24,7 +24,7 @@ import org.apache.arrow.vector.types.pojo.ArrowType
import org.apache.spark.{QueryContext, SparkArithmeticException, SparkBuildInfo, SparkDateTimeException, SparkException, SparkRuntimeException, SparkUnsupportedOperationException, SparkUpgradeException}
import org.apache.spark.sql.catalyst.WalkedTypePath
import org.apache.spark.sql.internal.SqlApiConf
-import org.apache.spark.sql.types.{DataType, DoubleType, StringType, UserDefinedType}
+import org.apache.spark.sql.types.{DataType, DoubleType, StringType}
import org.apache.spark.unsafe.types.UTF8String
private[sql] trait ExecutionErrors extends DataTypeErrorsBase {
@@ -160,13 +160,6 @@ private[sql] trait ExecutionErrors extends DataTypeErrorsBase {
messageParameters = Map("typeName" -> toSQLType(typeName)))
}
- def userDefinedTypeNotAnnotatedAndRegisteredError(udt: UserDefinedType[_]): Throwable = {
- new SparkException(
- errorClass = "_LEGACY_ERROR_TEMP_2155",
- messageParameters = Map("userClass" -> udt.userClass.getName),
- cause = null)
- }
-
def cannotFindEncoderForTypeError(typeName: String): SparkUnsupportedOperationException = {
new SparkUnsupportedOperationException(
errorClass = "ENCODER_NOT_FOUND",
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala b/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala
index 12f986b89fd2b..60ccf7a9282cf 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala
@@ -324,7 +324,9 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase {
ctx)
}
- def charTypeMissingLengthError(dataType: String, ctx: PrimitiveDataTypeContext): Throwable = {
+ def charVarcharTypeMissingLengthError(
+ dataType: String,
+ ctx: PrimitiveDataTypeContext): Throwable = {
new ParseException(
errorClass = "DATATYPE_MISSING_SIZE",
messageParameters = Map("type" -> toSQLType(dataType)),
@@ -333,7 +335,7 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase {
def nestedTypeMissingElementTypeError(
dataType: String,
- ctx: PrimitiveDataTypeContext): Throwable = {
+ ctx: ComplexDataTypeContext): Throwable = {
dataType.toUpperCase(Locale.ROOT) match {
case "ARRAY" =>
new ParseException(
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/functions.scala b/sql/api/src/main/scala/org/apache/spark/sql/functions.scala
index ce5c76807b5c1..0165d8d4cf708 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/functions.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/functions.scala
@@ -871,6 +871,22 @@ object functions {
def last_value(e: Column, ignoreNulls: Column): Column =
Column.fn("last_value", e, ignoreNulls)
+ /**
+ * Create time from hour, minute and second fields. For invalid inputs it will throw an error.
+ *
+ * @param hour
+ * the hour to represent, from 0 to 23
+ * @param minute
+ * the minute to represent, from 0 to 59
+ * @param second
+ * the second to represent, from 0 to 59.999999
+ * @group datetime_funcs
+ * @since 4.1.0
+ */
+ def make_time(hour: Column, minute: Column, second: Column): Column = {
+ Column.fn("make_time", hour, minute, second)
+ }
+
/**
* Aggregate function: returns the most frequent value in a group.
*
@@ -1376,6 +1392,36 @@ object functions {
*/
def count_if(e: Column): Column = Column.fn("count_if", e)
+ /**
+ * Returns the current time at the start of query evaluation. Note that the result will contain
+ * 6 fractional digits of seconds.
+ *
+ * @return
+ * A time.
+ *
+ * @group datetime_funcs
+ * @since 4.1.0
+ */
+ def current_time(): Column = {
+ Column.fn("current_time")
+ }
+
+ /**
+ * Returns the current time at the start of query evaluation.
+ *
+ * @param precision
+ * An integer literal in the range [0..6], indicating how many fractional digits of seconds to
+ * include in the result.
+ * @return
+ * A time.
+ *
+ * @group datetime_funcs
+ * @since 4.1.0
+ */
+ def current_time(precision: Int): Column = {
+ Column.fn("current_time", lit(precision))
+ }
+
/**
* Aggregate function: computes a histogram on numeric 'expr' using nb bins. The return value is
* an array of (x,y) pairs representing the centers of the histogram's bins. As the value of
@@ -5400,7 +5446,7 @@ object functions {
def dayofyear(e: Column): Column = Column.fn("dayofyear", e)
/**
- * Extracts the hours as an integer from a given date/timestamp/string.
+ * Extracts the hours as an integer from a given date/time/timestamp/string.
* @return
* An integer, or null if the input was a string that could not be cast to a date
* @group datetime_funcs
@@ -5473,7 +5519,7 @@ object functions {
def last_day(e: Column): Column = Column.fn("last_day", e)
/**
- * Extracts the minutes as an integer from a given date/timestamp/string.
+ * Extracts the minutes as an integer from a given date/time/timestamp/string.
* @return
* An integer, or null if the input was a string that could not be cast to a date
* @group datetime_funcs
@@ -5579,7 +5625,7 @@ object functions {
Column.fn("next_day", date, dayOfWeek)
/**
- * Extracts the seconds as an integer from a given date/timestamp/string.
+ * Extracts the seconds as an integer from a given date/time/timestamp/string.
* @return
* An integer, or null if the input was a string that could not be cast to a timestamp
* @group datetime_funcs
@@ -5683,6 +5729,41 @@ object functions {
def unix_timestamp(s: Column, p: String): Column =
Column.fn("unix_timestamp", s, lit(p))
+ /**
+ * Parses a string value to a time value.
+ *
+ * @param str
+ * A string to be parsed to time.
+ * @return
+ * A time, or raises an error if the input is malformed.
+ *
+ * @group datetime_funcs
+ * @since 4.1.0
+ */
+ def to_time(str: Column): Column = {
+ Column.fn("to_time", str)
+ }
+
+ /**
+ * Parses a string value to a time value.
+ *
+ * See Datetime
+ * Patterns for valid time format patterns.
+ *
+ * @param str
+ * A string to be parsed to time.
+ * @param format
+ * A time format pattern to follow.
+ * @return
+ * A time, or raises an error if the input is malformed.
+ *
+ * @group datetime_funcs
+ * @since 4.1.0
+ */
+ def to_time(str: Column, format: Column): Column = {
+ Column.fn("to_time", str, format)
+ }
+
/**
* Converts to a timestamp by casting rules to `TimestampType`.
*
@@ -5715,6 +5796,41 @@ object functions {
*/
def to_timestamp(s: Column, fmt: String): Column = Column.fn("to_timestamp", s, lit(fmt))
+ /**
+ * Parses a string value to a time value.
+ *
+ * @param str
+ * A string to be parsed to time.
+ * @return
+ * A time, or null if the input is malformed.
+ *
+ * @group datetime_funcs
+ * @since 4.1.0
+ */
+ def try_to_time(str: Column): Column = {
+ Column.fn("try_to_time", str)
+ }
+
+ /**
+ * Parses a string value to a time value.
+ *
+ * See Datetime
+ * Patterns for valid time format patterns.
+ *
+ * @param str
+ * A string to be parsed to time.
+ * @param format
+ * A time format pattern to follow.
+ * @return
+ * A time, or null if the input is malformed.
+ *
+ * @group datetime_funcs
+ * @since 4.1.0
+ */
+ def try_to_time(str: Column, format: Column): Column = {
+ Column.fn("try_to_time", str, format)
+ }
+
/**
* Parses the `s` with the `format` to a timestamp. The function always returns null on an
* invalid input with`/`without ANSI SQL mode enabled. The result data type is consistent with
@@ -5763,6 +5879,24 @@ object functions {
*/
def to_date(e: Column, fmt: String): Column = Column.fn("to_date", e, lit(fmt))
+ /**
+ * This is a special version of `to_date` that performs the same operation, but returns a NULL
+ * value instead of raising an error if date cannot be created.
+ *
+ * @group datetime_funcs
+ * @since 4.0.0
+ */
+ def try_to_date(e: Column): Column = Column.fn("try_to_date", e)
+
+ /**
+ * This is a special version of `to_date` that performs the same operation, but returns a NULL
+ * value instead of raising an error if date cannot be created.
+ *
+ * @group datetime_funcs
+ * @since 4.0.0
+ */
+ def try_to_date(e: Column, fmt: String): Column = Column.fn("try_to_date", e, lit(fmt))
+
/**
* Returns the number of days since 1970-01-01.
*
@@ -6158,6 +6292,49 @@ object functions {
def timestamp_add(unit: String, quantity: Column, ts: Column): Column =
Column.internalFn("timestampadd", lit(unit), quantity, ts)
+ /**
+ * Returns the difference between two times, measured in specified units. Throws a
+ * SparkIllegalArgumentException, in case the specified unit is not supported.
+ *
+ * @param unit
+ * A STRING representing the unit of the time difference. Supported units are: "HOUR",
+ * "MINUTE", "SECOND", "MILLISECOND", and "MICROSECOND". The unit is case-insensitive.
+ * @param start
+ * A starting TIME.
+ * @param end
+ * An ending TIME.
+ * @return
+ * The difference between `end` and `start` times, measured in specified units.
+ * @note
+ * If any of the inputs is `NULL`, the result is `NULL`.
+ * @group datetime_funcs
+ * @since 4.1.0
+ */
+ def time_diff(unit: Column, start: Column, end: Column): Column = {
+ Column.fn("time_diff", unit, start, end)
+ }
+
+ /**
+ * Returns `time` truncated to the `unit`.
+ *
+ * @param unit
+ * A STRING representing the unit to truncate the time to. Supported units are: "HOUR",
+ * "MINUTE", "SECOND", "MILLISECOND", and "MICROSECOND". The unit is case-insensitive.
+ * @param time
+ * A TIME to truncate.
+ * @return
+ * A TIME truncated to the specified unit.
+ * @note
+ * If any of the inputs is `NULL`, the result is `NULL`.
+ * @throws IllegalArgumentException
+ * If the `unit` is not supported.
+ * @group datetime_funcs
+ * @since 4.1.0
+ */
+ def time_trunc(unit: Column, time: Column): Column = {
+ Column.fn("time_trunc", unit, time)
+ }
+
/**
* Parses the `timestamp` expression with the `format` expression to a timestamp without time
* zone. Returns null with invalid input.
@@ -8464,6 +8641,15 @@ object functions {
secs: Column): Column =
Column.fn("make_timestamp_ntz", years, months, days, hours, mins, secs)
+ /**
+ * Create a local date-time from date and time fields.
+ *
+ * @group datetime_funcs
+ * @since 4.1.0
+ */
+ def make_timestamp_ntz(date: Column, time: Column): Column =
+ Column.fn("make_timestamp_ntz", date, time)
+
/**
* Try to create a local date-time from years, months, days, hours, mins, secs fields. The
* function returns NULL on invalid inputs.
@@ -8480,6 +8666,15 @@ object functions {
secs: Column): Column =
Column.fn("try_make_timestamp_ntz", years, months, days, hours, mins, secs)
+ /**
+ * Try to create a local date-time from date and time fields.
+ *
+ * @group datetime_funcs
+ * @since 4.1.0
+ */
+ def try_make_timestamp_ntz(date: Column, time: Column): Column =
+ Column.fn("try_make_timestamp_ntz", date, time)
+
/**
* Make year-month interval from years, months.
*
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala
index 76449f1704d26..3ab9b312feea8 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala
@@ -47,6 +47,9 @@ private[sql] trait SqlApiConf {
def stackTracesInDataFrameContext: Int
def dataFrameQueryContextEnabled: Boolean
def legacyAllowUntypedScalaUDFs: Boolean
+ def manageParserCaches: Boolean
+ def parserDfaCacheFlushThreshold: Int
+ def parserDfaCacheFlushRatio: Double
}
private[sql] object SqlApiConf {
@@ -60,6 +63,11 @@ private[sql] object SqlApiConf {
val LOCAL_RELATION_CACHE_THRESHOLD_KEY: String = {
SqlApiConfHelper.LOCAL_RELATION_CACHE_THRESHOLD_KEY
}
+ val PARSER_DFA_CACHE_FLUSH_THRESHOLD_KEY: String =
+ SqlApiConfHelper.PARSER_DFA_CACHE_FLUSH_THRESHOLD_KEY
+ val PARSER_DFA_CACHE_FLUSH_RATIO_KEY: String =
+ SqlApiConfHelper.PARSER_DFA_CACHE_FLUSH_RATIO_KEY
+ val MANAGE_PARSER_CACHES_KEY: String = SqlApiConfHelper.MANAGE_PARSER_CACHES_KEY
def get: SqlApiConf = SqlApiConfHelper.getConfGetter.get()()
@@ -88,4 +96,7 @@ private[sql] object DefaultSqlApiConf extends SqlApiConf {
override def stackTracesInDataFrameContext: Int = 1
override def dataFrameQueryContextEnabled: Boolean = true
override def legacyAllowUntypedScalaUDFs: Boolean = false
+ override def manageParserCaches: Boolean = false
+ override def parserDfaCacheFlushThreshold: Int = -1
+ override def parserDfaCacheFlushRatio: Double = -1.0
}
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala
index dace1dbaecfa5..727620bd5bd03 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala
@@ -33,6 +33,10 @@ private[sql] object SqlApiConfHelper {
val SESSION_LOCAL_TIMEZONE_KEY: String = "spark.sql.session.timeZone"
val LOCAL_RELATION_CACHE_THRESHOLD_KEY: String = "spark.sql.session.localRelationCacheThreshold"
val ARROW_EXECUTION_USE_LARGE_VAR_TYPES = "spark.sql.execution.arrow.useLargeVarTypes"
+ val PARSER_DFA_CACHE_FLUSH_THRESHOLD_KEY: String =
+ "spark.sql.parser.parserDfaCacheFlushThreshold"
+ val PARSER_DFA_CACHE_FLUSH_RATIO_KEY: String = "spark.sql.parser.parserDfaCacheFlushRatio"
+ val MANAGE_PARSER_CACHES_KEY: String = "spark.sql.parser.manageParserCaches"
val confGetter: AtomicReference[() => SqlApiConf] = {
new AtomicReference[() => SqlApiConf](() => DefaultSqlApiConf)
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala
index 9590fb23e16b1..a87482e88139f 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala
@@ -175,3 +175,17 @@ private[spark] object AnsiIntervalType extends AbstractDataType {
override private[sql] def defaultConcreteType: DataType = DayTimeIntervalType()
}
+
+/**
+ * A TIME type of any valid precision.
+ */
+private[sql] abstract class AnyTimeType extends DatetimeType
+
+private[spark] object AnyTimeType extends AbstractDataType with Serializable {
+ override private[sql] def simpleString: String = "time"
+
+ override private[sql] def acceptsType(other: DataType): Boolean =
+ other.isInstanceOf[AnyTimeType]
+
+ override private[sql] def defaultConcreteType: DataType = TimeType(TimeType.DEFAULT_PRECISION)
+}
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala
index f798276d60f7c..3ecc84a1578a4 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala
@@ -126,6 +126,7 @@ object DataType {
private val FIXED_DECIMAL = """decimal\(\s*(\d+)\s*,\s*(\-?\d+)\s*\)""".r
private val CHAR_TYPE = """char\(\s*(\d+)\s*\)""".r
private val VARCHAR_TYPE = """varchar\(\s*(\d+)\s*\)""".r
+ private val STRING_WITH_COLLATION = """string\s+collate\s+(\w+)""".r
val COLLATIONS_METADATA_KEY = "__COLLATIONS"
@@ -215,6 +216,7 @@ object DataType {
case FIXED_DECIMAL(precision, scale) => DecimalType(precision.toInt, scale.toInt)
case CHAR_TYPE(length) => CharType(length.toInt)
case VARCHAR_TYPE(length) => VarcharType(length.toInt)
+ case STRING_WITH_COLLATION(collation) => StringType(collation)
// For backwards compatibility, previously the type name of NullType is "null"
case "null" => NullType
case "timestamp_ltz" => TimestampType
@@ -234,11 +236,15 @@ object DataType {
}
}
+ private[sql] def parseDataType(json: JValue): DataType = {
+ parseDataType(json, fieldPath = "", collationsMap = Map.empty)
+ }
+
// NOTE: Map fields must be sorted in alphabetical order to keep consistent with the Python side.
private[sql] def parseDataType(
json: JValue,
- fieldPath: String = "",
- collationsMap: Map[String, String] = Map.empty): DataType = json match {
+ fieldPath: String,
+ collationsMap: Map[String, String]): DataType = json match {
case JString(name) =>
collationsMap.get(fieldPath) match {
case Some(collation) =>
@@ -442,15 +448,41 @@ object DataType {
}
/**
- * Check if `from` is equal to `to` type except for collations, which are checked to be
- * compatible so that data of type `from` can be interpreted as of type `to`.
+ * Compares two data types, ignoring compatible collation of StringType. If `checkComplexTypes`
+ * is true, it will also ignore collations for nested types.
*/
- private[sql] def equalsIgnoreCompatibleCollation(from: DataType, to: DataType): Boolean = {
- (from, to) match {
- // String types with possibly different collations are compatible.
- case (a: StringType, b: StringType) => a.constraint == b.constraint
+ private[sql] def equalsIgnoreCompatibleCollation(
+ from: DataType,
+ to: DataType,
+ checkComplexTypes: Boolean = true): Boolean = {
+ def transform: PartialFunction[DataType, DataType] = {
+ case dt @ (_: CharType | _: VarcharType) => dt
+ case _: StringType => StringType
+ // SPARK-53330 (see below)
+ case _: DayTimeIntervalType => DayTimeIntervalType.DEFAULT
+ }
- case (fromDataType, toDataType) => fromDataType == toDataType
+ if (checkComplexTypes) {
+ from.transformRecursively(transform) == to.transformRecursively(transform)
+ } else {
+ (from, to) match {
+ case (a: StringType, b: StringType) => a.constraint == b.constraint
+ // SPARK-53330: Arrow serialization always returns DayTimeIntervalType(0, 3)
+ // as it has the maximum range, we can always assume that we can match
+ // with the target type.
+ case (x: DayTimeIntervalType, y: DayTimeIntervalType) => true
+
+ case (fromDataType, toDataType) => fromDataType == toDataType
+ }
+ }
+ }
+
+ private[sql] def equalsIgnoreCompatibleCollation(
+ from: Seq[DataType],
+ to: Seq[DataType]): Boolean = {
+ from.length == to.length &&
+ from.zip(to).forall { case (fromDataType, toDataType) =>
+ equalsIgnoreCompatibleCollation(fromDataType, toDataType)
}
}
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/Metadata.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/Metadata.scala
index 8d7bcbac8af16..f2cfcb984594d 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/types/Metadata.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/types/Metadata.scala
@@ -40,13 +40,15 @@ import org.apache.spark.util.ArrayImplicits._
* @since 1.3.0
*/
@Stable
-sealed class Metadata private[types] (
- private[types] val map: Map[String, Any],
- @transient private[types] val runtimeMap: Map[String, Any])
+@SerialVersionUID(-3987058932362209243L)
+sealed class Metadata private[types] (private[types] val map: Map[String, Any])
extends Serializable {
+ @transient private[types] var runtimeMap: Map[String, Any] = _
+ private[types] def setRuntimeMap(map: Map[String, Any]): Unit = runtimeMap = map
+
/** No-arg constructor for kryo. */
- protected def this() = this(null, null)
+ protected def this() = this(null)
/** Tests whether this Metadata contains a binding for a key. */
def contains(key: String): Boolean = map.contains(key)
@@ -137,7 +139,7 @@ sealed class Metadata private[types] (
@Stable
object Metadata {
- private[this] val _empty = new Metadata(Map.empty, Map.empty)
+ private[this] val _empty = new Metadata(Map.empty)
/** Returns an empty Metadata. */
def empty: Metadata = _empty
@@ -305,7 +307,16 @@ class MetadataBuilder {
/** Builds the [[Metadata]] instance. */
def build(): Metadata = {
- new Metadata(map.toMap, runtimeMap.toMap)
+ if (map.isEmpty && runtimeMap.isEmpty) {
+ // Save some memory when the metadata is empty
+ Metadata.empty
+ } else {
+ val metadata = new Metadata(map.toMap)
+ if (runtimeMap.nonEmpty) {
+ metadata.setRuntimeMap(runtimeMap.toMap)
+ }
+ metadata
+ }
}
private def put(key: String, value: Any): this.type = {
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala
index 5fec578b03581..787730f775084 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala
@@ -17,8 +17,6 @@
package org.apache.spark.sql.types
-import org.json4s.JsonAST.{JString, JValue}
-
import org.apache.spark.annotation.Stable
import org.apache.spark.sql.catalyst.util.CollationFactory
import org.apache.spark.sql.internal.SqlApiConf
@@ -90,11 +88,6 @@ class StringType private[sql] (
private[sql] def collationName: String =
CollationFactory.fetchCollation(collationId).collationName
- // Due to backwards compatibility and compatibility with other readers
- // all string types are serialized in json as regular strings and
- // the collation information is written to struct field metadata
- override def jsonValue: JValue = JString("string")
-
override def equals(obj: Any): Boolean = {
obj match {
case s: StringType => s.collationId == collationId && s.constraint == constraint
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/StructField.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/StructField.scala
index 4c51980d4e6c4..eb3d30051880a 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/types/StructField.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/types/StructField.scala
@@ -70,11 +70,30 @@ case class StructField(
private[sql] def jsonValue: JValue = {
("name" -> name) ~
- ("type" -> dataType.jsonValue) ~
+ ("type" -> dataTypeJsonValue) ~
("nullable" -> nullable) ~
("metadata" -> metadataJson)
}
+ private[sql] def dataTypeJsonValue: JValue = {
+ if (collationMetadata.isEmpty) return dataType.jsonValue
+
+ def removeCollations(dt: DataType): DataType = dt match {
+ // Only recurse into map and array types as any child struct type
+ // will have already been processed.
+ case ArrayType(et, nullable) =>
+ ArrayType(removeCollations(et), nullable)
+ case MapType(kt, vt, nullable) =>
+ MapType(removeCollations(kt), removeCollations(vt), nullable)
+ case st: StringType => StringHelper.removeCollation(st)
+ case _ => dt
+ }
+
+ // As we want to be backwards compatible we should remove all collations information from the
+ // json and only keep that information in the metadata.
+ removeCollations(dataType).jsonValue
+ }
+
private def metadataJson: JValue = {
val metadataJsonValue = metadata.jsonValue
metadataJsonValue match {
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/StructType.scala
index 4c49d3a58f4fc..5b1d9f1f116a5 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/types/StructType.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/types/StructType.scala
@@ -30,11 +30,11 @@ import org.apache.spark.annotation.Stable
import org.apache.spark.sql.catalyst.analysis.SqlApiAnalysis
import org.apache.spark.sql.catalyst.parser.{DataTypeParser, LegacyTypeStringParser}
import org.apache.spark.sql.catalyst.trees.Origin
-import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, SparkStringUtils, StringConcat}
+import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, StringConcat}
import org.apache.spark.sql.errors.DataTypeErrors
import org.apache.spark.sql.errors.DataTypeErrors.toSQLId
import org.apache.spark.sql.internal.SqlApiConf
-import org.apache.spark.util.SparkCollectionUtils
+import org.apache.spark.util.{SparkCollectionUtils, SparkStringUtils}
/**
* A [[StructType]] object can be constructed by
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/TimeType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/TimeType.scala
index c42311c6a1dcc..135ad278438ed 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/types/TimeType.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/types/TimeType.scala
@@ -31,7 +31,7 @@ import org.apache.spark.sql.errors.DataTypeErrors
* @since 4.1.0
*/
@Unstable
-case class TimeType(precision: Int) extends DatetimeType {
+case class TimeType(precision: Int) extends AnyTimeType {
if (precision < TimeType.MIN_PRECISION || precision > TimeType.MAX_PRECISION) {
throw DataTypeErrors.unsupportedTimePrecisionError(precision)
@@ -51,6 +51,8 @@ object TimeType {
val MIN_PRECISION: Int = 0
val MICROS_PRECISION: Int = 6
val MAX_PRECISION: Int = MICROS_PRECISION
+ val DEFAULT_PRECISION: Int = MICROS_PRECISION
+ val NANOS_PRECISION: Int = 9
- def apply(): TimeType = new TimeType(MICROS_PRECISION)
+ def apply(): TimeType = new TimeType(DEFAULT_PRECISION)
}
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/UDTRegistration.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/UDTRegistration.scala
index 85d421a07577b..7bc9bde65c19b 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/types/UDTRegistration.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/types/UDTRegistration.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.types
import scala.collection.mutable
import org.apache.spark.annotation.{DeveloperApi, Since}
-import org.apache.spark.internal.{Logging, LogKeys, MDC}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.sql.errors.DataTypeErrors
import org.apache.spark.util.SparkClassUtils
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala
index dd8ca26c52462..3d3521d88fdf5 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala
@@ -94,6 +94,22 @@ abstract class UserDefinedType[UserType >: Null] extends DataType with Serializa
}
override def catalogString: String = sqlType.simpleString
+
+ /**
+ * This method is used to convert the value of a UDT to a string representation.
+ *
+ * By default, it simply calls `toString` on the object.
+ *
+ * @param obj
+ * The object to convert to a string.
+ * @return
+ * A string representation of the object.
+ * @since 4.1.0
+ */
+ @Since("4.1.0")
+ def stringifyValue(obj: Any): String = {
+ obj.toString
+ }
}
private[spark] object UserDefinedType {
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala b/sql/api/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala
index 587ca43e57300..6caabf20f8f6b 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala
@@ -58,6 +58,7 @@ private[sql] object ArrowUtils {
case TimestampType => new ArrowType.Timestamp(TimeUnit.MICROSECOND, timeZoneId)
case TimestampNTZType =>
new ArrowType.Timestamp(TimeUnit.MICROSECOND, null)
+ case _: TimeType => new ArrowType.Time(TimeUnit.NANOSECOND, 8 * 8)
case NullType => ArrowType.Null.INSTANCE
case _: YearMonthIntervalType => new ArrowType.Interval(IntervalUnit.YEAR_MONTH)
case _: DayTimeIntervalType => new ArrowType.Duration(TimeUnit.MICROSECOND)
@@ -88,6 +89,8 @@ private[sql] object ArrowUtils {
if ts.getUnit == TimeUnit.MICROSECOND && ts.getTimezone == null =>
TimestampNTZType
case ts: ArrowType.Timestamp if ts.getUnit == TimeUnit.MICROSECOND => TimestampType
+ case t: ArrowType.Time if t.getUnit == TimeUnit.NANOSECOND && t.getBitWidth == 8 * 8 =>
+ TimeType(TimeType.MICROS_PRECISION)
case ArrowType.Null.INSTANCE => NullType
case yi: ArrowType.Interval if yi.getUnit == IntervalUnit.YEAR_MONTH =>
YearMonthIntervalType()
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/util/ArtifactUtils.scala b/sql/api/src/main/scala/org/apache/spark/sql/util/ArtifactUtils.scala
index 8cd239b55cff0..0fc14b4c8b460 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/util/ArtifactUtils.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/util/ArtifactUtils.scala
@@ -19,6 +19,8 @@ package org.apache.spark.sql.util
import java.nio.file.{Path, Paths}
+import org.apache.spark.SparkRuntimeException
+
object ArtifactUtils {
private[sql] def concatenatePaths(basePath: Path, otherPath: Path): Path = {
@@ -40,4 +42,18 @@ object ArtifactUtils {
private[sql] def concatenatePaths(basePath: Path, otherPath: String): Path = {
concatenatePaths(basePath, Paths.get(otherPath))
}
+
+ /**
+ * Converts a sequence of exceptions into a single exception by adding all but the first
+ * exceptions as suppressed exceptions to the first one.
+ * @param exceptions
+ * @return
+ */
+ private[sql] def mergeExceptionsWithSuppressed(
+ exceptions: Seq[SparkRuntimeException]): SparkRuntimeException = {
+ require(exceptions.nonEmpty)
+ val mainException = exceptions.head
+ exceptions.drop(1).foreach(mainException.addSuppressed)
+ mainException
+ }
}
diff --git a/sql/api/src/main/scala/org/apache/spark/sql/util/ProtobufUtils.scala b/sql/api/src/main/scala/org/apache/spark/sql/util/ProtobufUtils.scala
index 11f35ceb060c0..be56e3cf13133 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/util/ProtobufUtils.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/util/ProtobufUtils.scala
@@ -18,18 +18,16 @@
package org.apache.spark.sql.util
import java.io.{File, FileNotFoundException}
-import java.nio.file.NoSuchFileException
+import java.nio.file.{Files, NoSuchFileException}
import scala.util.control.NonFatal
-import org.apache.commons.io.FileUtils
-
import org.apache.spark.sql.errors.CompilationErrors
object ProtobufUtils {
def readDescriptorFileContent(filePath: String): Array[Byte] = {
try {
- FileUtils.readFileToByteArray(new File(filePath))
+ Files.readAllBytes(new File(filePath).toPath())
} catch {
case ex: FileNotFoundException =>
throw CompilationErrors.cannotFindDescriptorFileError(filePath, ex)
diff --git a/sql/catalyst/benchmarks/CalendarIntervalBenchmark-jdk21-results.txt b/sql/catalyst/benchmarks/CalendarIntervalBenchmark-jdk21-results.txt
index 01fcb4c21cb28..6daa0f267ff51 100644
--- a/sql/catalyst/benchmarks/CalendarIntervalBenchmark-jdk21-results.txt
+++ b/sql/catalyst/benchmarks/CalendarIntervalBenchmark-jdk21-results.txt
@@ -2,10 +2,10 @@
CalendarInterval
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
CalendarInterval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Call setInterval & getInterval 1032 1033 1 130.0 7.7 1.0X
+Call setInterval & getInterval 1034 1034 0 129.9 7.7 1.0X
diff --git a/sql/catalyst/benchmarks/CalendarIntervalBenchmark-results.txt b/sql/catalyst/benchmarks/CalendarIntervalBenchmark-results.txt
index 1f828510aa2ab..88548550e418d 100644
--- a/sql/catalyst/benchmarks/CalendarIntervalBenchmark-results.txt
+++ b/sql/catalyst/benchmarks/CalendarIntervalBenchmark-results.txt
@@ -2,10 +2,10 @@
CalendarInterval
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
CalendarInterval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Call setInterval & getInterval 1051 1053 2 127.7 7.8 1.0X
+Call setInterval & getInterval 1037 1038 2 129.4 7.7 1.0X
diff --git a/sql/catalyst/benchmarks/EnumTypeSetBenchmark-jdk21-results.txt b/sql/catalyst/benchmarks/EnumTypeSetBenchmark-jdk21-results.txt
index 42b473742f2f1..54be9abdb59d7 100644
--- a/sql/catalyst/benchmarks/EnumTypeSetBenchmark-jdk21-results.txt
+++ b/sql/catalyst/benchmarks/EnumTypeSetBenchmark-jdk21-results.txt
@@ -1,105 +1,105 @@
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test contains use empty Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Use HashSet 1 1 0 1389.0 0.7 1.0X
-Use EnumSet 2 2 0 560.9 1.8 0.4X
+Use HashSet 1 1 0 1390.4 0.7 1.0X
+Use EnumSet 2 2 0 414.9 2.4 0.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test contains use 1 item Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Use HashSet 2 2 0 424.0 2.4 1.0X
-Use EnumSet 2 2 0 571.0 1.8 1.3X
+Use HashSet 2 2 0 489.8 2.0 1.0X
+Use EnumSet 2 2 0 507.3 2.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test contains use 3 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Use HashSet 2 3 0 413.2 2.4 1.0X
-Use EnumSet 2 2 0 601.3 1.7 1.5X
+Use HashSet 2 2 0 490.2 2.0 1.0X
+Use EnumSet 2 2 0 562.4 1.8 1.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test contains use 5 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Use HashSet 9 9 0 109.4 9.1 1.0X
-Use EnumSet 2 2 0 550.6 1.8 5.0X
+Use HashSet 9 9 0 116.1 8.6 1.0X
+Use EnumSet 2 2 0 532.6 1.9 4.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test contains use 10 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Use HashSet 10 10 0 99.7 10.0 1.0X
-Use EnumSet 2 2 0 550.7 1.8 5.5X
+Use HashSet 10 10 0 105.0 9.5 1.0X
+Use EnumSet 2 2 0 532.4 1.9 5.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test create empty Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Use HashSet 0 0 0 413.1 2.4 1.0X
-Use EnumSet 0 0 0 291.1 3.4 0.7X
+Use HashSet 0 0 0 412.3 2.4 1.0X
+Use EnumSet 0 0 0 291.5 3.4 0.7X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test create 1 item Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Use HashSet 1 1 0 102.3 9.8 1.0X
-Use EnumSet 0 0 0 290.9 3.4 2.8X
+Use HashSet 1 1 0 102.6 9.7 1.0X
+Use EnumSet 0 0 0 294.4 3.4 2.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test create 3 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Use HashSet 9 10 0 10.6 94.7 1.0X
-Use EnumSet 1 1 0 132.0 7.6 12.5X
+Use HashSet 10 10 0 10.2 98.5 1.0X
+Use EnumSet 1 1 0 132.6 7.5 13.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test create 5 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Use HashSet 35 36 1 2.8 354.1 1.0X
-Use EnumSet 1 1 0 144.5 6.9 51.2X
+Use HashSet 32 33 0 3.1 323.4 1.0X
+Use EnumSet 1 1 0 144.8 6.9 46.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test create 10 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Use HashSet 57 57 0 1.8 569.8 1.0X
-Use EnumSet 1 1 0 138.3 7.2 78.8X
+Use HashSet 57 57 1 1.8 568.2 1.0X
+Use EnumSet 1 1 0 138.5 7.2 78.7X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test create and contains use empty Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Use HashSet 4 4 0 267.5 3.7 1.0X
-Use EnumSet 5 5 0 198.5 5.0 0.7X
+Use HashSet 4 4 0 226.1 4.4 1.0X
+Use EnumSet 6 6 0 176.0 5.7 0.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test create and contains use 1 item Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Use HashSet 12 13 0 85.1 11.8 1.0X
-Use EnumSet 6 6 0 160.1 6.2 1.9X
+Use HashSet 13 14 0 74.3 13.5 1.0X
+Use EnumSet 7 7 0 149.4 6.7 2.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test create and contains use 3 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Use HashSet 26 27 0 38.4 26.1 1.0X
-Use EnumSet 6 6 0 163.7 6.1 4.3X
+Use HashSet 27 27 0 37.2 26.9 1.0X
+Use EnumSet 6 7 0 154.6 6.5 4.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test create and contains use 5 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Use HashSet 51 51 1 19.7 50.8 1.0X
-Use EnumSet 6 6 0 160.4 6.2 8.1X
+Use HashSet 54 54 0 18.6 53.7 1.0X
+Use EnumSet 7 7 0 152.2 6.6 8.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test create and contains use 10 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
-Use HashSet 78 79 0 12.8 78.1 1.0X
-Use EnumSet 7 7 0 149.0 6.7 11.6X
+Use HashSet 77 78 1 12.9 77.5 1.0X
+Use EnumSet 7 7 0 150.8 6.6 11.7X
diff --git a/sql/catalyst/benchmarks/EnumTypeSetBenchmark-results.txt b/sql/catalyst/benchmarks/EnumTypeSetBenchmark-results.txt
index a63a191f4e2d6..69baaea7f6e5d 100644
--- a/sql/catalyst/benchmarks/EnumTypeSetBenchmark-results.txt
+++ b/sql/catalyst/benchmarks/EnumTypeSetBenchmark-results.txt
@@ -1,105 +1,105 @@
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test contains use empty Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Use HashSet 1 1 0 1446.7 0.7 1.0X
-Use EnumSet 2 2 0 498.8 2.0 0.3X
+Use HashSet 1 1 0 1452.7 0.7 1.0X
+Use EnumSet 2 2 0 561.7 1.8 0.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test contains use 1 item Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Use HashSet 2 2 0 495.9 2.0 1.0X
-Use EnumSet 2 2 0 491.3 2.0 1.0X
+Use HashSet 2 2 0 506.0 2.0 1.0X
+Use EnumSet 2 2 0 483.6 2.1 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test contains use 3 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Use HashSet 2 2 0 488.0 2.0 1.0X
-Use EnumSet 2 2 0 552.0 1.8 1.1X
+Use HashSet 2 2 0 542.0 1.8 1.0X
+Use EnumSet 2 2 0 577.9 1.7 1.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test contains use 5 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Use HashSet 9 9 0 108.3 9.2 1.0X
-Use EnumSet 2 2 0 533.8 1.9 4.9X
+Use HashSet 8 8 0 121.7 8.2 1.0X
+Use EnumSet 2 2 0 540.7 1.8 4.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test contains use 10 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Use HashSet 10 10 0 103.2 9.7 1.0X
-Use EnumSet 2 2 0 534.3 1.9 5.2X
+Use HashSet 9 9 0 109.4 9.1 1.0X
+Use EnumSet 2 2 0 541.0 1.8 4.9X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test create empty Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Use HashSet 0 0 0 414.2 2.4 1.0X
-Use EnumSet 1 1 0 132.4 7.6 0.3X
+Use HashSet 0 0 0 413.0 2.4 1.0X
+Use EnumSet 1 1 0 132.1 7.6 0.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test create 1 item Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Use HashSet 2 2 0 58.9 17.0 1.0X
-Use EnumSet 1 1 0 135.5 7.4 2.3X
+Use HashSet 1 1 0 98.2 10.2 1.0X
+Use EnumSet 0 0 0 209.7 4.8 2.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test create 3 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Use HashSet 9 9 0 11.0 90.9 1.0X
-Use EnumSet 1 1 0 137.6 7.3 12.5X
+Use HashSet 9 10 1 11.2 89.6 1.0X
+Use EnumSet 1 1 0 150.4 6.6 13.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test create 5 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Use HashSet 34 35 1 2.9 341.5 1.0X
-Use EnumSet 1 1 0 137.4 7.3 46.9X
+Use HashSet 31 31 0 3.3 306.4 1.0X
+Use EnumSet 1 1 0 150.6 6.6 46.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test create 10 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Use HashSet 58 58 1 1.7 576.3 1.0X
-Use EnumSet 1 1 0 131.3 7.6 75.7X
+Use HashSet 58 58 1 1.7 576.1 1.0X
+Use EnumSet 1 1 0 138.2 7.2 79.6X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test create and contains use empty Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Use HashSet 4 4 0 251.0 4.0 1.0X
-Use EnumSet 5 5 0 190.6 5.2 0.8X
+Use HashSet 4 4 0 234.2 4.3 1.0X
+Use EnumSet 6 6 0 181.0 5.5 0.8X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test create and contains use 1 item Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Use HashSet 11 12 0 87.1 11.5 1.0X
-Use EnumSet 6 7 0 157.5 6.4 1.8X
+Use HashSet 10 11 0 96.0 10.4 1.0X
+Use EnumSet 7 7 0 150.9 6.6 1.6X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test create and contains use 3 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Use HashSet 29 29 0 34.9 28.6 1.0X
-Use EnumSet 6 6 0 157.4 6.4 4.5X
+Use HashSet 28 29 0 35.3 28.3 1.0X
+Use EnumSet 7 7 0 150.9 6.6 4.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test create and contains use 5 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Use HashSet 55 56 1 18.1 55.4 1.0X
-Use EnumSet 6 7 0 154.3 6.5 8.5X
+Use HashSet 53 54 0 18.8 53.3 1.0X
+Use EnumSet 7 7 0 148.8 6.7 7.9X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test create and contains use 10 items Set: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
-Use HashSet 81 82 1 12.4 80.8 1.0X
-Use EnumSet 7 7 0 153.7 6.5 12.4X
+Use HashSet 78 79 1 12.8 78.0 1.0X
+Use EnumSet 7 7 0 147.4 6.8 11.5X
diff --git a/sql/catalyst/benchmarks/EscapePathBenchmark-jdk21-results.txt b/sql/catalyst/benchmarks/EscapePathBenchmark-jdk21-results.txt
index 471fea1fbd045..680c21beb2f52 100644
--- a/sql/catalyst/benchmarks/EscapePathBenchmark-jdk21-results.txt
+++ b/sql/catalyst/benchmarks/EscapePathBenchmark-jdk21-results.txt
@@ -2,23 +2,23 @@
Escape
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Escape Tests: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Legacy 6403 6417 22 0.2 6403.1 1.0X
-New 792 822 11 1.3 791.5 8.1X
+Legacy 7201 7208 5 0.1 7200.9 1.0X
+New 809 811 2 1.2 809.0 8.9X
================================================================================================
Unescape
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Unescape Tests: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Legacy 5057 5069 11 0.2 5057.1 1.0X
-New 589 592 3 1.7 588.7 8.6X
+Legacy 5298 5304 7 0.2 5297.6 1.0X
+New 591 598 4 1.7 591.3 9.0X
diff --git a/sql/catalyst/benchmarks/EscapePathBenchmark-results.txt b/sql/catalyst/benchmarks/EscapePathBenchmark-results.txt
index 9dc0f68498fed..0297bd37455e2 100644
--- a/sql/catalyst/benchmarks/EscapePathBenchmark-results.txt
+++ b/sql/catalyst/benchmarks/EscapePathBenchmark-results.txt
@@ -2,23 +2,23 @@
Escape
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Escape Tests: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Legacy 6776 6787 13 0.1 6776.1 1.0X
-New 680 682 1 1.5 680.5 10.0X
+Legacy 6814 6819 4 0.1 6814.3 1.0X
+New 688 719 11 1.5 687.6 9.9X
================================================================================================
Unescape
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Unescape Tests: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Legacy 6899 6912 13 0.1 6898.6 1.0X
-New 592 598 7 1.7 592.3 11.6X
+Legacy 6008 6016 5 0.2 6008.2 1.0X
+New 593 599 6 1.7 593.1 10.1X
diff --git a/sql/catalyst/benchmarks/GenericArrayDataBenchmark-jdk21-results.txt b/sql/catalyst/benchmarks/GenericArrayDataBenchmark-jdk21-results.txt
index 2c9329b736de5..38c290b69c5f2 100644
--- a/sql/catalyst/benchmarks/GenericArrayDataBenchmark-jdk21-results.txt
+++ b/sql/catalyst/benchmarks/GenericArrayDataBenchmark-jdk21-results.txt
@@ -1,10 +1,10 @@
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
constructor: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-arrayOfAny 6 6 0 1611.5 0.6 1.0X
-arrayOfAnyAsObject 6 6 0 1611.6 0.6 1.0X
-arrayOfAnyAsSeq 173 175 3 57.7 17.3 0.0X
-arrayOfInt 271 272 0 36.8 27.1 0.0X
-arrayOfIntAsObject 251 252 1 39.8 25.1 0.0X
+arrayOfAny 6 6 0 1611.7 0.6 1.0X
+arrayOfAnyAsObject 6 6 0 1611.8 0.6 1.0X
+arrayOfAnyAsSeq 174 175 1 57.5 17.4 0.0X
+arrayOfInt 393 395 1 25.4 39.3 0.0X
+arrayOfIntAsObject 419 419 1 23.9 41.9 0.0X
diff --git a/sql/catalyst/benchmarks/GenericArrayDataBenchmark-results.txt b/sql/catalyst/benchmarks/GenericArrayDataBenchmark-results.txt
index 19dc9e88757be..7ee0bb288eb73 100644
--- a/sql/catalyst/benchmarks/GenericArrayDataBenchmark-results.txt
+++ b/sql/catalyst/benchmarks/GenericArrayDataBenchmark-results.txt
@@ -1,10 +1,10 @@
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
constructor: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-arrayOfAny 6 6 0 1611.5 0.6 1.0X
-arrayOfAnyAsObject 6 6 0 1611.4 0.6 1.0X
-arrayOfAnyAsSeq 158 160 1 63.1 15.8 0.0X
-arrayOfInt 254 255 1 39.4 25.4 0.0X
-arrayOfIntAsObject 254 256 3 39.4 25.4 0.0X
+arrayOfAny 6 6 0 1611.8 0.6 1.0X
+arrayOfAnyAsObject 6 6 0 1611.8 0.6 1.0X
+arrayOfAnyAsSeq 158 159 1 63.4 15.8 0.0X
+arrayOfInt 258 259 1 38.8 25.8 0.0X
+arrayOfIntAsObject 258 258 0 38.8 25.8 0.0X
diff --git a/sql/catalyst/benchmarks/HashBenchmark-jdk21-results.txt b/sql/catalyst/benchmarks/HashBenchmark-jdk21-results.txt
index 08fcb7d351e26..8175c4c7120db 100644
--- a/sql/catalyst/benchmarks/HashBenchmark-jdk21-results.txt
+++ b/sql/catalyst/benchmarks/HashBenchmark-jdk21-results.txt
@@ -2,69 +2,69 @@
single ints
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Hash For single ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-interpreted version 2093 2095 3 256.5 3.9 1.0X
-codegen version 3536 3540 6 151.8 6.6 0.6X
-codegen version 64-bit 3124 3124 1 171.9 5.8 0.7X
-codegen HiveHash version 2560 2561 2 209.7 4.8 0.8X
+interpreted version 2095 2098 4 256.2 3.9 1.0X
+codegen version 3534 3537 5 151.9 6.6 0.6X
+codegen version 64-bit 3123 3127 5 171.9 5.8 0.7X
+codegen HiveHash version 2563 2569 8 209.4 4.8 0.8X
================================================================================================
single longs
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Hash For single longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-interpreted version 2707 2710 4 198.3 5.0 1.0X
-codegen version 4519 4520 2 118.8 8.4 0.6X
-codegen version 64-bit 3833 3842 13 140.0 7.1 0.7X
-codegen HiveHash version 3138 3145 9 171.1 5.8 0.9X
+interpreted version 2712 2713 1 197.9 5.1 1.0X
+codegen version 4518 4521 4 118.8 8.4 0.6X
+codegen version 64-bit 3658 3659 0 146.8 6.8 0.7X
+codegen HiveHash version 3139 3140 2 171.0 5.8 0.9X
================================================================================================
normal
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Hash For normal: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-interpreted version 1414 1415 2 1.5 674.3 1.0X
-codegen version 1905 1906 1 1.1 908.5 0.7X
-codegen version 64-bit 749 753 3 2.8 357.3 1.9X
-codegen HiveHash version 3833 3837 5 0.5 1827.9 0.4X
+interpreted version 1342 1344 3 1.6 640.1 1.0X
+codegen version 1858 1860 2 1.1 886.2 0.7X
+codegen version 64-bit 731 731 1 2.9 348.4 1.8X
+codegen HiveHash version 3704 3706 2 0.6 1766.4 0.4X
================================================================================================
array
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Hash For array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-interpreted version 1081 1083 2 0.1 8249.1 1.0X
-codegen version 3788 3788 1 0.0 28896.9 0.3X
-codegen version 64-bit 2596 2601 6 0.1 19805.4 0.4X
-codegen HiveHash version 761 762 2 0.2 5807.2 1.4X
+interpreted version 965 966 1 0.1 7363.6 1.0X
+codegen version 3439 3440 1 0.0 26238.1 0.3X
+codegen version 64-bit 2363 2364 1 0.1 18030.0 0.4X
+codegen HiveHash version 695 696 1 0.2 5304.0 1.4X
================================================================================================
map
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Hash For map: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-interpreted version 0 0 0 91.2 11.0 1.0X
-codegen version 260 263 3 0.0 63355.0 0.0X
-codegen version 64-bit 177 179 3 0.0 43169.7 0.0X
-codegen HiveHash version 29 29 0 0.1 7041.8 0.0X
+interpreted version 0 0 0 85.0 11.8 1.0X
+codegen version 236 236 0 0.0 57574.4 0.0X
+codegen version 64-bit 161 161 0 0.0 39306.9 0.0X
+codegen HiveHash version 26 26 0 0.2 6395.2 0.0X
diff --git a/sql/catalyst/benchmarks/HashBenchmark-results.txt b/sql/catalyst/benchmarks/HashBenchmark-results.txt
index b1dbc78213bca..8aa274e1a510a 100644
--- a/sql/catalyst/benchmarks/HashBenchmark-results.txt
+++ b/sql/catalyst/benchmarks/HashBenchmark-results.txt
@@ -2,69 +2,69 @@
single ints
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Hash For single ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-interpreted version 2179 2180 1 246.4 4.1 1.0X
-codegen version 3670 3678 11 146.3 6.8 0.6X
-codegen version 64-bit 3423 3426 4 156.8 6.4 0.6X
-codegen HiveHash version 2864 2865 1 187.5 5.3 0.8X
+interpreted version 2174 2177 4 246.9 4.0 1.0X
+codegen version 3672 3675 4 146.2 6.8 0.6X
+codegen version 64-bit 3423 3427 5 156.8 6.4 0.6X
+codegen HiveHash version 2857 2857 1 187.9 5.3 0.8X
================================================================================================
single longs
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Hash For single longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-interpreted version 3019 3021 3 177.8 5.6 1.0X
-codegen version 5477 5480 4 98.0 10.2 0.6X
-codegen version 64-bit 3755 3759 5 143.0 7.0 0.8X
-codegen HiveHash version 3319 3326 9 161.7 6.2 0.9X
+interpreted version 3018 3019 2 177.9 5.6 1.0X
+codegen version 5238 5240 2 102.5 9.8 0.6X
+codegen version 64-bit 3762 3765 4 142.7 7.0 0.8X
+codegen HiveHash version 3342 3342 0 160.6 6.2 0.9X
================================================================================================
normal
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Hash For normal: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-interpreted version 2487 2488 1 0.8 1186.0 1.0X
-codegen version 2253 2255 2 0.9 1074.5 1.1X
-codegen version 64-bit 705 705 1 3.0 336.0 3.5X
-codegen HiveHash version 3671 3675 6 0.6 1750.3 0.7X
+interpreted version 2617 2618 2 0.8 1248.0 1.0X
+codegen version 2296 2296 0 0.9 1094.8 1.1X
+codegen version 64-bit 704 706 1 3.0 335.9 3.7X
+codegen HiveHash version 3782 3786 5 0.6 1803.6 0.7X
================================================================================================
array
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Hash For array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-interpreted version 1042 1045 4 0.1 7947.6 1.0X
-codegen version 3728 3735 10 0.0 28442.1 0.3X
-codegen version 64-bit 2487 2499 18 0.1 18971.5 0.4X
-codegen HiveHash version 775 776 1 0.2 5910.2 1.3X
+interpreted version 1015 1021 8 0.1 7747.0 1.0X
+codegen version 3673 3676 4 0.0 28025.1 0.3X
+codegen version 64-bit 2453 2454 1 0.1 18717.5 0.4X
+codegen HiveHash version 781 782 1 0.2 5961.0 1.3X
================================================================================================
map
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Hash For map: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-interpreted version 0 0 0 97.0 10.3 1.0X
-codegen version 262 262 1 0.0 63915.5 0.0X
-codegen version 64-bit 177 177 0 0.0 43098.3 0.0X
-codegen HiveHash version 29 29 1 0.1 7091.4 0.0X
+interpreted version 0 0 0 97.3 10.3 1.0X
+codegen version 256 256 0 0.0 62469.7 0.0X
+codegen version 64-bit 181 182 1 0.0 44237.6 0.0X
+codegen HiveHash version 30 30 0 0.1 7329.7 0.0X
diff --git a/sql/catalyst/benchmarks/HashByteArrayBenchmark-jdk21-results.txt b/sql/catalyst/benchmarks/HashByteArrayBenchmark-jdk21-results.txt
index 7c326a530dcf9..e5a5f508343eb 100644
--- a/sql/catalyst/benchmarks/HashByteArrayBenchmark-jdk21-results.txt
+++ b/sql/catalyst/benchmarks/HashByteArrayBenchmark-jdk21-results.txt
@@ -2,76 +2,76 @@
Benchmark for MurMurHash 3 and xxHash64
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Hash byte arrays with length 8: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Murmur3_x86_32 10 10 0 206.4 4.8 1.0X
-xxHash 64-bit 10 10 0 200.7 5.0 1.0X
-HiveHasher 14 14 0 151.5 6.6 0.7X
+Murmur3_x86_32 11 11 0 196.5 5.1 1.0X
+xxHash 64-bit 11 11 0 199.5 5.0 1.0X
+HiveHasher 14 14 1 150.9 6.6 0.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Hash byte arrays with length 16: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
Murmur3_x86_32 14 15 0 145.5 6.9 1.0X
-xxHash 64-bit 12 13 0 168.0 6.0 1.2X
-HiveHasher 23 23 0 90.8 11.0 0.6X
+xxHash 64-bit 13 13 0 167.2 6.0 1.1X
+HiveHasher 23 23 0 91.0 11.0 0.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Hash byte arrays with length 24: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
Murmur3_x86_32 20 20 0 104.3 9.6 1.0X
-xxHash 64-bit 14 15 0 144.9 6.9 1.4X
-HiveHasher 33 33 0 62.7 15.9 0.6X
+xxHash 64-bit 14 15 0 144.8 6.9 1.4X
+HiveHasher 33 34 1 62.7 15.9 0.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Hash byte arrays with length 31: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Murmur3_x86_32 35 35 0 60.6 16.5 1.0X
-xxHash 64-bit 27 27 0 76.9 13.0 1.3X
-HiveHasher 44 44 1 48.2 20.8 0.8X
+Murmur3_x86_32 32 32 0 65.5 15.3 1.0X
+xxHash 64-bit 27 27 0 76.9 13.0 1.2X
+HiveHasher 44 44 0 48.2 20.8 0.7X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Hash byte arrays with length 95: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Murmur3_x86_32 82 82 1 25.5 39.1 1.0X
-xxHash 64-bit 57 58 0 36.5 27.4 1.4X
-HiveHasher 157 157 0 13.4 74.8 0.5X
+Murmur3_x86_32 70 71 0 29.8 33.6 1.0X
+xxHash 64-bit 57 58 0 36.5 27.4 1.2X
+HiveHasher 157 157 0 13.4 74.8 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Hash byte arrays with length 287: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Murmur3_x86_32 224 224 0 9.4 106.6 1.0X
-xxHash 64-bit 102 102 0 20.6 48.4 2.2X
-HiveHasher 532 534 1 3.9 253.9 0.4X
+Murmur3_x86_32 197 198 0 10.6 94.1 1.0X
+xxHash 64-bit 102 102 0 20.6 48.6 1.9X
+HiveHasher 532 533 0 3.9 253.9 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Hash byte arrays with length 1055: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Murmur3_x86_32 803 803 0 2.6 382.7 1.0X
-xxHash 64-bit 294 294 1 7.1 140.1 2.7X
-HiveHasher 2043 2044 0 1.0 974.4 0.4X
+Murmur3_x86_32 710 711 1 3.0 338.6 1.0X
+xxHash 64-bit 293 293 0 7.2 139.7 2.4X
+HiveHasher 2042 2043 0 1.0 973.8 0.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Hash byte arrays with length 2079: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Murmur3_x86_32 1597 1597 0 1.3 761.3 1.0X
-xxHash 64-bit 550 551 0 3.8 262.4 2.9X
-HiveHasher 4047 4047 0 0.5 1929.6 0.4X
+Murmur3_x86_32 1387 1387 0 1.5 661.3 1.0X
+xxHash 64-bit 550 550 0 3.8 262.1 2.5X
+HiveHasher 4047 4048 0 0.5 1929.9 0.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Hash byte arrays with length 8223: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Murmur3_x86_32 6287 6290 4 0.3 2997.9 1.0X
-xxHash 64-bit 2071 2071 0 1.0 987.4 3.0X
-HiveHasher 16084 16086 3 0.1 7669.3 0.4X
+Murmur3_x86_32 5727 5729 2 0.4 2730.9 1.0X
+xxHash 64-bit 2066 2066 1 1.0 985.1 2.8X
+HiveHasher 16085 16086 1 0.1 7669.9 0.4X
diff --git a/sql/catalyst/benchmarks/HashByteArrayBenchmark-results.txt b/sql/catalyst/benchmarks/HashByteArrayBenchmark-results.txt
index 45b4e8ac70025..a7e8c45f4f42e 100644
--- a/sql/catalyst/benchmarks/HashByteArrayBenchmark-results.txt
+++ b/sql/catalyst/benchmarks/HashByteArrayBenchmark-results.txt
@@ -2,76 +2,76 @@
Benchmark for MurMurHash 3 and xxHash64
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Hash byte arrays with length 8: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Murmur3_x86_32 11 11 0 183.1 5.5 1.0X
+Murmur3_x86_32 11 12 0 183.1 5.5 1.0X
xxHash 64-bit 10 10 0 213.5 4.7 1.2X
-HiveHasher 14 14 0 150.0 6.7 0.8X
+HiveHasher 14 14 0 149.9 6.7 0.8X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Hash byte arrays with length 16: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Murmur3_x86_32 17 17 0 123.2 8.1 1.0X
-xxHash 64-bit 12 12 0 175.4 5.7 1.4X
+Murmur3_x86_32 17 17 0 122.9 8.1 1.0X
+xxHash 64-bit 12 12 0 175.2 5.7 1.4X
HiveHasher 23 23 0 93.0 10.8 0.8X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Hash byte arrays with length 24: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Murmur3_x86_32 24 24 2 89.1 11.2 1.0X
-xxHash 64-bit 14 14 0 145.3 6.9 1.6X
-HiveHasher 33 34 2 63.0 15.9 0.7X
+Murmur3_x86_32 24 24 0 89.1 11.2 1.0X
+xxHash 64-bit 14 15 0 145.1 6.9 1.6X
+HiveHasher 33 33 0 63.0 15.9 0.7X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Hash byte arrays with length 31: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Murmur3_x86_32 37 37 0 56.2 17.8 1.0X
-xxHash 64-bit 27 28 0 76.4 13.1 1.4X
-HiveHasher 43 43 0 49.2 20.3 0.9X
+Murmur3_x86_32 36 36 0 57.9 17.3 1.0X
+xxHash 64-bit 28 28 0 75.5 13.2 1.3X
+HiveHasher 43 43 0 49.2 20.3 0.8X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Hash byte arrays with length 95: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Murmur3_x86_32 88 88 0 23.9 41.8 1.0X
-xxHash 64-bit 64 64 0 32.9 30.4 1.4X
-HiveHasher 158 160 4 13.2 75.6 0.6X
+Murmur3_x86_32 87 87 0 24.0 41.6 1.0X
+xxHash 64-bit 65 65 0 32.5 30.8 1.3X
+HiveHasher 158 159 0 13.2 75.5 0.6X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Hash byte arrays with length 287: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Murmur3_x86_32 244 244 0 8.6 116.5 1.0X
-xxHash 64-bit 123 123 0 17.1 58.5 2.0X
+Murmur3_x86_32 244 244 0 8.6 116.4 1.0X
+xxHash 64-bit 124 125 1 16.9 59.2 2.0X
HiveHasher 534 534 0 3.9 254.8 0.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Hash byte arrays with length 1055: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Murmur3_x86_32 878 878 0 2.4 418.7 1.0X
-xxHash 64-bit 401 402 1 5.2 191.1 2.2X
-HiveHasher 2044 2044 0 1.0 974.7 0.4X
+Murmur3_x86_32 879 879 0 2.4 418.9 1.0X
+xxHash 64-bit 401 401 0 5.2 191.2 2.2X
+HiveHasher 2045 2045 0 1.0 975.0 0.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Hash byte arrays with length 2079: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Murmur3_x86_32 1718 1724 9 1.2 819.2 1.0X
-xxHash 64-bit 781 781 0 2.7 372.4 2.2X
-HiveHasher 4050 4052 3 0.5 1931.4 0.4X
+Murmur3_x86_32 1714 1715 0 1.2 817.5 1.0X
+xxHash 64-bit 783 784 1 2.7 373.5 2.2X
+HiveHasher 4050 4051 1 0.5 1931.4 0.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Hash byte arrays with length 8223: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Murmur3_x86_32 6746 6747 2 0.3 3216.6 1.0X
-xxHash 64-bit 3042 3044 2 0.7 1450.7 2.2X
-HiveHasher 16080 16081 3 0.1 7667.3 0.4X
+Murmur3_x86_32 6745 6745 0 0.3 3216.1 1.0X
+xxHash 64-bit 3046 3046 0 0.7 1452.4 2.2X
+HiveHasher 16085 16086 2 0.1 7669.9 0.4X
diff --git a/sql/catalyst/benchmarks/HexBenchmark-jdk21-results.txt b/sql/catalyst/benchmarks/HexBenchmark-jdk21-results.txt
index afad7d3f6c6f4..238fe92dd9593 100644
--- a/sql/catalyst/benchmarks/HexBenchmark-jdk21-results.txt
+++ b/sql/catalyst/benchmarks/HexBenchmark-jdk21-results.txt
@@ -2,13 +2,13 @@
UnHex Comparison
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Cardinality 1000000: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Common Codecs 4735 4800 106 0.2 4734.6 1.0X
-Java 4008 4011 3 0.2 4008.3 1.2X
-Spark 3493 3519 44 0.3 3492.8 1.4X
-Spark Binary 2641 2648 11 0.4 2641.3 1.8X
+Common Codecs 4701 4786 118 0.2 4700.5 1.0X
+Java 3962 3969 10 0.3 3962.4 1.2X
+Spark 3492 3494 2 0.3 3491.6 1.3X
+Spark Binary 2638 2638 0 0.4 2637.8 1.8X
diff --git a/sql/catalyst/benchmarks/HexBenchmark-results.txt b/sql/catalyst/benchmarks/HexBenchmark-results.txt
index ebb1fb05aef2c..796f48468187d 100644
--- a/sql/catalyst/benchmarks/HexBenchmark-results.txt
+++ b/sql/catalyst/benchmarks/HexBenchmark-results.txt
@@ -2,13 +2,13 @@
UnHex Comparison
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Cardinality 1000000: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Common Codecs 4537 4561 21 0.2 4536.8 1.0X
-Java 3137 3142 6 0.3 3137.5 1.4X
-Spark 3793 3799 6 0.3 3793.3 1.2X
-Spark Binary 2755 2758 2 0.4 2755.5 1.6X
+Common Codecs 4821 4894 64 0.2 4820.6 1.0X
+Java 2565 2572 10 0.4 2564.8 1.9X
+Spark 3811 3812 1 0.3 3810.7 1.3X
+Spark Binary 2758 2759 1 0.4 2757.9 1.7X
diff --git a/sql/catalyst/benchmarks/InternalRowComparableWrapperBenchmark-jdk21-results.txt b/sql/catalyst/benchmarks/InternalRowComparableWrapperBenchmark-jdk21-results.txt
index 754895dda89b4..2ae65e2b4a716 100644
--- a/sql/catalyst/benchmarks/InternalRowComparableWrapperBenchmark-jdk21-results.txt
+++ b/sql/catalyst/benchmarks/InternalRowComparableWrapperBenchmark-jdk21-results.txt
@@ -1,7 +1,7 @@
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
internal row comparable wrapper: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-toSet 100 103 3 2.0 499.8 1.0X
-mergePartitions 181 184 3 1.1 907.3 0.6X
+toSet 81 84 2 2.5 406.5 1.0X
+mergePartitions 139 141 2 1.4 695.7 0.6X
diff --git a/sql/catalyst/benchmarks/InternalRowComparableWrapperBenchmark-results.txt b/sql/catalyst/benchmarks/InternalRowComparableWrapperBenchmark-results.txt
index ebb7cb08939ed..6ce0c60a96565 100644
--- a/sql/catalyst/benchmarks/InternalRowComparableWrapperBenchmark-results.txt
+++ b/sql/catalyst/benchmarks/InternalRowComparableWrapperBenchmark-results.txt
@@ -1,7 +1,7 @@
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
internal row comparable wrapper: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-toSet 116 118 2 1.7 581.5 1.0X
-mergePartitions 208 211 3 1.0 1040.8 0.6X
+toSet 114 117 3 1.8 568.6 1.0X
+mergePartitions 212 213 1 0.9 1059.5 0.5X
diff --git a/sql/catalyst/benchmarks/UnsafeProjectionBenchmark-jdk21-results.txt b/sql/catalyst/benchmarks/UnsafeProjectionBenchmark-jdk21-results.txt
index 7cf3689a033d6..e20ffc3ffec3a 100644
--- a/sql/catalyst/benchmarks/UnsafeProjectionBenchmark-jdk21-results.txt
+++ b/sql/catalyst/benchmarks/UnsafeProjectionBenchmark-jdk21-results.txt
@@ -2,13 +2,13 @@
unsafe projection
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
unsafe projection: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-single long 1331 1333 2 201.6 5.0 1.0X
-single nullable long 2369 2380 16 113.3 8.8 0.6X
-7 primitive types 7127 7128 2 37.7 26.5 0.2X
-7 nullable primitive types 10678 10678 0 25.1 39.8 0.1X
+single long 1331 1332 1 201.7 5.0 1.0X
+single nullable long 2369 2412 61 113.3 8.8 0.6X
+7 primitive types 7149 7154 8 37.5 26.6 0.2X
+7 nullable primitive types 10649 10652 4 25.2 39.7 0.1X
diff --git a/sql/catalyst/benchmarks/UnsafeProjectionBenchmark-results.txt b/sql/catalyst/benchmarks/UnsafeProjectionBenchmark-results.txt
index b2f82a851689f..b1c8e9a375809 100644
--- a/sql/catalyst/benchmarks/UnsafeProjectionBenchmark-results.txt
+++ b/sql/catalyst/benchmarks/UnsafeProjectionBenchmark-results.txt
@@ -2,13 +2,13 @@
unsafe projection
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
unsafe projection: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-single long 1376 1377 2 195.1 5.1 1.0X
-single nullable long 2447 2449 3 109.7 9.1 0.6X
-7 primitive types 7004 7011 10 38.3 26.1 0.2X
-7 nullable primitive types 10357 10363 7 25.9 38.6 0.1X
+single long 1377 1378 0 194.9 5.1 1.0X
+single nullable long 2385 2420 50 112.5 8.9 0.6X
+7 primitive types 7018 7019 2 38.3 26.1 0.2X
+7 nullable primitive types 10399 10400 1 25.8 38.7 0.1X
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ArrayOfDecimalsSerDe.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ArrayOfDecimalsSerDe.java
new file mode 100644
index 0000000000000..3a445fea56906
--- /dev/null
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ArrayOfDecimalsSerDe.java
@@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 java.math.BigDecimal;
+import java.math.BigInteger;
+import java.util.Objects;
+
+import org.apache.datasketches.common.ArrayOfItemsSerDe;
+import org.apache.datasketches.common.ArrayOfLongsSerDe;
+import org.apache.datasketches.common.ArrayOfNumbersSerDe;
+import org.apache.datasketches.common.Util;
+import org.apache.datasketches.memory.Memory;
+
+import org.apache.spark.sql.types.Decimal;
+import org.apache.spark.sql.types.DecimalType;
+
+import static org.apache.datasketches.common.ByteArrayUtil.copyBytes;
+import static org.apache.datasketches.common.ByteArrayUtil.putIntLE;
+
+public class ArrayOfDecimalsSerDe extends ArrayOfItemsSerDe {
+
+ private final int precision;
+ private final int scale;
+ private final DecimalType decimalType;
+ private final ArrayOfItemsSerDe> delegate;
+
+ public ArrayOfDecimalsSerDe(DecimalType decimalType) {
+ this.decimalType = decimalType;
+ this.precision = decimalType.precision();
+ this.scale = decimalType.scale();
+
+ if (DecimalType.is32BitDecimalType(decimalType)) {
+ this.delegate = new ArrayOfNumbersSerDe();
+ } else if (DecimalType.is64BitDecimalType(decimalType)) {
+ this.delegate = new ArrayOfLongsSerDe();
+ } else {
+ this.delegate = new ArrayOfDecimalByteArrSerDe(decimalType);
+ }
+ }
+
+ @Override
+ public byte[] serializeToByteArray(Decimal item) {
+ Objects.requireNonNull(item, "Item must not be null");
+ if (DecimalType.is32BitDecimalType(decimalType)) {
+ return ((ArrayOfNumbersSerDe) delegate).serializeToByteArray(decimalToInt(item));
+ } else if (DecimalType.is64BitDecimalType(decimalType)) {
+ return ((ArrayOfLongsSerDe) delegate).serializeToByteArray(item.toUnscaledLong());
+ } else {
+ return ((ArrayOfDecimalByteArrSerDe) delegate).serializeToByteArray(item);
+ }
+ }
+
+ @Override
+ public byte[] serializeToByteArray(Decimal[] items) {
+ Objects.requireNonNull(items, "Item must not be null");
+ if (DecimalType.is32BitDecimalType(decimalType)) {
+ Number[] intItems = new Number[items.length];
+ for (int i = 0; i < items.length; i++) {
+ intItems[i] = decimalToInt(items[i]);
+ }
+ return ((ArrayOfNumbersSerDe) delegate).serializeToByteArray(intItems);
+ } else if (DecimalType.is64BitDecimalType(decimalType)) {
+ Long[] longItems = new Long[items.length];
+ for (int i = 0; i < items.length; i++) {
+ longItems[i] = items[i].toUnscaledLong();
+ }
+ return ((ArrayOfLongsSerDe) delegate).serializeToByteArray(longItems);
+ } else {
+ return ((ArrayOfDecimalByteArrSerDe) delegate).serializeToByteArray(items);
+ }
+ }
+
+ @Override
+ public Decimal[] deserializeFromMemory(Memory mem, long offsetBytes, int numItems) {
+ Objects.requireNonNull(mem, "Memory must not be null");
+ if (DecimalType.is32BitDecimalType(decimalType)) {
+ Number[] intArray = ((ArrayOfNumbersSerDe) delegate)
+ .deserializeFromMemory(mem, offsetBytes, numItems);
+ Decimal[] result = new Decimal[intArray.length];
+ for (int i = 0; i < intArray.length; i++) {
+ result[i] = Decimal.createUnsafe((int) intArray[i], precision, scale);
+ }
+ return result;
+ } else if (DecimalType.is64BitDecimalType(decimalType)) {
+ Long[] longArray = ((ArrayOfLongsSerDe) delegate)
+ .deserializeFromMemory(mem, offsetBytes, numItems);
+ Decimal[] result = new Decimal[longArray.length];
+ for (int i = 0; i < longArray.length; i++) {
+ result[i] = Decimal.createUnsafe(longArray[i], precision, scale);
+ }
+ return result;
+ } else {
+ return ((ArrayOfDecimalByteArrSerDe) delegate)
+ .deserializeFromMemory(mem, offsetBytes, numItems);
+ }
+ }
+
+ @Override
+ public int sizeOf(Decimal item) {
+ Objects.requireNonNull(item, "Item must not be null");
+ if (DecimalType.is32BitDecimalType(decimalType)) {
+ return ((ArrayOfNumbersSerDe) delegate).sizeOf(decimalToInt(item));
+ } else if (DecimalType.is64BitDecimalType(decimalType)) {
+ return ((ArrayOfLongsSerDe) delegate).sizeOf(item.toUnscaledLong());
+ } else {
+ return ((ArrayOfDecimalByteArrSerDe) delegate).sizeOf(item);
+ }
+ }
+
+ @Override
+ public int sizeOf(Memory mem, long offsetBytes, int numItems) {
+ Objects.requireNonNull(mem, "Memory must not be null");
+ return delegate.sizeOf(mem, offsetBytes, numItems);
+ }
+
+ @Override
+ public String toString(Decimal item) {
+ if (item == null) {
+ return "null";
+ }
+ return item.toString();
+ }
+
+ @Override
+ public Class getClassOfT() {
+ return Decimal.class;
+ }
+
+ private int decimalToInt(Decimal item) {
+ return ((int) item.toUnscaledLong());
+ }
+
+
+ /**
+ * Serialize and deserialize Decimal as byte array.
+ */
+ private static class ArrayOfDecimalByteArrSerDe extends ArrayOfItemsSerDe {
+ private final int precision;
+ private final int scale;
+
+ ArrayOfDecimalByteArrSerDe(DecimalType decimalType) {
+ assert DecimalType.isByteArrayDecimalType(decimalType);
+ this.precision = decimalType.precision();
+ this.scale = decimalType.scale();
+ }
+
+ @Override
+ public byte[] serializeToByteArray(Decimal item) {
+ Objects.requireNonNull(item, "Item must not be null");
+ final byte[] decimalByteArr = item.toJavaBigDecimal().unscaledValue().toByteArray();
+ final int numBytes = decimalByteArr.length;
+ final byte[] out = new byte[numBytes + Integer.BYTES];
+ copyBytes(decimalByteArr, 0, out, 4, numBytes);
+ putIntLE(out, 0, numBytes);
+ return out;
+ }
+
+ @Override
+ public byte[] serializeToByteArray(Decimal[] items) {
+ Objects.requireNonNull(items, "Items must not be null");
+ if (items.length == 0) {
+ return new byte[0];
+ }
+ int totalBytes = 0;
+ final int numItems = items.length;
+ final byte[][] serialized2DArray = new byte[numItems][];
+ for (int i = 0; i < numItems; i++) {
+ serialized2DArray[i] = items[i].toJavaBigDecimal().unscaledValue().toByteArray();
+ totalBytes += serialized2DArray[i].length + Integer.BYTES;
+ }
+ final byte[] bytesOut = new byte[totalBytes];
+ int offset = 0;
+ for (int i = 0; i < numItems; i++) {
+ final int decimalLen = serialized2DArray[i].length;
+ putIntLE(bytesOut, offset, decimalLen);
+ offset += Integer.BYTES;
+ copyBytes(serialized2DArray[i], 0, bytesOut, offset, decimalLen);
+ offset += decimalLen;
+ }
+ return bytesOut;
+ }
+
+ @Override
+ public Decimal[] deserializeFromMemory(Memory mem, long offsetBytes, int numItems) {
+ Objects.requireNonNull(mem, "Memory must not be null");
+ if (numItems <= 0) {
+ return new Decimal[0];
+ }
+ final Decimal[] array = new Decimal[numItems];
+ long offset = offsetBytes;
+ for (int i = 0; i < numItems; i++) {
+ Util.checkBounds(offset, Integer.BYTES, mem.getCapacity());
+ final int decimalLength = mem.getInt(offset);
+ offset += Integer.BYTES;
+ final byte[] decimalBytes = new byte[decimalLength];
+ Util.checkBounds(offset, decimalLength, mem.getCapacity());
+ mem.getByteArray(offset, decimalBytes, 0, decimalLength);
+ offset += decimalLength;
+ BigInteger bigInteger = new BigInteger(decimalBytes);
+ BigDecimal javaDecimal = new BigDecimal(bigInteger, scale);
+ array[i] = Decimal.apply(javaDecimal, precision, scale);
+ }
+ return array;
+ }
+
+ @Override
+ public int sizeOf(Decimal item) {
+ Objects.requireNonNull(item, "Item must not be null");
+ return item.toJavaBigDecimal().unscaledValue().toByteArray().length + Integer.BYTES;
+ }
+
+ @Override
+ public int sizeOf(Memory mem, long offsetBytes, int numItems) {
+ Objects.requireNonNull(mem, "Memory must not be null");
+ if (numItems <= 0) {
+ return 0;
+ }
+ long offset = offsetBytes;
+ final long memCap = mem.getCapacity();
+ for (int i = 0; i < numItems; i++) {
+ Util.checkBounds(offset, Integer.BYTES, memCap);
+ final int itemLenBytes = mem.getInt(offset);
+ offset += Integer.BYTES;
+ Util.checkBounds(offset, itemLenBytes, memCap);
+ offset += itemLenBytes;
+ }
+ return (int) (offset - offsetBytes);
+ }
+
+ @Override
+ public String toString(Decimal item) {
+ if (item == null) {
+ return "null";
+ }
+ return item.toString();
+ }
+
+ @Override
+ public Class getClassOfT() {
+ return Decimal.class;
+ }
+ }
+}
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatch.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatch.java
index 5074348a1fd6a..4a041b70cf592 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatch.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatch.java
@@ -129,7 +129,7 @@ private boolean acquirePage(long requiredSize) {
page = allocatePage(requiredSize);
} catch (SparkOutOfMemoryError e) {
logger.warn("Failed to allocate page ({} bytes).",
- MDC.of(LogKeys.PAGE_SIZE$.MODULE$, requiredSize));
+ MDC.of(LogKeys.PAGE_SIZE, requiredSize));
return false;
}
base = page.getBaseObject();
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java
index f9a75ccd1c8da..3a1e0d9f7011e 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java
@@ -20,6 +20,7 @@
import org.apache.spark.annotation.Evolving;
import org.apache.spark.sql.connector.catalog.constraints.Constraint;
import org.apache.spark.sql.connector.expressions.Transform;
+import org.apache.spark.sql.errors.QueryCompilationErrors;
import org.apache.spark.sql.types.StructType;
import java.util.Collections;
@@ -56,7 +57,9 @@ public interface Table {
* @deprecated This is deprecated. Please override {@link #columns} instead.
*/
@Deprecated(since = "3.4.0")
- StructType schema();
+ default StructType schema() {
+ throw QueryCompilationErrors.mustOverrideOneMethodError("columns");
+ }
/**
* Returns the columns of this table. If the table is not readable and doesn't have a schema, an
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCapability.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCapability.java
index 5732c0f3af4ee..0a01c0c266b9a 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCapability.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCapability.java
@@ -93,6 +93,12 @@ public enum TableCapability {
*/
ACCEPT_ANY_SCHEMA,
+ /**
+ * Signals that table supports Spark altering the schema if necessary
+ * as part of an operation.
+ */
+ AUTOMATIC_SCHEMA_EVOLUTION,
+
/**
* Signals that the table supports append writes using the V1 InsertableRelation interface.
*
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableInfo.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableInfo.java
index 8dc71c5aee472..a5b4e333afa87 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableInfo.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableInfo.java
@@ -16,14 +16,14 @@
*/
package org.apache.spark.sql.connector.catalog;
-import static com.google.common.base.Preconditions.checkNotNull;
-import com.google.common.collect.Maps;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
import org.apache.spark.sql.connector.catalog.constraints.Constraint;
import org.apache.spark.sql.connector.expressions.Transform;
import org.apache.spark.sql.types.StructType;
-import java.util.Map;
-
public class TableInfo {
private final Column[] columns;
@@ -62,7 +62,7 @@ public Transform[] partitions() {
public static class Builder {
private Column[] columns;
- private Map properties = Maps.newHashMap();
+ private Map properties = new HashMap<>();
private Transform[] partitions = new Transform[0];
private Constraint[] constraints = new Constraint[0];
@@ -87,7 +87,7 @@ public Builder withConstraints(Constraint[] constraints) {
}
public TableInfo build() {
- checkNotNull(columns, "columns should not be null");
+ Objects.requireNonNull(columns, "columns should not be null");
return new TableInfo(this);
}
}
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableSummary.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableSummary.java
index e08f83b98319e..8f46a372342a8 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableSummary.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableSummary.java
@@ -17,9 +17,9 @@
package org.apache.spark.sql.connector.catalog;
-import org.apache.spark.annotation.Evolving;
+import java.util.Objects;
-import static com.google.common.base.Preconditions.checkNotNull;
+import org.apache.spark.annotation.Evolving;
@Evolving
public interface TableSummary {
@@ -38,7 +38,7 @@ static TableSummary of(Identifier identifier, String tableType) {
record TableSummaryImpl(Identifier identifier, String tableType) implements TableSummary {
TableSummaryImpl {
- checkNotNull(identifier, "Identifier of a table summary object cannot be null");
- checkNotNull(tableType, "Table type of a table summary object cannot be null");
+ Objects.requireNonNull(identifier, "Identifier of a table summary object cannot be null");
+ Objects.requireNonNull(tableType, "Table type of a table summary object cannot be null");
}
}
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/constraints/BaseConstraint.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/constraints/BaseConstraint.java
index 28791a9f3a58f..f93d716a27842 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/constraints/BaseConstraint.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/constraints/BaseConstraint.java
@@ -63,15 +63,26 @@ public boolean rely() {
@Override
public String toDDL() {
+ // The validation status is not included in the DDL output as it's not part of
+ // the Spark SQL syntax for constraints.
return String.format(
- "CONSTRAINT %s %s %s %s %s",
+ "CONSTRAINT %s %s %s %s",
name,
definition(),
enforced ? "ENFORCED" : "NOT ENFORCED",
- validationStatus,
rely ? "RELY" : "NORELY");
}
+ public String toDescription() {
+ StringJoiner joiner = new StringJoiner(" ");
+ joiner.add(definition());
+ joiner.add(enforced ? "ENFORCED" : "NOT ENFORCED");
+ if (rely) {
+ joiner.add("RELY");
+ }
+ return joiner.toString();
+ }
+
@Override
public String toString() {
return toDDL();
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/constraints/Constraint.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/constraints/Constraint.java
index c3a2cd73e9abe..93ffb22422e60 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/constraints/Constraint.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/constraints/Constraint.java
@@ -58,6 +58,12 @@ public interface Constraint {
*/
String toDDL();
+ /**
+ * Returns the constraint description for DESCRIBE TABLE output, excluding the constraint
+ * name (shown separately).
+ */
+ String toDescription();
+
/**
* Instantiates a builder for a CHECK constraint.
*
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/filter/Predicate.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/filter/Predicate.java
index e58cddc274c5f..65f2bd88940a3 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/filter/Predicate.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/filter/Predicate.java
@@ -136,6 +136,12 @@
*
Since version: 3.3.0
*
*
+ *
Name: BOOLEAN_EXPRESSION
+ *
+ *
A simple wrapper for any expression that returns boolean type.
+ *
Since version: 4.1.0
+ *
+ *
*
*
* @since 3.3.0
@@ -145,5 +151,8 @@ public class Predicate extends GeneralScalarExpression {
public Predicate(String name, Expression[] children) {
super(name, children);
+ if ("BOOLEAN_EXPRESSION".equals(name)) {
+ assert children.length == 1;
+ }
}
}
diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/RetriesExceeded.scala b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/join/JoinType.java
similarity index 76%
rename from sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/RetriesExceeded.scala
rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/join/JoinType.java
index 77e1c0deab24f..56fb0a51c243a 100644
--- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/RetriesExceeded.scala
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/join/JoinType.java
@@ -15,11 +15,18 @@
* limitations under the License.
*/
-package org.apache.spark.sql.connect.client
+package org.apache.spark.sql.connector.join;
+
+import org.apache.spark.annotation.Evolving;
/**
- * Represents an exception which was considered retriable but has exceeded retry limits.
+ * Enum representing the join type in public API.
*
- * The actual exceptions incurred can be retrieved with getSuppressed()
+ * @since 4.1.0
*/
-class RetriesExceeded extends Throwable
+@Evolving
+public enum JoinType {
+ INNER_JOIN,
+ LEFT_OUTER_JOIN,
+ RIGHT_OUTER_JOIN,
+}
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownJoin.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownJoin.java
new file mode 100644
index 0000000000000..a48a78671922a
--- /dev/null
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownJoin.java
@@ -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.connector.read;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.connector.expressions.filter.Predicate;
+import org.apache.spark.sql.connector.join.JoinType;
+
+/**
+ * A mix-in interface for {@link ScanBuilder}. Data sources can implement this interface to
+ * push down join operators.
+ *
+ * @since 4.1.0
+ */
+@Evolving
+public interface SupportsPushDownJoin extends ScanBuilder {
+ /**
+ * Returns true if the other side of the join is compatible with the
+ * current {@code SupportsPushDownJoin} for a join push down, meaning both sides can be
+ * processed together within the same underlying data source.
+ *
+ *
+ * For example, JDBC connectors are compatible if they use the same
+ * host, port, username, and password.
+ */
+ boolean isOtherSideCompatibleForJoin(SupportsPushDownJoin other);
+
+ /**
+ * Pushes down the join of the current {@code SupportsPushDownJoin} and the other side of join
+ * {@code SupportsPushDownJoin}.
+ *
+ * @param other {@code SupportsPushDownJoin} that this {@code SupportsPushDownJoin}
+ * gets joined with.
+ * @param joinType the type of join.
+ * @param leftSideRequiredColumnsWithAliases required output of the
+ * left side {@code SupportsPushDownJoin}
+ * @param rightSideRequiredColumnsWithAliases required output of the
+ * right side {@code SupportsPushDownJoin}
+ * @param condition join condition. Columns are named after the specified aliases in
+ * {@code leftSideRequiredColumnWithAliases} and {@code rightSideRequiredColumnWithAliases}
+ * @return True if join has been successfully pushed down.
+ */
+ boolean pushDownJoin(
+ SupportsPushDownJoin other,
+ JoinType joinType,
+ ColumnWithAlias[] leftSideRequiredColumnsWithAliases,
+ ColumnWithAlias[] rightSideRequiredColumnsWithAliases,
+ Predicate condition
+ );
+
+ /**
+ * A helper class used when there are duplicated names coming from 2 sides of the join
+ * operator.
+ *
+ * Holds information of original output name and the alias of the new output.
+ */
+ record ColumnWithAlias(String colName, String alias) {
+ public String prettyString() {
+ if (alias == null) return colName;
+ else return colName + " AS " + alias;
+ }
+ }
+}
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java
index 4298f31227500..2bc994acaf33f 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java
@@ -99,6 +99,8 @@ public String build(Expression expr) {
case "CONTAINS" -> visitContains(build(e.children()[0]), build(e.children()[1]));
case "=", "<>", "<=>", "<", "<=", ">", ">=" ->
visitBinaryComparison(name, e.children()[0], e.children()[1]);
+ case "BOOLEAN_EXPRESSION" ->
+ build(expr.children()[0]);
case "+", "*", "/", "%", "&", "|", "^" ->
visitBinaryArithmetic(name, inputToSQL(e.children()[0]), inputToSQL(e.children()[1]));
case "-" -> {
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/BatchWrite.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/BatchWrite.java
index 8c068928415f4..c8febd0fe4934 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/BatchWrite.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/BatchWrite.java
@@ -19,6 +19,8 @@
import org.apache.spark.annotation.Evolving;
+import java.util.Map;
+
/**
* An interface that defines how to write the data to data source for batch processing.
*
@@ -88,6 +90,49 @@ default void onDataWriterCommit(WriterCommitMessage message) {}
*/
void commit(WriterCommitMessage[] messages);
+ /**
+ * Commits this writing job with a list of commit messages and operation metrics.
+ *
+ * If this method fails (by throwing an exception), this writing job is considered to to have been
+ * failed, and {@link #abort(WriterCommitMessage[])} would be called. The state of the destination
+ * is undefined and @{@link #abort(WriterCommitMessage[])} may not be able to deal with it.
+ *
+ * Note that speculative execution may cause multiple tasks to run for a partition. By default,
+ * Spark uses the commit coordinator to allow at most one task to commit. Implementations can
+ * disable this behavior by overriding {@link #useCommitCoordinator()}. If disabled, multiple
+ * tasks may have committed successfully and one successful commit message per task will be
+ * passed to this commit method. The remaining commit messages are ignored by Spark.
+ *
+ * @param messages a list of commit messages from successful data writers, produced by
+ * {@link DataWriter#commit()}.
+ * @param metrics a map of operation metrics collected from the query producing write.
+ * The keys will be prefixed by operation type, eg `merge`.
+ *
+ * Currently supported metrics are:
+ *
+ *
Operation Type = `merge`
+ *
+ *
`numTargetRowsCopied`: number of target rows copied unmodified because
+ * they did not match any action
+ *
`numTargetRowsDeleted`: number of target rows deleted
+ *
`numTargetRowsUpdated`: number of target rows updated
+ *
`numTargetRowsInserted`: number of target rows inserted
+ *
`numTargetRowsMatchedUpdated`: number of target rows updated by a
+ * matched clause
+ *
`numTargetRowsMatchedDeleted`: number of target rows deleted by a
+ * matched clause
+ *
`numTargetRowsNotMatchedBySourceUpdated`: number of target rows
+ * updated by a not matched by source clause
+ *
`numTargetRowsNotMatchedBySourceDeleted`: number of target rows
+ * deleted by a not matched by source clause
+ *
+ *
+ *
+ */
+ default void commit(WriterCommitMessage[] messages, Map metrics) {
+ commit(messages);
+ }
+
/**
* Aborts this writing job because some data writers are failed and keep failing when retry,
* or the Spark job fails with some unknown reasons,
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/util/CaseInsensitiveStringMap.java b/sql/catalyst/src/main/java/org/apache/spark/sql/util/CaseInsensitiveStringMap.java
index ec461f9740019..0c85f9b41cbaf 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/util/CaseInsensitiveStringMap.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/util/CaseInsensitiveStringMap.java
@@ -63,7 +63,7 @@ public CaseInsensitiveStringMap(Map originalMap) {
String key = toLowerCase(entry.getKey());
if (delegate.containsKey(key)) {
logger.warn("Converting duplicated key {} into CaseInsensitiveStringMap.",
- MDC.of(LogKeys.KEY$.MODULE$, entry.getKey()));
+ MDC.of(LogKeys.KEY, entry.getKey()));
}
delegate.put(key, entry.getValue());
}
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java
index 7382d96e20baa..66116d7c952fd 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java
@@ -182,6 +182,8 @@ void initAccessor(ValueVector vector) {
accessor = new TimestampAccessor(timeStampMicroTZVector);
} else if (vector instanceof TimeStampMicroVector timeStampMicroVector) {
accessor = new TimestampNTZAccessor(timeStampMicroVector);
+ } else if (vector instanceof TimeNanoVector timeNanoVector) {
+ accessor = new TimeNanoAccessor(timeNanoVector);
} else if (vector instanceof MapVector mapVector) {
accessor = new MapAccessor(mapVector);
} else if (vector instanceof ListVector listVector) {
@@ -522,6 +524,21 @@ final long getLong(int rowId) {
}
}
+ static class TimeNanoAccessor extends ArrowVectorAccessor {
+
+ private final TimeNanoVector accessor;
+
+ TimeNanoAccessor(TimeNanoVector vector) {
+ super(vector);
+ this.accessor = vector;
+ }
+
+ @Override
+ final long getLong(int rowId) {
+ return accessor.get(rowId);
+ }
+ }
+
static class ArrayAccessor extends ArrowVectorAccessor {
private final ListVector accessor;
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java
index 54b62c00283fa..f1d1f5b3ea800 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java
@@ -16,6 +16,8 @@
*/
package org.apache.spark.sql.vectorized;
+import scala.PartialFunction;
+
import org.apache.spark.annotation.Evolving;
import org.apache.spark.sql.types.DataType;
import org.apache.spark.sql.types.Decimal;
@@ -336,10 +338,21 @@ public final VariantVal getVariant(int rowId) {
* Sets up the data type of this column vector.
*/
protected ColumnVector(DataType type) {
- if (type instanceof UserDefinedType) {
- this.type = ((UserDefinedType) type).sqlType();
- } else {
- this.type = type;
- }
+ this.type = type.transformRecursively(
+ new PartialFunction() {
+ @Override
+ public boolean isDefinedAt(DataType x) {
+ return x instanceof UserDefinedType>;
+ }
+
+ @Override
+ public DataType apply(DataType t) {
+ if (t instanceof UserDefinedType> udt) {
+ return udt.sqlType();
+ } else {
+ return t;
+ }
+ }
+ });
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CapturesConfig.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CapturesConfig.scala
new file mode 100644
index 0000000000000..c77c3ac537e14
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CapturesConfig.scala
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * Trait used for persisting the conf values in views/UDFs.
+ */
+trait CapturesConfig {
+ private val configPrefixDenyList = Seq(
+ SQLConf.MAX_NESTED_VIEW_DEPTH.key,
+ "spark.sql.optimizer.",
+ "spark.sql.codegen.",
+ "spark.sql.execution.",
+ "spark.sql.shuffle.",
+ "spark.sql.adaptive.",
+ // ignore optimization configs used in `RelationConversions`
+ "spark.sql.hive.convertMetastoreParquet",
+ "spark.sql.hive.convertMetastoreOrc",
+ "spark.sql.hive.convertInsertingPartitionedTable",
+ "spark.sql.hive.convertInsertingUnpartitionedTable",
+ "spark.sql.hive.convertMetastoreCtas",
+ SQLConf.ADDITIONAL_REMOTE_REPOSITORIES.key)
+
+ private val configAllowList = Set(
+ SQLConf.DISABLE_HINTS.key
+ )
+
+ /**
+ * Set of single-pass resolver confs that shouldn't be stored during view/UDF/proc creation.
+ * This is needed to avoid accidental failures in tentative and dual-run modes when querying the
+ * view.
+ */
+ private val singlePassResolverDenyList = Set(
+ SQLConf.ANALYZER_SINGLE_PASS_RESOLVER_ENABLED_TENTATIVELY.key,
+ SQLConf.ANALYZER_DUAL_RUN_LEGACY_AND_SINGLE_PASS_RESOLVER.key
+ )
+
+ /**
+ * Convert the provided SQL configs to `properties`. Here we only capture the SQL configs that are
+ * modifiable and should be captured, i.e. not in the denyList and in the allowList. We also
+ * capture `SESSION_LOCAL_TIMEZONE` whose default value relies on the JVM system timezone and
+ * the `ANSI_ENABLED` value.
+ *
+ * We need to always capture them to make sure we apply the same configs when querying the
+ * view/UDF.
+ */
+ def sqlConfigsToProps(conf: SQLConf, prefix: String): Map[String, String] = {
+ val modifiedConfs = getModifiedConf(conf)
+
+ val alwaysCaptured = Seq(SQLConf.SESSION_LOCAL_TIMEZONE, SQLConf.ANSI_ENABLED)
+ .filter(c => !modifiedConfs.contains(c.key))
+ .map(c => (c.key, conf.getConf(c).toString))
+
+ val props = new mutable.HashMap[String, String]
+ for ((key, value) <- modifiedConfs ++ alwaysCaptured) {
+ props.put(s"$prefix$key", value)
+ }
+ props.toMap
+ }
+
+ /**
+ * Get all configurations that are modifiable and should be captured.
+ */
+ private def getModifiedConf(conf: SQLConf): Map[String, String] = {
+ conf.getAllConfs.filter { case (k, _) =>
+ conf.isModifiable(k) && shouldCaptureConfig(k)
+ }
+ }
+
+ /**
+ * Capture view config either of:
+ * 1. exists in allowList
+ * 2. do not exists in denyList
+ */
+ private def shouldCaptureConfig(key: String): Boolean = {
+ configAllowList.contains(key) || (
+ !configPrefixDenyList.exists(prefix => key.startsWith(prefix)) &&
+ !singlePassResolverDenyList.contains(key)
+ )
+ }
+}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala
index bb6afb3b13faf..c1e0674d391d2 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala
@@ -375,14 +375,14 @@ object CatalystTypeConverters {
private object TimeConverter extends CatalystTypeConverter[LocalTime, LocalTime, Any] {
override def toCatalystImpl(scalaValue: LocalTime): Long = {
- DateTimeUtils.localTimeToMicros(scalaValue)
+ DateTimeUtils.localTimeToNanos(scalaValue)
}
override def toScala(catalystValue: Any): LocalTime = {
if (catalystValue == null) null
- else DateTimeUtils.microsToLocalTime(catalystValue.asInstanceOf[Long])
+ else DateTimeUtils.nanosToLocalTime(catalystValue.asInstanceOf[Long])
}
override def toScalaImpl(row: InternalRow, column: Int): LocalTime =
- DateTimeUtils.microsToLocalTime(row.getLong(column))
+ DateTimeUtils.nanosToLocalTime(row.getLong(column))
}
private object TimestampConverter extends CatalystTypeConverter[Any, Timestamp, Any] {
@@ -579,6 +579,7 @@ object CatalystTypeConverters {
new DecimalConverter(DecimalType(Math.max(d.precision, d.scale), d.scale)).toCatalyst(d)
case d: JavaBigDecimal =>
new DecimalConverter(DecimalType(Math.max(d.precision, d.scale), d.scale)).toCatalyst(d)
+ case seq: scala.collection.mutable.ArraySeq[_] => convertToCatalyst(seq.array)
case seq: Seq[Any] => new GenericArrayData(seq.map(convertToCatalyst).toArray)
case r: Row => InternalRow(r.toSeq.map(convertToCatalyst): _*)
case arr: Array[Byte] => arr
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala
index 9b22f28ed12da..15de70e35a45d 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala
@@ -160,7 +160,7 @@ object DeserializerBuildHelper {
StaticInvoke(
DateTimeUtils.getClass,
ObjectType(classOf[java.time.LocalTime]),
- "microsToLocalTime",
+ "nanosToLocalTime",
path :: Nil,
returnNullable = false)
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala
index 5158cdb276fd4..f9bf0ebdfd9ad 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala
@@ -175,7 +175,7 @@ object InternalRow {
case ShortType => (input, v) => input.setShort(ordinal, v.asInstanceOf[Short])
case IntegerType | DateType | _: YearMonthIntervalType =>
(input, v) => input.setInt(ordinal, v.asInstanceOf[Int])
- case LongType | TimestampType | TimestampNTZType | _: DayTimeIntervalType =>
+ case LongType | TimestampType | TimestampNTZType | _: DayTimeIntervalType | _: TimeType =>
(input, v) => input.setLong(ordinal, v.asInstanceOf[Long])
case FloatType => (input, v) => input.setFloat(ordinal, v.asInstanceOf[Float])
case DoubleType => (input, v) => input.setDouble(ordinal, v.asInstanceOf[Double])
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala
index c8bf1f5237997..82b3cdc508bf9 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala
@@ -103,7 +103,7 @@ object SerializerBuildHelper {
StaticInvoke(
DateTimeUtils.getClass,
TimeType(),
- "localTimeToMicros",
+ "localTimeToNanos",
inputObject :: Nil,
returnNullable = false)
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index f30c22c2566ba..1896a1c7ac279 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -18,7 +18,7 @@
package org.apache.spark.sql.catalyst.analysis
import java.util
-import java.util.Locale
+import java.util.{LinkedHashMap, Locale}
import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer
@@ -31,9 +31,7 @@ import org.apache.spark.sql.catalyst._
import org.apache.spark.sql.catalyst.analysis.resolver.{
AnalyzerBridgeState,
HybridAnalyzer,
- Resolver => OperatorResolver,
- ResolverExtension,
- ResolverGuard
+ ResolverExtension
}
import org.apache.spark.sql.catalyst.catalog._
import org.apache.spark.sql.catalyst.encoders.OuterScopes
@@ -258,7 +256,10 @@ object Analyzer {
"spark.sql.expressionTreeChangeLog.level"
)
- def retainResolutionConfigsForAnalysis(newConf: SQLConf, existingConf: SQLConf): Unit = {
+ def retainResolutionConfigsForAnalysis(
+ newConf: SQLConf,
+ existingConf: SQLConf,
+ createSparkVersion: String = ""): Unit = {
val retainedConfigs = existingConf.getAllConfs.filter { case (key, _) =>
// Also apply catalog configs
RETAINED_ANALYSIS_FLAGS.contains(key) || key.startsWith("spark.sql.catalog.")
@@ -267,6 +268,25 @@ object Analyzer {
retainedConfigs.foreach { case (k, v) =>
newConf.settings.put(k, v)
}
+
+ trySetAnsiValue(newConf, createSparkVersion)
+ }
+
+ /**
+ * In case ANSI value wasn't persisted for a view or a UDF, we set it to `true` in case Spark
+ * version used to create the view is 4.0.0 or higher. We set it to `false` in case Spark version
+ * is lower than 4.0.0 or if the Spark version wasn't stored (in that case we assume that the
+ * value is `false`)
+ */
+ def trySetAnsiValue(sqlConf: SQLConf, createSparkVersion: String = ""): Unit = {
+ if (conf.getConf(SQLConf.ASSUME_ANSI_FALSE_IF_NOT_PERSISTED) &&
+ !sqlConf.settings.containsKey(SQLConf.ANSI_ENABLED.key)) {
+ if (createSparkVersion.startsWith("4.")) {
+ sqlConf.settings.put(SQLConf.ANSI_ENABLED.key, "true")
+ } else {
+ sqlConf.settings.put(SQLConf.ANSI_ENABLED.key, "false")
+ }
+ }
}
}
@@ -297,17 +317,17 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
def getRelationResolution: RelationResolution = relationResolution
def executeAndCheck(plan: LogicalPlan, tracker: QueryPlanningTracker): LogicalPlan = {
- if (plan.analyzed) return plan
- AnalysisHelper.markInAnalyzer {
- new HybridAnalyzer(
- this,
- new ResolverGuard(catalogManager),
- new OperatorResolver(
- catalogManager,
- singlePassResolverExtensions,
- singlePassMetadataResolverExtensions
- )
- ).apply(plan, tracker)
+ if (plan.analyzed) {
+ plan
+ } else {
+ AnalysisContext.reset()
+ try {
+ AnalysisHelper.markInAnalyzer {
+ HybridAnalyzer.fromLegacyAnalyzer(legacyAnalyzer = this).apply(plan, tracker)
+ }
+ } finally {
+ AnalysisContext.reset()
+ }
}
}
@@ -424,6 +444,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
AddMetadataColumns ::
DeduplicateRelations ::
ResolveCollationName ::
+ ResolveMergeIntoSchemaEvolution ::
new ResolveReferences(catalogManager) ::
// Please do not insert any other rules in between. See the TODO comments in rule
// ResolveLateralColumnAliasReference for more details.
@@ -454,7 +475,6 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
ResolveNaturalAndUsingJoin ::
ResolveOutputRelation ::
new ResolveTableConstraints(catalogManager) ::
- new ResolveDataFrameDropColumns(catalogManager) ::
new ResolveSetVariable(catalogManager) ::
ExtractWindowExpressions ::
GlobalAggregates ::
@@ -519,7 +539,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
* 1. if both side are interval, stays the same;
* 2. else if one side is date and the other is interval,
* turns it to [[DateAddInterval]];
- * 3. else if one side is interval, turns it to [[TimeAdd]];
+ * 3. else if one side is interval, turns it to [[TimestampAddInterval]];
* 4. else if one side is date, turns it to [[DateAdd]] ;
* 5. else stays the same.
*
@@ -527,7 +547,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
* 1. if both side are interval, stays the same;
* 2. else if the left side is date and the right side is interval,
* turns it to [[DateAddInterval(l, -r)]];
- * 3. else if the right side is an interval, turns it to [[TimeAdd(l, -r)]];
+ * 3. else if the right side is an interval, turns it to [[TimestampAddInterval(l, -r)]];
* 4. else if one side is timestamp, turns it to [[SubtractTimestamps]];
* 5. else if the right side is date, turns it to [[DateDiff]]/[[SubtractDates]];
* 6. else if the left side is date, turns it to [[DateSub]];
@@ -795,6 +815,10 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
} else {
colResolved.havingCondition
}
+ // `cond` might contain unresolved aggregate functions so defer its resolution to
+ // `ResolveAggregateFunctions` rule if needed.
+ if (!cond.resolved) return colResolved
+
// Try resolving the condition of the filter as though it is in the aggregate clause
val (extraAggExprs, Seq(resolvedHavingCond)) =
ResolveAggregateFunctions.resolveExprsWithAggregate(Seq(cond), aggForResolving)
@@ -1483,6 +1507,8 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
new ResolveReferencesInUpdate(catalogManager)
private val resolveReferencesInSort =
new ResolveReferencesInSort(catalogManager)
+ private val resolveDataFrameDropColumns =
+ new ResolveDataFrameDropColumns(catalogManager)
/**
* Return true if there're conflicting attributes among children's outputs of a plan
@@ -1572,29 +1598,11 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
)
case u @ Union(children, _, _)
- // if there are duplicate output columns, give them unique expr ids
- if children.exists(c => c.output.map(_.exprId).distinct.length < c.output.length) =>
- val newChildren = children.map { c =>
- if (c.output.map(_.exprId).distinct.length < c.output.length) {
- val existingExprIds = mutable.HashSet[ExprId]()
- val projectList = c.output.map { attr =>
- if (existingExprIds.contains(attr.exprId)) {
- // replace non-first duplicates with aliases and tag them
- val newMetadata = new MetadataBuilder().withMetadata(attr.metadata)
- .putNull("__is_duplicate").build()
- Alias(attr, attr.name)(explicitMetadata = Some(newMetadata))
- } else {
- // leave first duplicate alone
- existingExprIds.add(attr.exprId)
- attr
- }
- }
- Project(projectList, c)
- } else {
- c
- }
- }
- u.withNewChildren(newChildren)
+ // if there are duplicate output columns, give them unique expr ids
+ if (u.allChildrenCompatible &&
+ conf.getConf(SQLConf.ENFORCE_TYPE_COERCION_BEFORE_UNION_DEDUPLICATION)) &&
+ children.exists(c => c.output.map(_.exprId).distinct.length < c.output.length) =>
+ DeduplicateUnionChildOutput.deduplicateOutputPerChild(u)
// A special case for Generate, because the output of Generate should not be resolved by
// ResolveReferences. Attributes in the output will be resolved by ResolveGenerate.
@@ -1662,7 +1670,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
case u: UpdateTable => resolveReferencesInUpdate(u)
case m @ MergeIntoTable(targetTable, sourceTable, _, _, _, _, _)
- if !m.resolved && targetTable.resolved && sourceTable.resolved =>
+ if !m.resolved && targetTable.resolved && sourceTable.resolved && !m.needSchemaEvolution =>
EliminateSubqueryAliases(targetTable) match {
case r: NamedRelation if r.skipSchemaResolution =>
@@ -1685,9 +1693,12 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
// The update value can access columns from both target and source tables.
resolveAssignments(assignments, m, MergeResolvePolicy.BOTH))
case UpdateStarAction(updateCondition) =>
- val assignments = targetTable.output.map { attr =>
- Assignment(attr, UnresolvedAttribute(Seq(attr.name)))
- }
+ // Use only source columns. Missing columns in target will be handled in
+ // ResolveRowLevelCommandAssignments.
+ val assignments = targetTable.output.flatMap{ targetAttr =>
+ sourceTable.output.find(
+ sourceCol => conf.resolver(sourceCol.name, targetAttr.name))
+ .map(Assignment(targetAttr, _))}
UpdateAction(
updateCondition.map(resolveExpressionByPlanChildren(_, m)),
// For UPDATE *, the value must be from source table.
@@ -1708,9 +1719,12 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
// access columns from the source table.
val resolvedInsertCondition = insertCondition.map(
resolveExpressionByPlanOutput(_, m.sourceTable))
- val assignments = targetTable.output.map { attr =>
- Assignment(attr, UnresolvedAttribute(Seq(attr.name)))
- }
+ // Use only source columns. Missing columns in target will be handled in
+ // ResolveRowLevelCommandAssignments.
+ val assignments = targetTable.output.flatMap{ targetAttr =>
+ sourceTable.output.find(
+ sourceCol => conf.resolver(sourceCol.name, targetAttr.name))
+ .map(Assignment(targetAttr, _))}
InsertAction(
resolvedInsertCondition,
resolveAssignments(assignments, m, MergeResolvePolicy.SOURCE))
@@ -1791,6 +1805,9 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
// Pass for Execute Immediate as arguments will be resolved by [[SubstituteExecuteImmediate]].
case e : ExecuteImmediateQuery => e
+ case d: DataFrameDropColumns if !d.resolved =>
+ resolveDataFrameDropColumns(d)
+
case q: LogicalPlan =>
logTrace(s"Attempting to resolve ${q.simpleString(conf.maxToStringFields)}")
q.mapExpressions(resolveExpressionByPlanChildren(_, q, includeLastResort = true))
@@ -2241,12 +2258,15 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
case _ => tvf
}
- Project(
- Seq(UnresolvedStar(Some(Seq(alias)))),
- LateralJoin(
- tableArgs.map(_._2).reduceLeft(Join(_, _, Inner, None, JoinHint.NONE)),
- LateralSubquery(SubqueryAlias(alias, tvfWithTableColumnIndexes)), Inner, None)
- )
+ val lateralJoin = LateralJoin(
+ tableArgs.map(_._2).reduceLeft(Join(_, _, Inner, None, JoinHint.NONE)),
+ LateralSubquery(SubqueryAlias(alias, tvfWithTableColumnIndexes)), Inner, None)
+
+ // Set the tag so that it can be used to differentiate lateral join added by
+ // TABLE argument vs added by user.
+ lateralJoin.setTagValue(LateralJoin.BY_TABLE_ARGUMENT, ())
+
+ Project(Seq(UnresolvedStar(Some(Seq(alias)))), lateralJoin)
}
case q: LogicalPlan =>
@@ -2301,7 +2321,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
}
PythonUDTF(
u.name, u.func, analyzeResult.schema, Some(analyzeResult.pickledAnalyzeResult),
- newChildren, u.evalType, u.udfDeterministic, u.resultId)
+ newChildren, u.evalType, u.udfDeterministic, u.resultId, None, u.tableArguments)
}
}
}
@@ -2557,9 +2577,10 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
// Extract the function input project list from the SQL function plan and
// inline the SQL function expression.
plan match {
- case Project(body :: Nil, Project(aliases, _: LocalRelation)) =>
- projectList ++= aliases
- SQLScalarFunction(f.function, aliases.map(_.toAttribute), body)
+ case Project(body :: Nil, Project(aliases, _: OneRowRelation)) =>
+ val inputs = aliases.map(stripOuterReference)
+ projectList ++= inputs
+ SQLScalarFunction(f.function, inputs.map(_.toAttribute), body)
case o =>
throw new AnalysisException(
errorClass = "INVALID_SQL_FUNCTION_PLAN_STRUCTURE",
@@ -2935,7 +2956,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
def resolveExprsWithAggregate(
exprs: Seq[Expression],
agg: Aggregate): (Seq[NamedExpression], Seq[Expression]) = {
- val extraAggExprs = ArrayBuffer.empty[NamedExpression]
+ val extraAggExprs = new LinkedHashMap[Expression, NamedExpression]
val transformed = exprs.map { e =>
if (!e.resolved) {
e
@@ -2943,39 +2964,44 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
buildAggExprList(e, agg, extraAggExprs)
}
}
- (extraAggExprs.toSeq, transformed)
+ (extraAggExprs.values().asScala.toSeq, transformed)
}
private def buildAggExprList(
expr: Expression,
agg: Aggregate,
- aggExprList: ArrayBuffer[NamedExpression]): Expression = {
+ aggExprMap: LinkedHashMap[Expression, NamedExpression]): Expression = {
// Avoid adding an extra aggregate expression if it's already present in
- // `agg.aggregateExpressions`.
- val index = agg.aggregateExpressions.indexWhere {
- case Alias(child, _) => child semanticEquals expr
- case other => other semanticEquals expr
- }
- if (index >= 0) {
- agg.aggregateExpressions(index).toAttribute
+ // `agg.aggregateExpressions`. Trim inner aliases from aggregate expressions because of
+ // expressions like `spark_grouping_id` that can have inner aliases.
+ val replacement: Option[NamedExpression] =
+ agg.aggregateExpressions.foldLeft(Option.empty[NamedExpression]) {
+ case (None, alias: Alias) if expr.semanticEquals(trimAliases(alias.child)) => Some(alias)
+ case (None | Some(_: Alias), aggExpr) if expr.semanticEquals(aggExpr) => Some(aggExpr)
+ case (current, _) => current
+ }
+ if (replacement.isDefined) {
+ replacement.get.toAttribute
} else {
expr match {
case ae: AggregateExpression =>
val cleaned = trimTempResolvedColumn(ae)
- val alias =
- Alias(cleaned, toPrettySQL(e = cleaned, shouldTrimTempResolvedColumn = true))()
- aggExprList += alias
- alias.toAttribute
+ val resultAlias = aggExprMap.computeIfAbsent(
+ cleaned.canonicalized,
+ _ => Alias(cleaned, toPrettySQL(e = cleaned, shouldTrimTempResolvedColumn = true))()
+ )
+ resultAlias.toAttribute
case grouping: Expression if agg.groupingExpressions.exists(grouping.semanticEquals) =>
trimTempResolvedColumn(grouping) match {
case ne: NamedExpression =>
- aggExprList += ne
- ne.toAttribute
+ val resultAttribute = aggExprMap.computeIfAbsent(ne.canonicalized, _ => ne)
+ resultAttribute.toAttribute
case other =>
- val alias =
- Alias(other, toPrettySQL(e = other, shouldTrimTempResolvedColumn = true))()
- aggExprList += alias
- alias.toAttribute
+ val resultAlias = aggExprMap.computeIfAbsent(
+ other.canonicalized,
+ _ => Alias(other, toPrettySQL(e = other, shouldTrimTempResolvedColumn = true))()
+ )
+ resultAlias.toAttribute
}
case t: TempResolvedColumn =>
if (t.child.isInstanceOf[Attribute]) {
@@ -2990,7 +3016,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
val childWithTempCol = t.child.transformUp {
case a: Attribute => TempResolvedColumn(a, Seq(a.name))
}
- val newChild = buildAggExprList(childWithTempCol, agg, aggExprList)
+ val newChild = buildAggExprList(childWithTempCol, agg, aggExprMap)
if (newChild.containsPattern(TEMP_RESOLVED_COLUMN)) {
withOrigin(t.origin)(t.copy(hasTried = true))
} else {
@@ -2998,7 +3024,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
}
}
case other =>
- other.withNewChildren(other.children.map(buildAggExprList(_, agg, aggExprList)))
+ other.withNewChildren(other.children.map(buildAggExprList(_, agg, aggExprMap)))
}
}
}
@@ -3643,23 +3669,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
object ResolveWindowFrame extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan.resolveExpressionsWithPruning(
_.containsPattern(WINDOW_EXPRESSION), ruleId) {
- case WindowExpression(wf: FrameLessOffsetWindowFunction,
- WindowSpecDefinition(_, _, f: SpecifiedWindowFrame)) if wf.frame != f =>
- throw QueryCompilationErrors.cannotSpecifyWindowFrameError(wf.prettyName)
- case WindowExpression(wf: WindowFunction, WindowSpecDefinition(_, _, f: SpecifiedWindowFrame))
- if wf.frame != UnspecifiedFrame && wf.frame != f =>
- throw QueryCompilationErrors.windowFrameNotMatchRequiredFrameError(f, wf.frame)
- case WindowExpression(wf: WindowFunction, s @ WindowSpecDefinition(_, _, UnspecifiedFrame))
- if wf.frame != UnspecifiedFrame =>
- WindowExpression(wf, s.copy(frameSpecification = wf.frame))
- case we @ WindowExpression(e, s @ WindowSpecDefinition(_, o, UnspecifiedFrame))
- if e.resolved =>
- val frame = if (o.nonEmpty) {
- SpecifiedWindowFrame(RangeFrame, UnboundedPreceding, CurrentRow)
- } else {
- SpecifiedWindowFrame(RowFrame, UnboundedPreceding, UnboundedFollowing)
- }
- we.copy(windowSpec = s.copy(frameSpecification = frame))
+ case we: WindowExpression => WindowResolution.resolveFrame(we)
}
}
@@ -3669,11 +3679,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
object ResolveWindowOrder extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan.resolveExpressionsWithPruning(
_.containsPattern(WINDOW_EXPRESSION), ruleId) {
- case WindowExpression(wf: WindowFunction, spec) if spec.orderSpec.isEmpty =>
- throw QueryCompilationErrors.windowFunctionWithWindowFrameNotOrderedError(wf)
- case WindowExpression(rank: RankLike, spec) if spec.resolved =>
- val order = spec.orderSpec.map(_.child)
- WindowExpression(rank.withOrder(order), spec)
+ case we: WindowExpression => WindowResolution.resolveOrder(we)
}
}
@@ -4218,13 +4224,14 @@ object UpdateOuterReferences extends Rule[LogicalPlan] {
private def updateOuterReferenceInSubquery(
plan: LogicalPlan,
refExprs: Seq[Expression]): LogicalPlan = {
- plan resolveExpressions { case e =>
- val outerAlias =
- refExprs.find(stripAlias(_).semanticEquals(stripOuterReference(e)))
- outerAlias match {
- case Some(a: Alias) => OuterReference(a.toAttribute)
- case _ => e
- }
+ plan resolveExpressions {
+ case e if e.containsPattern(OUTER_REFERENCE) =>
+ val outerAlias =
+ refExprs.find(stripAlias(_).semanticEquals(stripOuterReference(e)))
+ outerAlias match {
+ case Some(a: Alias) => OuterReference(a.toAttribute)
+ case _ => e
+ }
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiStringPromotionTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiStringPromotionTypeCoercion.scala
index e7be95bc645ea..f6d7a9605831a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiStringPromotionTypeCoercion.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiStringPromotionTypeCoercion.scala
@@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.{
Literal,
SubtractDates,
SubtractTimestamps,
- TimeAdd,
+ TimestampAddInterval,
UnaryMinus,
UnaryPositive
}
@@ -77,7 +77,7 @@ object AnsiStringPromotionTypeCoercion {
s.copy(left = Cast(s.left, DateType))
case s @ SubtractDates(_, right @ StringTypeExpression(), _) =>
s.copy(right = Cast(s.right, DateType))
- case t @ TimeAdd(left @ StringTypeExpression(), _, _) =>
+ case t @ TimestampAddInterval(left @ StringTypeExpression(), _, _) =>
t.copy(start = Cast(t.start, TimestampType))
case t @ SubtractTimestamps(left @ StringTypeExpression(), _, _, _) =>
t.copy(left = Cast(t.left, t.right.dataType))
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/BinaryArithmeticWithDatetimeResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/BinaryArithmeticWithDatetimeResolver.scala
index 36a059b2b0f3c..08407bbe96cce 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/BinaryArithmeticWithDatetimeResolver.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/BinaryArithmeticWithDatetimeResolver.scala
@@ -38,8 +38,10 @@ import org.apache.spark.sql.catalyst.expressions.{
MultiplyYMInterval,
Subtract,
SubtractDates,
+ SubtractTimes,
SubtractTimestamps,
- TimeAdd,
+ TimeAddInterval,
+ TimestampAddInterval,
TimestampAddYMInterval,
UnaryMinus
}
@@ -47,12 +49,14 @@ import org.apache.spark.sql.types.{
AnsiIntervalType,
AnyTimestampTypeExpression,
CalendarIntervalType,
+ DatetimeType,
DateType,
DayTimeIntervalType,
NullType,
StringType,
TimestampNTZType,
TimestampType,
+ TimeType,
YearMonthIntervalType
}
import org.apache.spark.sql.types.DayTimeIntervalType.DAY
@@ -62,9 +66,9 @@ object BinaryArithmeticWithDatetimeResolver {
case a @ Add(l, r, mode) =>
(l.dataType, r.dataType) match {
case (DateType, DayTimeIntervalType(DAY, DAY)) => DateAdd(l, ExtractANSIIntervalDays(r))
- case (DateType, _: DayTimeIntervalType) => TimeAdd(Cast(l, TimestampType), r)
+ case (DateType, _: DayTimeIntervalType) => TimestampAddInterval(Cast(l, TimestampType), r)
case (DayTimeIntervalType(DAY, DAY), DateType) => DateAdd(r, ExtractANSIIntervalDays(l))
- case (_: DayTimeIntervalType, DateType) => TimeAdd(Cast(r, TimestampType), l)
+ case (_: DayTimeIntervalType, DateType) => TimestampAddInterval(Cast(r, TimestampType), l)
case (DateType, _: YearMonthIntervalType) => DateAddYMInterval(l, r)
case (_: YearMonthIntervalType, DateType) => DateAddYMInterval(r, l)
case (TimestampType | TimestampNTZType, _: YearMonthIntervalType) =>
@@ -80,10 +84,18 @@ object BinaryArithmeticWithDatetimeResolver {
a.copy(right = Cast(a.right, a.left.dataType))
case (DateType, CalendarIntervalType) =>
DateAddInterval(l, r, ansiEnabled = mode == EvalMode.ANSI)
- case (_, CalendarIntervalType | _: DayTimeIntervalType) => Cast(TimeAdd(l, r), l.dataType)
+ case (_: TimeType, _: DayTimeIntervalType) => TimeAddInterval(l, r)
+ case (_: DatetimeType, _: NullType) =>
+ a.copy(right = Cast(a.right, DayTimeIntervalType.DEFAULT))
+ case (_: DayTimeIntervalType, _: TimeType) => TimeAddInterval(r, l)
+ case (_: NullType, _: DatetimeType) =>
+ a.copy(left = Cast(a.left, DayTimeIntervalType.DEFAULT))
+ case (_, CalendarIntervalType | _: DayTimeIntervalType) =>
+ Cast(TimestampAddInterval(l, r), l.dataType)
case (CalendarIntervalType, DateType) =>
DateAddInterval(r, l, ansiEnabled = mode == EvalMode.ANSI)
- case (CalendarIntervalType | _: DayTimeIntervalType, _) => Cast(TimeAdd(r, l), r.dataType)
+ case (CalendarIntervalType | _: DayTimeIntervalType, _) =>
+ Cast(TimestampAddInterval(r, l), r.dataType)
case (DateType, dt) if dt != StringType => DateAdd(l, r)
case (dt, DateType) if dt != StringType => DateAdd(r, l)
case _ => a
@@ -93,7 +105,8 @@ object BinaryArithmeticWithDatetimeResolver {
case (DateType, DayTimeIntervalType(DAY, DAY)) =>
DateAdd(l, UnaryMinus(ExtractANSIIntervalDays(r), mode == EvalMode.ANSI))
case (DateType, _: DayTimeIntervalType) =>
- DatetimeSub(l, r, TimeAdd(Cast(l, TimestampType), UnaryMinus(r, mode == EvalMode.ANSI)))
+ DatetimeSub(l, r,
+ TimestampAddInterval(Cast(l, TimestampType), UnaryMinus(r, mode == EvalMode.ANSI)))
case (DateType, _: YearMonthIntervalType) =>
DatetimeSub(l, r, DateAddYMInterval(l, UnaryMinus(r, mode == EvalMode.ANSI)))
case (TimestampType | TimestampNTZType, _: YearMonthIntervalType) =>
@@ -101,9 +114,9 @@ object BinaryArithmeticWithDatetimeResolver {
case (CalendarIntervalType, CalendarIntervalType) |
(_: DayTimeIntervalType, _: DayTimeIntervalType) =>
s
- case (_: NullType, _: AnsiIntervalType) =>
+ case (_: NullType, _: AnsiIntervalType | _: DatetimeType) =>
s.copy(left = Cast(s.left, s.right.dataType))
- case (_: AnsiIntervalType, _: NullType) =>
+ case (_: AnsiIntervalType | _: DatetimeType, _: NullType) =>
s.copy(right = Cast(s.right, s.left.dataType))
case (DateType, CalendarIntervalType) =>
DatetimeSub(
@@ -115,14 +128,18 @@ object BinaryArithmeticWithDatetimeResolver {
ansiEnabled = mode == EvalMode.ANSI
)
)
+ case (_: TimeType, _: DayTimeIntervalType) =>
+ DatetimeSub(l, r, TimeAddInterval(l, UnaryMinus(r, mode == EvalMode.ANSI)))
case (_, CalendarIntervalType | _: DayTimeIntervalType) =>
- Cast(DatetimeSub(l, r, TimeAdd(l, UnaryMinus(r, mode == EvalMode.ANSI))), l.dataType)
+ Cast(DatetimeSub(l, r,
+ TimestampAddInterval(l, UnaryMinus(r, mode == EvalMode.ANSI))), l.dataType)
case _
if AnyTimestampTypeExpression.unapply(l) ||
AnyTimestampTypeExpression.unapply(r) =>
SubtractTimestamps(l, r)
case (_, DateType) => SubtractDates(l, r)
case (DateType, dt) if dt != StringType => DateSub(l, r)
+ case (_: TimeType, _: TimeType) => SubtractTimes(l, r)
case _ => s
}
case m @ Multiply(l, r, mode) =>
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
index f34cf8ce53498..2ff842553bee6 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
@@ -19,11 +19,12 @@ package org.apache.spark.sql.catalyst.analysis
import scala.collection.mutable
import org.apache.spark.{SparkException, SparkThrowable}
+import org.apache.spark.api.python.PythonEvalType
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.ExtendedAnalysisException
import org.apache.spark.sql.catalyst.analysis.ResolveWithCTE.checkIfSelfReferenceIsPlacedCorrectly
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction, ListAgg, Median, PercentileCont, PercentileDisc}
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction, ListAgg}
import org.apache.spark.sql.catalyst.optimizer.InlineCTE
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.trees.TreePattern.{LATERAL_COLUMN_ALIAS_REFERENCE, PLAN_EXPRESSION, UNRESOLVED_WINDOW_EXPRESSION}
@@ -437,51 +438,13 @@ trait CheckAnalysis extends LookupCatalog with QueryErrorsBase with PlanToString
errorClass = "WINDOW_FUNCTION_WITHOUT_OVER_CLAUSE",
messageParameters = Map("funcName" -> toSQLExpr(w)))
- case w @ WindowExpression(AggregateExpression(_, _, true, _, _), _) =>
- w.failAnalysis(
- errorClass = "DISTINCT_WINDOW_FUNCTION_UNSUPPORTED",
- messageParameters = Map("windowExpr" -> toSQLExpr(w)))
-
- case w @ WindowExpression(wf: FrameLessOffsetWindowFunction,
- WindowSpecDefinition(_, order, frame: SpecifiedWindowFrame))
- if order.isEmpty || !frame.isOffset =>
- w.failAnalysis(
- errorClass = "WINDOW_FUNCTION_AND_FRAME_MISMATCH",
- messageParameters = Map(
- "funcName" -> toSQLExpr(wf),
- "windowExpr" -> toSQLExpr(w)))
-
case agg @ AggregateExpression(listAgg: ListAgg, _, _, _, _)
if agg.isDistinct && listAgg.needSaveOrderValue =>
throw QueryCompilationErrors.functionAndOrderExpressionMismatchError(
listAgg.prettyName, listAgg.child, listAgg.orderExpressions)
case w: WindowExpression =>
- // Only allow window functions with an aggregate expression or an offset window
- // function or a Pandas window UDF.
- w.windowFunction match {
- case agg @ AggregateExpression(fun: ListAgg, _, _, _, _)
- // listagg(...) WITHIN GROUP (ORDER BY ...) OVER (ORDER BY ...) is unsupported
- if fun.orderingFilled && (w.windowSpec.orderSpec.nonEmpty ||
- w.windowSpec.frameSpecification !=
- SpecifiedWindowFrame(RowFrame, UnboundedPreceding, UnboundedFollowing)) =>
- agg.failAnalysis(
- errorClass = "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC",
- messageParameters = Map("aggFunc" -> toSQLExpr(agg.aggregateFunction)))
- case agg @ AggregateExpression(
- _: PercentileCont | _: PercentileDisc | _: Median, _, _, _, _)
- if w.windowSpec.orderSpec.nonEmpty || w.windowSpec.frameSpecification !=
- SpecifiedWindowFrame(RowFrame, UnboundedPreceding, UnboundedFollowing) =>
- agg.failAnalysis(
- errorClass = "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC",
- messageParameters = Map("aggFunc" -> toSQLExpr(agg.aggregateFunction)))
- case _: AggregateExpression | _: FrameLessOffsetWindowFunction |
- _: AggregateWindowFunction => // OK
- case other =>
- other.failAnalysis(
- errorClass = "UNSUPPORTED_EXPR_FOR_WINDOW",
- messageParameters = Map("sqlExpr" -> toSQLExpr(other)))
- }
+ WindowResolution.validateResolvedWindowExpression(w)
case s: SubqueryExpression =>
checkSubqueryExpression(operator, s)
@@ -569,7 +532,19 @@ trait CheckAnalysis extends LookupCatalog with QueryErrorsBase with PlanToString
messageParameters = Map.empty)
}
- case a: Aggregate => ExprUtils.assertValidAggregation(a)
+ case a: Aggregate =>
+ a.groupingExpressions.foreach(
+ expression =>
+ if (!expression.deterministic) {
+ throw SparkException.internalError(
+ msg = s"Non-deterministic expression '${toSQLExpr(expression)}' should not " +
+ "appear in grouping expression.",
+ context = expression.origin.getQueryContext,
+ summary = expression.origin.context.summary
+ )
+ }
+ )
+ ExprUtils.assertValidAggregation(a)
case CollectMetrics(name, metrics, _, _) =>
if (name == null || name.isEmpty) {
@@ -747,7 +722,7 @@ trait CheckAnalysis extends LookupCatalog with QueryErrorsBase with PlanToString
case c: CreateVariable
if c.resolved && c.defaultExpr.child.containsPattern(PLAN_EXPRESSION) =>
- val ident = c.name.asInstanceOf[ResolvedIdentifier]
+ val ident = c.names(0).asInstanceOf[ResolvedIdentifier]
val varName = toSQLId(
(ident.catalog.name +: ident.identifier.namespace :+ ident.identifier.name)
.toImmutableArraySeq)
@@ -915,6 +890,17 @@ trait CheckAnalysis extends LookupCatalog with QueryErrorsBase with PlanToString
messageParameters = Map(
"invalidExprSqls" -> invalidExprSqls.mkString(", ")))
+ case j @ LateralJoin(_, right, _, _)
+ if j.getTagValue(LateralJoin.BY_TABLE_ARGUMENT).isEmpty =>
+ right.plan.foreach {
+ case Generate(pyudtf: PythonUDTF, _, _, _, _, _)
+ if pyudtf.evalType == PythonEvalType.SQL_ARROW_UDTF =>
+ j.failAnalysis(
+ errorClass = "LATERAL_JOIN_WITH_ARROW_UDTF_UNSUPPORTED",
+ messageParameters = Map.empty)
+ case _ =>
+ }
+
case _ => // Analysis successful!
}
}
@@ -947,30 +933,14 @@ trait CheckAnalysis extends LookupCatalog with QueryErrorsBase with PlanToString
if (expr.plan.isStreaming) {
plan.failAnalysis("INVALID_SUBQUERY_EXPRESSION.STREAMING_QUERY", Map.empty)
}
- assertNoRecursiveCTE(expr.plan)
checkAnalysis0(expr.plan)
ValidateSubqueryExpression(plan, expr)
}
- private def assertNoRecursiveCTE(plan: LogicalPlan): Unit = {
- plan.foreach {
- case r: CTERelationRef if r.recursive =>
- throw new AnalysisException(
- errorClass = "INVALID_RECURSIVE_REFERENCE.PLACE",
- messageParameters = Map.empty)
- case p => p.expressions.filter(_.containsPattern(PLAN_EXPRESSION)).foreach {
- expr => expr.foreach {
- case s: SubqueryExpression => assertNoRecursiveCTE(s.plan)
- case _ =>
- }
- }
- }
- }
-
/**
* Validate that collected metrics names are unique. The same name cannot be used for metrics
- * with different results. However multiple instances of metrics with with same result and name
- * are allowed (e.g. self-joins).
+ * with different results. However, multiple instances of metrics with same result and name are
+ * allowed (e.g. self-joins).
*/
private def checkCollectedMetrics(plan: LogicalPlan): Unit = {
val metricsMap = mutable.Map.empty[String, CollectMetrics]
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala
index 6823cdbf36ba2..bade5f0bee9d6 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala
@@ -23,14 +23,12 @@ import scala.collection.mutable
import org.apache.spark.internal.Logging
import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.catalyst.SqlScriptingContextManager
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.SubExprUtils.wrapOuterReference
-import org.apache.spark.sql.catalyst.parser.SqlScriptingLabelContext.isForbiddenLabelName
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.trees.CurrentOrigin.withOrigin
import org.apache.spark.sql.catalyst.trees.TreePattern._
-import org.apache.spark.sql.connector.catalog.{CatalogManager, Identifier}
+import org.apache.spark.sql.connector.catalog.CatalogManager
import org.apache.spark.sql.errors.{DataTypeErrorsBase, QueryCompilationErrors}
import org.apache.spark.sql.internal.SQLConf
@@ -210,7 +208,11 @@ trait ColumnResolutionHelper extends Logging with DataTypeErrorsBase {
case u @ UnresolvedHaving(_, agg: Aggregate) =>
agg.resolveChildren(nameParts, conf.resolver)
.orElse(u.resolveChildren(nameParts, conf.resolver))
- .map(wrapOuterReference)
+ .map {
+ case alias: Alias =>
+ wrapOuterReference(alias.child)
+ case other => wrapOuterReference(other)
+ }
case other =>
other.resolveChildren(nameParts, conf.resolver).map(wrapOuterReference)
}
@@ -231,102 +233,17 @@ trait ColumnResolutionHelper extends Logging with DataTypeErrorsBase {
}
}
- /**
- * Look up variable by nameParts.
- * If in SQL Script, first check local variables, unless in EXECUTE IMMEDIATE
- * (EXECUTE IMMEDIATE generated query cannot access local variables).
- * if not found fall back to session variables.
- * @param nameParts NameParts of the variable.
- * @return Reference to the variable.
- */
- def lookupVariable(nameParts: Seq[String]): Option[VariableReference] = {
- // The temp variables live in `SYSTEM.SESSION`, and the name can be qualified or not.
- def maybeTempVariableName(nameParts: Seq[String]): Boolean = {
- nameParts.length == 1 || {
- if (nameParts.length == 2) {
- nameParts.head.equalsIgnoreCase(CatalogManager.SESSION_NAMESPACE)
- } else if (nameParts.length == 3) {
- nameParts(0).equalsIgnoreCase(CatalogManager.SYSTEM_CATALOG_NAME) &&
- nameParts(1).equalsIgnoreCase(CatalogManager.SESSION_NAMESPACE)
- } else {
- false
- }
- }
- }
-
- val namePartsCaseAdjusted = if (conf.caseSensitiveAnalysis) {
- nameParts
- } else {
- nameParts.map(_.toLowerCase(Locale.ROOT))
- }
-
- SqlScriptingContextManager.get().map(_.getVariableManager)
- // If we are in EXECUTE IMMEDIATE lookup only session variables.
- .filterNot(_ => AnalysisContext.get.isExecuteImmediate)
- // If variable name is qualified with session. treat it as a session variable.
- .filterNot(_ =>
- nameParts.length > 2 || (nameParts.length == 2 && isForbiddenLabelName(nameParts.head)))
- .flatMap(_.get(namePartsCaseAdjusted))
- .map { varDef =>
- VariableReference(
- nameParts,
- FakeLocalCatalog,
- Identifier.of(Array(varDef.identifier.namespace().last), namePartsCaseAdjusted.last),
- varDef)
- }
- .orElse(
- if (maybeTempVariableName(nameParts)) {
- catalogManager.tempVariableManager
- .get(namePartsCaseAdjusted)
- .map { varDef =>
- VariableReference(
- nameParts,
- FakeSystemCatalog,
- Identifier.of(Array(CatalogManager.SESSION_NAMESPACE), namePartsCaseAdjusted.last),
- varDef
- )}
- } else {
- None
- }
- )
- }
-
// Resolves `UnresolvedAttribute` to its value.
protected def resolveVariables(e: Expression): Expression = {
- def resolveVariable(nameParts: Seq[String]): Option[Expression] = {
- val isResolvingView = AnalysisContext.get.catalogAndNamespace.nonEmpty
- if (isResolvingView) {
- if (AnalysisContext.get.referredTempVariableNames.contains(nameParts)) {
- lookupVariable(nameParts)
- } else {
- None
- }
- } else {
- lookupVariable(nameParts)
- }
- }
+ val variableResolution = new VariableResolution(catalogManager.tempVariableManager)
def resolve(nameParts: Seq[String]): Option[Expression] = {
- var resolvedVariable: Option[Expression] = None
- // We only support temp variables for now, so the variable name can at most have 3 parts.
- var numInnerFields: Int = math.max(0, nameParts.length - 3)
- // Follow the column resolution and prefer the longest match. This makes sure that users
- // can always use fully qualified variable name to avoid name conflicts.
- while (resolvedVariable.isEmpty && numInnerFields < nameParts.length) {
- resolvedVariable = resolveVariable(nameParts.dropRight(numInnerFields))
- if (resolvedVariable.isEmpty) numInnerFields += 1
- }
-
- resolvedVariable.map { variable =>
- if (numInnerFields != 0) {
- val nestedFields = nameParts.takeRight(numInnerFields)
- nestedFields.foldLeft(variable: Expression) { (e, name) =>
- ExtractValue(e, Literal(name), conf.resolver)
- }
- } else {
- variable
- }
- }.map(e => Alias(e, nameParts.last)())
+ variableResolution.resolveMultipartName(
+ nameParts = nameParts,
+ resolvingView = AnalysisContext.get.catalogAndNamespace.nonEmpty,
+ resolvingExecuteImmediate = AnalysisContext.get.isExecuteImmediate,
+ referredTempVariableNames = AnalysisContext.get.referredTempVariableNames
+ ).map(e => Alias(e, nameParts.last)())
}
def innerResolve(e: Expression, isTopLevel: Boolean): Expression = withOrigin(e.origin) {
@@ -505,6 +422,33 @@ trait ColumnResolutionHelper extends Logging with DataTypeErrorsBase {
includeLastResort = includeLastResort)
}
+ // Try to resolve `UnresolvedAttribute` by the children with Plan Ids.
+ // The `UnresolvedAttribute` must have a Plan Id:
+ // - If Plan Id not found in the plan, raise CANNOT_RESOLVE_DATAFRAME_COLUMN.
+ // - If Plan Id found in the plan, but column not found, return None.
+ // - Otherwise, return the resolved expression.
+ private[sql] def tryResolveColumnByPlanChildren(
+ u: UnresolvedAttribute,
+ q: LogicalPlan,
+ includeLastResort: Boolean = false): Option[Expression] = {
+ assert(u.getTagValue(LogicalPlan.PLAN_ID_TAG).nonEmpty,
+ s"UnresolvedAttribute $u should have a Plan Id tag")
+
+ resolveDataFrameColumn(u, q.children).map { r =>
+ resolveExpression(
+ r,
+ resolveColumnByName = nameParts => {
+ q.resolveChildren(nameParts, conf.resolver)
+ },
+ getAttrCandidates = () => {
+ assert(q.children.length == 1)
+ q.children.head.output
+ },
+ throws = true,
+ includeLastResort = includeLastResort)
+ }
+ }
+
/**
* The last resort to resolve columns. Currently it does two things:
* - Try to resolve column names as outer references
@@ -526,10 +470,15 @@ trait ColumnResolutionHelper extends Logging with DataTypeErrorsBase {
// expression are from Spark Connect, and need to be resolved in this way:
// 1. extract the attached plan id from UnresolvedAttribute;
// 2. top-down traverse the query plan to find the plan node that matches the plan id;
- // 3. if can not find the matching node, fail the analysis due to illegal references;
- // 4. if more than one matching nodes are found, fail due to ambiguous column reference;
- // 5. resolve the expression with the matching node, if any error occurs here, return the
- // original expression as it is.
+ // 3. if can not find the matching node, fails with 'CANNOT_RESOLVE_DATAFRAME_COLUMN';
+ // 4, if the matching node is found, but can not resolve the column, also fails with
+ // 'CANNOT_RESOLVE_DATAFRAME_COLUMN';
+ // 5, resolve the expression against the target node, the resolved attribute will be
+ // filtered by the output attributes of nodes in the path (from matching to root node);
+ // 6. if more than one resolved attributes are found in the above recursive process,
+ // fails with 'AMBIGUOUS_COLUMN_REFERENCE'.
+ // 7. if all the resolved attributes are filtered out, return the original expression
+ // as it is.
private def tryResolveDataFrameColumns(
e: Expression,
q: Seq[LogicalPlan]): Expression = e match {
@@ -591,18 +540,16 @@ trait ColumnResolutionHelper extends Logging with DataTypeErrorsBase {
p: LogicalPlan,
currentDepth: Int): (Option[(NamedExpression, Int)], Boolean) = {
val (resolved, matched) = if (p.getTagValue(LogicalPlan.PLAN_ID_TAG).contains(id)) {
- val resolved = try {
- if (!isMetadataAccess) {
- p.resolve(u.nameParts, conf.resolver)
- } else if (u.nameParts.size == 1) {
- p.getMetadataAttributeByNameOpt(u.nameParts.head)
- } else {
- None
- }
- } catch {
- case e: AnalysisException =>
- logDebug(s"Fail to resolve $u with $p due to $e")
- None
+ val resolved = if (!isMetadataAccess) {
+ p.resolve(u.nameParts, conf.resolver)
+ } else if (u.nameParts.size == 1) {
+ p.getMetadataAttributeByNameOpt(u.nameParts.head)
+ } else {
+ None
+ }
+ if (resolved.isEmpty) {
+ // The targe plan node is found, but the column cannot be resolved.
+ throw QueryCompilationErrors.cannotResolveDataFrameColumn(u)
}
(resolved.map(r => (r, currentDepth)), true)
} else {
@@ -631,14 +578,20 @@ trait ColumnResolutionHelper extends Logging with DataTypeErrorsBase {
// When resolving the column reference df1.a, the target node with plan_id=1
// can be found in both sides of the Join node.
// To correctly resolve df1.a, the analyzer discards the resolved attribute
- // in the right side, by filtering out the result by the output attributes of
+ // on the right side, by filtering out the result by the output attributes of
// Project plan_id=2.
//
// However, there are analyzer rules (e.g. ResolveReferencesInSort)
// supporting missing column resolution. Then a valid resolved attribute
- // maybe filtered out here. In this case, resolveDataFrameColumnByPlanId
- // returns None, the dataframe column will remain unresolved, and the analyzer
- // will try to resolve it without plan id later.
+ // maybe filtered out here. For example:
+ //
+ // from pyspark.sql import functions as sf
+ // df = spark.range(10).withColumn("v", sf.col("id") + 1)
+ // df.select(df.v).sort(df.id)
+ //
+ // In this case, resolveDataFrameColumnByPlanId returns None,
+ // the dataframe column 'df.id' will remain unresolved, and the analyzer
+ // will try to resolve 'id' without plan id later.
val filtered = resolved.filter { r =>
if (isMetadataAccess) {
r._1.references.subsetOf(AttributeSet(p.output ++ p.metadataOutput))
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala
index 752a2a648ce99..b8da376bead6f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala
@@ -19,16 +19,14 @@ package org.apache.spark.sql.catalyst.analysis
import scala.collection.mutable
+import org.apache.spark.sql.catalyst.analysis.resolver.ResolverTag
import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap, AttributeReference, AttributeSet, Expression, NamedExpression, OuterReference, SubqueryExpression}
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.catalyst.trees.TreeNodeTag
import org.apache.spark.sql.catalyst.trees.TreePattern._
+import org.apache.spark.sql.internal.SQLConf
object DeduplicateRelations extends Rule[LogicalPlan] {
- val PROJECT_FOR_EXPRESSION_ID_DEDUPLICATION =
- TreeNodeTag[Unit]("project_for_expression_id_deduplication")
-
type ExprIdMap = mutable.HashMap[Class[_], mutable.HashSet[Long]]
override def apply(plan: LogicalPlan): LogicalPlan = {
@@ -59,23 +57,30 @@ object DeduplicateRelations extends Rule[LogicalPlan] {
case e @ Except(left, right, _) if !e.duplicateResolved && noMissingInput(right) =>
e.copy(right = dedupRight(left, right))
// Only after we finish by-name resolution for Union
- case u: Union if !u.byName && !u.duplicateResolved =>
+ case u: Union if !u.byName && !u.duplicatesResolvedBetweenBranches =>
+ val unionWithChildOutputsDeduplicated =
+ DeduplicateUnionChildOutput.deduplicateOutputPerChild(u)
// Use projection-based de-duplication for Union to avoid breaking the checkpoint sharing
// feature in streaming.
- val newChildren = u.children.foldRight(Seq.empty[LogicalPlan]) { (head, tail) =>
- head +: tail.map {
- case child if head.outputSet.intersect(child.outputSet).isEmpty =>
- child
- case child =>
- val projectList = child.output.map { attr =>
- Alias(attr, attr.name)()
+ val newChildren =
+ unionWithChildOutputsDeduplicated.children.foldRight(Seq.empty[LogicalPlan]) {
+ (head, tail) =>
+ head +: tail.map {
+ case child if head.outputSet.intersect(child.outputSet).isEmpty =>
+ child
+ case child =>
+ val projectList = child.output.map { attr =>
+ Alias(attr, attr.name)()
+ }
+ val project = Project(projectList, child)
+ project.setTagValue(
+ ResolverTag.PROJECT_FOR_EXPRESSION_ID_DEDUPLICATION,
+ ()
+ )
+ project
}
- val project = Project(projectList, child)
- project.setTagValue(DeduplicateRelations.PROJECT_FOR_EXPRESSION_ID_DEDUPLICATION, ())
- project
}
- }
- u.copy(children = newChildren)
+ unionWithChildOutputsDeduplicated.copy(children = newChildren)
case merge: MergeIntoTable
if !merge.duplicateResolved && noMissingInput(merge.sourceTable) =>
merge.copy(sourceTable = dedupRight(merge.targetTable, merge.sourceTable))
@@ -237,8 +242,17 @@ object DeduplicateRelations extends Rule[LogicalPlan] {
if (planChanged) {
if (planWithNewSubquery.childrenResolved) {
val planWithNewChildren = planWithNewSubquery.withNewChildren(newChildren.toSeq)
+ val childrenOutputLookup = AttributeSet.fromAttributeSets(newChildren.map(_.outputSet))
+ val childrenOutput = newChildren.flatMap(_.output)
val attrMap = AttributeMap(plan.children.flatMap(_.output)
- .zip(newChildren.flatMap(_.output)).filter { case (a1, a2) => a1.exprId != a2.exprId })
+ .zip(childrenOutput).filter { case (a1, a2) => a1.exprId != a2.exprId })
+ val preventDeduplicationIfOldExprIdStillExists =
+ conf.getConf(SQLConf.DONT_DEDUPLICATE_EXPRESSION_IF_EXPR_ID_IN_OUTPUT)
+ val missingAttributeMap = AttributeMap(attrMap.filter {
+ case (oldAttribute, _) =>
+ !preventDeduplicationIfOldExprIdStillExists ||
+ !childrenOutputLookup.contains(oldAttribute)
+ })
if (attrMap.isEmpty) {
planWithNewChildren
} else {
@@ -282,7 +296,7 @@ object DeduplicateRelations extends Rule[LogicalPlan] {
rightDeserializer = newRightDes, leftGroup = newLeftGroup,
rightGroup = newRightGroup, leftAttr = newLeftAttr, rightAttr = newRightAttr,
leftOrder = newLeftOrder, rightOrder = newRightOrder)
- case _ => planWithNewChildren.rewriteAttrs(attrMap)
+ case _ => planWithNewChildren.rewriteAttrs(missingAttributeMap)
}
}
} else {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateUnionChildOutput.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateUnionChildOutput.scala
new file mode 100644
index 0000000000000..11a10710a6170
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateUnionChildOutput.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.catalyst.analysis
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, ExprId}
+import org.apache.spark.sql.catalyst.plans.logical.{Project, Union}
+import org.apache.spark.sql.types.MetadataBuilder
+
+/**
+ * Deduplicates columns with same [[ExprId]]s in single [[Union]] child output, by placing aliases
+ * on non-first duplicates.
+ */
+object DeduplicateUnionChildOutput {
+
+ /**
+ * Deduplicate expression IDs at the scope of each individual child output. This is necessary to
+ * handle the following case:
+ *
+ * {{{
+ * -- The correct answer is (1, 1), (1, 2). Without deduplication it would be (1, 1), because
+ * -- aggregation would be done only based on the first column.
+ * SELECT
+ * a, a
+ * FROM
+ * VALUES (1, 1), (1, 2) AS t1 (a, b)
+ * UNION
+ * SELECT
+ * a, b
+ * FROM
+ * VALUES (1, 1), (1, 2) AS t2 (a, b)
+ * }}}
+ *
+ * Putting [[Alias]] introduces a new expression ID for the attribute duplicates in the output. We
+ * also add `__is_duplicate` metadata so that [[AttributeSeq.getCandidatesForResolution]] doesn't
+ * produce conflicting candidates when resolving names in the upper [[Project]] - this is
+ * technically still the same attribute.
+ *
+ * See SPARK-37865 for more details.
+ */
+ def deduplicateOutputPerChild(union: Union): Union = {
+ val newChildren = union.children.map { c =>
+ if (c.output.map(_.exprId).distinct.length < c.output.length) {
+ val existingExprIds = mutable.HashSet[ExprId]()
+ val projectList = c.output.map { attr =>
+ if (existingExprIds.contains(attr.exprId)) {
+ // replace non-first duplicates with aliases and tag them
+ val newMetadata = new MetadataBuilder()
+ .withMetadata(attr.metadata)
+ .putNull("__is_duplicate")
+ .build()
+ Alias(attr, attr.name)(explicitMetadata = Some(newMetadata))
+ } else {
+ // leave first duplicate alone
+ existingExprIds.add(attr.exprId)
+ attr
+ }
+ }
+ Project(projectList, c)
+ } else {
+ c
+ }
+ }
+ union.withNewChildren(newChildren).asInstanceOf[Union]
+ }
+}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
index 66db1fe8b5965..99e0c707d8871 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
@@ -24,7 +24,7 @@ import scala.collection.mutable
import scala.reflect.ClassTag
import org.apache.spark.SparkUnsupportedOperationException
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.FUNCTION_NAME
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.FunctionIdentifier
@@ -463,6 +463,7 @@ object FunctionRegistry {
expressionBuilder("try_sum", TrySumExpressionBuilder, setAlias = true),
expression[TryToBinary]("try_to_binary"),
expressionBuilder("try_to_timestamp", TryToTimestampExpressionBuilder, setAlias = true),
+ expressionBuilder("try_to_date", TryToDateExpressionBuilder, setAlias = true),
expressionBuilder("try_to_time", TryToTimeExpressionBuilder, setAlias = true),
expression[TryAesDecrypt]("try_aes_decrypt"),
expression[TryReflect]("try_reflect"),
@@ -527,6 +528,8 @@ object FunctionRegistry {
expressionBuilder("mode", ModeBuilder),
expression[HllSketchAgg]("hll_sketch_agg"),
expression[HllUnionAgg]("hll_union_agg"),
+ expression[ApproxTopK]("approx_top_k"),
+ expression[ApproxTopKAccumulate]("approx_top_k_accumulate"),
// string functions
expression[Ascii]("ascii"),
@@ -652,6 +655,7 @@ object FunctionRegistry {
expressionBuilder("second", SecondExpressionBuilder),
expression[ParseToTimestamp]("to_timestamp"),
expression[ParseToDate]("to_date"),
+ expression[TimeDiff]("time_diff"),
expression[ToTime]("to_time"),
expression[ToBinary]("to_binary"),
expression[ToUnixTimestamp]("to_unix_timestamp"),
@@ -672,7 +676,8 @@ object FunctionRegistry {
expression[WindowTime]("window_time"),
expression[MakeDate]("make_date"),
expression[MakeTime]("make_time"),
- expression[MakeTimestamp]("make_timestamp"),
+ expression[TimeTrunc]("time_trunc"),
+ expressionBuilder("make_timestamp", MakeTimestampExpressionBuilder),
expression[TryMakeTimestamp]("try_make_timestamp"),
expression[MonthName]("monthname"),
// We keep the 2 expression builders below to have different function docs.
@@ -785,6 +790,7 @@ object FunctionRegistry {
expression[EqualNull]("equal_null"),
expression[HllSketchEstimate]("hll_sketch_estimate"),
expression[HllUnion]("hll_union"),
+ expression[ApproxTopKEstimate]("approx_top_k_estimate"),
// grouping sets
expression[Grouping]("grouping"),
@@ -870,6 +876,7 @@ object FunctionRegistry {
castAlias("decimal", DecimalType.USER_DEFAULT),
castAlias("date", DateType),
castAlias("timestamp", TimestampType),
+ castAlias("time", TimeType()),
castAlias("binary", BinaryType),
castAlias("string", StringType),
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HintErrorLogger.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HintErrorLogger.scala
index 5301a3683c7da..8edc5278b5319 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HintErrorLogger.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HintErrorLogger.scala
@@ -17,7 +17,7 @@
package org.apache.spark.sql.catalyst.analysis
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{QUERY_HINT, RELATION_NAME, UNSUPPORTED_HINT_REASON}
import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.catalyst.plans.logical.{HintErrorHandler, HintInfo}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/IdentifierResolution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/IdentifierResolution.scala
new file mode 100644
index 0000000000000..e274c1e2162a5
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/IdentifierResolution.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.catalyst.analysis
+
+import org.apache.spark.sql.catalyst.expressions.{AliasHelper, EvalHelper, Expression}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
+import org.apache.spark.sql.types.StringType
+
+object IdentifierResolution extends AliasHelper with EvalHelper {
+ def evalIdentifierExpr(expr: Expression): Seq[String] = {
+ trimAliases(prepareForEval(expr)) match {
+ case e if !e.foldable =>
+ expr.failAnalysis(
+ errorClass = "NOT_A_CONSTANT_STRING.NOT_CONSTANT",
+ messageParameters = Map("name" -> "IDENTIFIER", "expr" -> expr.sql)
+ )
+ case e if e.dataType != StringType =>
+ expr.failAnalysis(
+ errorClass = "NOT_A_CONSTANT_STRING.WRONG_TYPE",
+ messageParameters =
+ Map("name" -> "IDENTIFIER", "expr" -> expr.sql, "dataType" -> e.dataType.catalogString)
+ )
+ case e =>
+ e.eval() match {
+ case null =>
+ expr.failAnalysis(
+ errorClass = "NOT_A_CONSTANT_STRING.NULL",
+ messageParameters = Map("name" -> "IDENTIFIER", "expr" -> expr.sql)
+ )
+ case other =>
+ // Parse the identifier string to name parts.
+ CatalystSqlParser.parseMultipartIdentifier(other.toString)
+ }
+ }
+ }
+}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NondeterministicExpressionCollection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NondeterministicExpressionCollection.scala
index d530cfe5175ba..1dc5d79dc1b59 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NondeterministicExpressionCollection.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NondeterministicExpressionCollection.scala
@@ -38,7 +38,7 @@ object NondeterministicExpressionCollection {
case namedExpression: NamedExpression => namedExpression
case _ => Alias(nondeterministicExpr, "_nondeterministic")()
}
- nonDeterministicToAttributes.put(nondeterministicExpr, namedExpression)
+ nonDeterministicToAttributes.put(nondeterministicExpr.canonicalized, namedExpression)
}
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/PullOutNondeterministic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/PullOutNondeterministic.scala
index e0b984540cac0..09d3a6f93a878 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/PullOutNondeterministic.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/PullOutNondeterministic.scala
@@ -19,7 +19,9 @@ package org.apache.spark.sql.catalyst.analysis
import scala.jdk.CollectionConverters._
+import org.apache.spark.SparkException
import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.ExprUtils.toSQLExpr
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules.Rule
@@ -39,10 +41,20 @@ object PullOutNondeterministic extends Rule[LogicalPlan] {
val nondeterToAttr =
NondeterministicExpressionCollection.getNondeterministicToAttributes(a.groupingExpressions)
val newChild = Project(a.child.output ++ nondeterToAttr.values.asScala.toSeq, a.child)
- a.transformExpressions { case e =>
- Option(nondeterToAttr.get(e)).map(_.toAttribute).getOrElse(e)
+ val deterministicAggregate = a.transformExpressions { case e =>
+ Option(nondeterToAttr.get(e.canonicalized)).map(_.toAttribute).getOrElse(e)
}.copy(child = newChild)
+ deterministicAggregate.groupingExpressions.foreach(expr => if (!expr.deterministic) {
+ throw SparkException.internalError(
+ msg = s"Non-deterministic expression '${toSQLExpr(expr)}' should not appear in " +
+ "grouping expression.",
+ context = expr.origin.getQueryContext,
+ summary = expr.origin.context.summary)
+ })
+
+ deterministicAggregate
+
// Don't touch collect metrics. Top-level metrics are not supported (check analysis will fail)
// and we want to retain them inside the aggregate functions.
case m: CollectMetrics => m
@@ -57,7 +69,7 @@ object PullOutNondeterministic extends Rule[LogicalPlan] {
val nondeterToAttr =
NondeterministicExpressionCollection.getNondeterministicToAttributes(p.expressions)
val newPlan = p.transformExpressions { case e =>
- Option(nondeterToAttr.get(e)).map(_.toAttribute).getOrElse(e)
+ Option(nondeterToAttr.get(e.canonicalized)).map(_.toAttribute).getOrElse(e)
}
val newChild = Project(p.child.output ++ nondeterToAttr.values.asScala.toSeq, p.child)
Project(p.output, newPlan.withNewChildren(newChild :: Nil))
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala
index 72d92e5a9445e..851db598c4e39 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala
@@ -40,35 +40,40 @@ class ResolveCatalogs(val catalogManager: CatalogManager)
override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperatorsDown {
// We only support temp variables for now and the system catalog is not properly implemented
// yet. We need to resolve `UnresolvedIdentifier` for variable commands specially.
- case c @ CreateVariable(UnresolvedIdentifier(nameParts, _), _, _) =>
- // From scripts we can only create local variables, which must be unqualified,
- // and must not be DECLARE OR REPLACE.
- val resolved = if (withinSqlScript) {
- if (c.replace) {
- throw new AnalysisException(
- "INVALID_VARIABLE_DECLARATION.REPLACE_LOCAL_VARIABLE",
- Map("varName" -> toSQLId(nameParts))
- )
- }
-
- if (nameParts.length != 1) {
- throw new AnalysisException(
- "INVALID_VARIABLE_DECLARATION.QUALIFIED_LOCAL_VARIABLE",
- Map("varName" -> toSQLId(nameParts)))
- }
-
- SqlScriptingContextManager.get().map(_.getVariableManager)
- .getOrElse(throw SparkException.internalError(
- "Scripting local variable manager should be present in SQL script."))
- .qualify(nameParts.last)
- } else {
- val resolvedIdentifier = catalogManager.tempVariableManager.qualify(nameParts.last)
-
- assertValidSessionVariableNameParts(nameParts, resolvedIdentifier)
- resolvedIdentifier
+ case c @ CreateVariable(identifiers, _, _) =>
+ // We resolve only UnresolvedIdentifiers, and pass on the other nodes
+ val resolved = identifiers.map {
+ case UnresolvedIdentifier(nameParts, _) =>
+ // From scripts we can only create local variables, which must be unqualified,
+ // and must not be DECLARE OR REPLACE.
+ if (withinSqlScript) {
+ if (c.replace) {
+ throw new AnalysisException(
+ "INVALID_VARIABLE_DECLARATION.REPLACE_LOCAL_VARIABLE",
+ Map("varName" -> toSQLId(nameParts))
+ )
+ }
+
+ if (nameParts.length != 1) {
+ throw new AnalysisException(
+ "INVALID_VARIABLE_DECLARATION.QUALIFIED_LOCAL_VARIABLE",
+ Map("varName" -> toSQLId(nameParts)))
+ }
+
+ SqlScriptingContextManager.get().map(_.getVariableManager)
+ .getOrElse(throw SparkException.internalError(
+ "Scripting local variable manager should be present in SQL script."))
+ .qualify(nameParts.last)
+ } else {
+ val resolvedIdentifier
+ = catalogManager.tempVariableManager.qualify(nameParts.last)
+
+ assertValidSessionVariableNameParts(nameParts, resolvedIdentifier)
+ resolvedIdentifier
+ }
+ case plan => plan
}
-
- c.copy(name = resolved)
+ c.copy(names = resolved)
case d @ DropVariable(UnresolvedIdentifier(nameParts, _), _) =>
if (withinSqlScript) {
throw new AnalysisException(
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDataFrameDropColumns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDataFrameDropColumns.scala
index 0f9b93cc2986d..a0f67fa3f445f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDataFrameDropColumns.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDataFrameDropColumns.scala
@@ -17,8 +17,8 @@
package org.apache.spark.sql.catalyst.analysis
+import org.apache.spark.sql.catalyst.SQLConfHelper
import org.apache.spark.sql.catalyst.plans.logical.{DataFrameDropColumns, LogicalPlan, Project}
-import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.catalyst.trees.TreePattern.DF_DROP_COLUMNS
import org.apache.spark.sql.connector.catalog.CatalogManager
@@ -27,17 +27,24 @@ import org.apache.spark.sql.connector.catalog.CatalogManager
* Note that DataFrameDropColumns allows and ignores non-existing columns.
*/
class ResolveDataFrameDropColumns(val catalogManager: CatalogManager)
- extends Rule[LogicalPlan] with ColumnResolutionHelper {
+ extends SQLConfHelper with ColumnResolutionHelper {
- override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
+ def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
_.containsPattern(DF_DROP_COLUMNS)) {
case d: DataFrameDropColumns if d.childrenResolved =>
// expressions in dropList can be unresolved, e.g.
// df.drop(col("non-existing-column"))
- val dropped = d.dropList.map {
+ val dropped = d.dropList.flatMap {
case u: UnresolvedAttribute =>
- resolveExpressionByPlanChildren(u, d)
- case e => e
+ if (u.getTagValue(LogicalPlan.PLAN_ID_TAG).nonEmpty) {
+ // Plan Id comes from Spark Connect,
+ // Here we ignore the `UnresolvedAttribute` if its Plan Id can be found
+ // but column not found.
+ tryResolveColumnByPlanChildren(u, d)
+ } else {
+ Some(resolveExpressionByPlanChildren(u, d))
+ }
+ case e => Some(e)
}
val remaining = d.child.output.filterNot(attr => dropped.exists(_.semanticEquals(attr)))
if (remaining.size == d.child.output.size) {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveIdentifierClause.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveIdentifierClause.scala
index 96ed8a63585d0..7150c81ad64ec 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveIdentifierClause.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveIdentifierClause.scala
@@ -19,20 +19,18 @@ package org.apache.spark.sql.catalyst.analysis
import scala.collection.mutable
-import org.apache.spark.sql.catalyst.expressions.{AliasHelper, EvalHelper, Expression, SubqueryExpression, VariableReference}
-import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
+import org.apache.spark.sql.catalyst.expressions.{Expression, SubqueryExpression, VariableReference}
import org.apache.spark.sql.catalyst.plans.logical.{CreateView, LogicalPlan}
import org.apache.spark.sql.catalyst.rules.{Rule, RuleExecutor}
import org.apache.spark.sql.catalyst.trees.TreePattern._
import org.apache.spark.sql.errors.QueryCompilationErrors
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.types.StringType
/**
* Resolves the identifier expressions and builds the original plans/expressions.
*/
class ResolveIdentifierClause(earlyBatches: Seq[RuleExecutor[LogicalPlan]#Batch])
- extends Rule[LogicalPlan] with AliasHelper with EvalHelper {
+ extends Rule[LogicalPlan] {
private val executor = new RuleExecutor[LogicalPlan] {
override def batches: Seq[Batch] = earlyBatches.asInstanceOf[Seq[Batch]]
@@ -70,7 +68,8 @@ class ResolveIdentifierClause(earlyBatches: Seq[RuleExecutor[LogicalPlan]#Batch]
referredTempVars.get ++= collectTemporaryVariablesInLogicalPlan(p)
}
- executor.execute(p.planBuilder.apply(evalIdentifierExpr(p.identifierExpr), p.children))
+ executor.execute(p.planBuilder.apply(
+ IdentifierResolution.evalIdentifierExpr(p.identifierExpr), p.children))
case other =>
other.transformExpressionsWithPruning(_.containsAnyPattern(UNRESOLVED_IDENTIFIER)) {
case e: ExpressionWithUnresolvedIdentifier if e.identifierExpr.resolved =>
@@ -79,7 +78,8 @@ class ResolveIdentifierClause(earlyBatches: Seq[RuleExecutor[LogicalPlan]#Batch]
referredTempVars.get ++= collectTemporaryVariablesInExpressionTree(e)
}
- e.exprBuilder.apply(evalIdentifierExpr(e.identifierExpr), e.otherExprs)
+ e.exprBuilder.apply(
+ IdentifierResolution.evalIdentifierExpr(e.identifierExpr), e.otherExprs)
}
}
@@ -104,31 +104,4 @@ class ResolveIdentifierClause(earlyBatches: Seq[RuleExecutor[LogicalPlan]#Batch]
}
collectTempVars(child)
}
-
- private def evalIdentifierExpr(expr: Expression): Seq[String] = {
- trimAliases(prepareForEval(expr)) match {
- case e if !e.foldable => expr.failAnalysis(
- errorClass = "NOT_A_CONSTANT_STRING.NOT_CONSTANT",
- messageParameters = Map(
- "name" -> "IDENTIFIER",
- "expr" -> expr.sql))
- case e if e.dataType != StringType => expr.failAnalysis(
- errorClass = "NOT_A_CONSTANT_STRING.WRONG_TYPE",
- messageParameters = Map(
- "name" -> "IDENTIFIER",
- "expr" -> expr.sql,
- "dataType" -> e.dataType.catalogString))
- case e =>
- e.eval() match {
- case null => expr.failAnalysis(
- errorClass = "NOT_A_CONSTANT_STRING.NULL",
- messageParameters = Map(
- "name" -> "IDENTIFIER",
- "expr" -> expr.sql))
- case other =>
- // Parse the identifier string to name parts.
- CatalystSqlParser.parseMultipartIdentifier(other.toString)
- }
- }
- }
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
index 62f3997491c07..08cb70ddd5dba 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
@@ -17,11 +17,11 @@
package org.apache.spark.sql.catalyst.analysis
-import org.apache.spark.sql.catalyst.EvaluateUnresolvedInlineTable
import org.apache.spark.sql.catalyst.expressions.EvalHelper
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.catalyst.trees.AlwaysProcess
+import org.apache.spark.sql.catalyst.util.EvaluateUnresolvedInlineTable
/**
* An analyzer rule that replaces [[UnresolvedInlineTable]] with [[ResolvedInlineTable]].
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveLateralColumnAliasReference.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveLateralColumnAliasReference.scala
index 632d01f18ec2a..042e4483a37d3 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveLateralColumnAliasReference.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveLateralColumnAliasReference.scala
@@ -113,7 +113,7 @@ import org.apache.spark.sql.internal.SQLConf
* [[ExtractWindowExpressions]].
*/
// scalastyle:on line.size.limit
-object ResolveLateralColumnAliasReference extends Rule[LogicalPlan] {
+object ResolveLateralColumnAliasReference extends Rule[LogicalPlan] with AliasHelper {
case class AliasEntry(alias: Alias, index: Int)
private def assignAlias(expr: Expression): NamedExpression = {
@@ -170,7 +170,7 @@ object ResolveLateralColumnAliasReference extends Rule[LogicalPlan] {
UnresolvedAttribute(lcaRef.nameParts)
}.asInstanceOf[NamedExpression]
}
- val newProjectList = projectList.zipWithIndex.map {
+ val newProjectList = projectList.map(trimNonTopLevelAliases).zipWithIndex.map {
case (a: Alias, idx) =>
val lcaResolved = unwrapLCAReference(a)
// Insert the original alias instead of rewritten one to detect chained LCA
@@ -229,7 +229,9 @@ object ResolveLateralColumnAliasReference extends Rule[LogicalPlan] {
case e => e.children.forall(eligibleToLiftUp)
}
}
- if (!aggregateExpressions.forall(eligibleToLiftUp)) {
+ val aggregateExpressionsWithTrimmedAliases =
+ aggregateExpressions.map(trimNonTopLevelAliases)
+ if (!aggregateExpressionsWithTrimmedAliases.forall(eligibleToLiftUp)) {
agg
} else {
val newAggExprs = new LinkedHashSet[NamedExpression]
@@ -268,8 +270,9 @@ object ResolveLateralColumnAliasReference extends Rule[LogicalPlan] {
case e => e.mapChildren(extractExpressions)
}
}
- val projectExprs = aggregateExpressions.map(
- extractExpressions(_).asInstanceOf[NamedExpression])
+ val projectExprs = aggregateExpressionsWithTrimmedAliases.map(
+ extractExpressions(_).asInstanceOf[NamedExpression]
+ )
val newProject = Project(
projectList = projectExprs,
child = agg.copy(aggregateExpressions = newAggExprs.asScala.toSeq)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveMergeIntoSchemaEvolution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveMergeIntoSchemaEvolution.scala
new file mode 100644
index 0000000000000..7e7776098a04a
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveMergeIntoSchemaEvolution.scala
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.types.DataTypeUtils
+import org.apache.spark.sql.connector.catalog.{CatalogV2Util, TableCatalog}
+import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+
+
+/**
+ * A rule that resolves schema evolution for MERGE INTO.
+ *
+ * This rule will call the DSV2 Catalog to update the schema of the target table.
+ */
+object ResolveMergeIntoSchemaEvolution extends Rule[LogicalPlan] {
+
+ override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+ case m @ MergeIntoTable(_, _, _, _, _, _, _)
+ if m.needSchemaEvolution =>
+ val newTarget = m.targetTable.transform {
+ case r : DataSourceV2Relation => performSchemaEvolution(r, m.sourceTable)
+ }
+ m.copy(targetTable = newTarget)
+ }
+
+ private def performSchemaEvolution(relation: DataSourceV2Relation, source: LogicalPlan)
+ : DataSourceV2Relation = {
+ (relation.catalog, relation.identifier) match {
+ case (Some(c: TableCatalog), Some(i)) =>
+ val changes = MergeIntoTable.schemaChanges(relation.schema, source.schema)
+ c.alterTable(i, changes: _*)
+ val newTable = c.loadTable(i)
+ val newSchema = CatalogV2Util.v2ColumnsToStructType(newTable.columns())
+ // Check if there are any remaining changes not applied.
+ val remainingChanges = MergeIntoTable.schemaChanges(newSchema, source.schema)
+ if (remainingChanges.nonEmpty) {
+ throw QueryCompilationErrors.unsupportedTableChangesInAutoSchemaEvolutionError(
+ remainingChanges, i.toQualifiedNameParts(c))
+ }
+ relation.copy(table = newTable, output = DataTypeUtils.toAttributes(newSchema))
+ case _ => logWarning(s"Schema Evolution enabled but data source $relation " +
+ s"does not support it, skipping.")
+ relation
+ }
+ }
+}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveReferencesInSort.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveReferencesInSort.scala
index 6fa723d4a75fd..6e120ef015026 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveReferencesInSort.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveReferencesInSort.scala
@@ -20,53 +20,78 @@ import org.apache.spark.sql.catalyst.SQLConfHelper
import org.apache.spark.sql.catalyst.expressions.SortOrder
import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, LogicalPlan, Project, Sort}
import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.internal.SQLConf
/**
* A virtual rule to resolve [[UnresolvedAttribute]] in [[Sort]]. It's only used by the real
* rule `ResolveReferences`. The column resolution order for [[Sort]] is:
- * 1. Resolves the column to [[AttributeReference]] with the output of the child plan. This
+ * 1. Checks whether there are [[UnresolvedOrdinal]]s in the sort order list. In case there are
+ * delay the resolution until we resolve all the ordinals. Without this check, we proceed to
+ * resolve the following query correctly:
+ * {{{ SELECT col1 FROM VALUES(1, 2) ORDER BY 2, col2; }}}
+ * That's because we add missing input in `ResolveReferencesInSort` to the underlying operator
+ * and then successfully resolve the ordinal because at that point there are two elements below.
+ * 2. Resolves the column to [[AttributeReference]] with the output of the child plan. This
* includes metadata columns as well.
- * 2. Resolves the column to a literal function which is allowed to be invoked without braces, e.g.
+ * 3. Resolves the column to a literal function which is allowed to be invoked without braces, e.g.
* `SELECT col, current_date FROM t`.
- * 3. If the child plan is Aggregate or Filter(_, Aggregate), resolves the column to
+ * 4. If the child plan is Aggregate or Filter(_, Aggregate), resolves the column to
* [[TempResolvedColumn]] with the output of Aggregate's child plan.
* This is to allow Sort to host grouping expressions and aggregate functions, which can
* be pushed down to the Aggregate later. For example,
* `SELECT max(a) FROM t GROUP BY b HAVING max(a) > 1 ORDER BY min(a)`.
- * 4. Resolves the column to [[AttributeReference]] with the output of a descendant plan node.
+ * 5. Resolves the column to [[AttributeReference]] with the output of a descendant plan node.
* Spark will propagate the missing attributes from the descendant plan node to the Sort node.
* This is to allow users to ORDER BY columns that are not in the SELECT clause, which is
* widely supported in other SQL dialects. For example, `SELECT a FROM t ORDER BY b`.
- * 5. If the order by expressions only have one single unresolved column named ALL, expanded it to
+ * 6. If the order by expressions only have one single unresolved column named ALL, expanded it to
* include all columns in the SELECT list. This is to support SQL pattern like
* `SELECT col1, col2 FROM t ORDER BY ALL`. This should also support specifying asc/desc, and
* nulls first/last.
- * 6. Resolves the column to outer references with the outer plan if we are resolving subquery
+ * 7. Resolves the column to outer references with the outer plan if we are resolving subquery
* expressions.
*
- * Note, 3 and 4 are actually orthogonal. If the child plan is Aggregate, 4 can only resolve columns
- * as the grouping columns, which is completely covered by 3.
+ * Note, 4 and 5 are actually orthogonal. If the child plan is Aggregate, 5 can only resolve columns
+ * as the grouping columns, which is completely covered by 4.
*/
class ResolveReferencesInSort(val catalogManager: CatalogManager)
extends SQLConfHelper with ColumnResolutionHelper {
def apply(s: Sort): LogicalPlan = {
- val resolvedBasic = s.order.map(resolveExpressionByPlanOutput(_, s.child))
- val resolvedWithAgg = s.child match {
+ if (conf.getConf(SQLConf.PRIORITIZE_ORDINAL_RESOLUTION_IN_SORT) && hasUnresolvedOrdinals(s)) {
+ s
+ } else {
+ resolveReferencesInSort(s)
+ }
+ }
+
+ private def hasUnresolvedOrdinals(sort: Sort): Boolean = {
+ sort.order.exists { sortOrder =>
+ sortOrder.child match {
+ case _: UnresolvedOrdinal => true
+ case _ => false
+ }
+ }
+ }
+
+ private def resolveReferencesInSort(sort: Sort): LogicalPlan = {
+ val resolvedBasic = sort.order.map(resolveExpressionByPlanOutput(_, sort.child))
+ val resolvedWithAgg = sort.child match {
case Filter(_, agg: Aggregate) => resolvedBasic.map(resolveColWithAgg(_, agg))
- case _ => resolvedBasic.map(resolveColWithAgg(_, s.child))
+ case _ => resolvedBasic.map(resolveColWithAgg(_, sort.child))
}
- val (missingAttrResolved, newChild) = resolveExprsAndAddMissingAttrs(resolvedWithAgg, s.child)
+ val (missingAttrResolved, newChild) =
+ resolveExprsAndAddMissingAttrs(resolvedWithAgg, sort.child)
val orderByAllResolved = resolveOrderByAll(
- s.global, newChild, missingAttrResolved.map(_.asInstanceOf[SortOrder]))
+ sort.global, newChild, missingAttrResolved.map(_.asInstanceOf[SortOrder]))
val resolvedFinal = orderByAllResolved
.map(e => resolveColsLastResort(e).asInstanceOf[SortOrder])
- if (s.child.output == newChild.output) {
- s.copy(order = resolvedFinal)
+ if (sort.child.output == newChild.output) {
+ sort.copy(order = resolvedFinal)
} else {
// Add missing attributes and then project them away.
- val newSort = s.copy(order = resolvedFinal, child = newChild)
- Project(s.child.output, newSort)
+ val newSort = sort.copy(order = resolvedFinal, child = newChild)
+ Project(sort.child.output, newSort)
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveRowLevelCommandAssignments.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveRowLevelCommandAssignments.scala
index 3f3e707b054bb..83520b780f121 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveRowLevelCommandAssignments.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveRowLevelCommandAssignments.scala
@@ -48,7 +48,8 @@ object ResolveRowLevelCommandAssignments extends Rule[LogicalPlan] {
case u: UpdateTable if !u.skipSchemaResolution && u.resolved && !u.aligned =>
resolveAssignments(u)
- case m: MergeIntoTable if !m.skipSchemaResolution && m.resolved && m.rewritable && !m.aligned =>
+ case m: MergeIntoTable if !m.skipSchemaResolution && m.resolved && m.rewritable && !m.aligned &&
+ !m.needSchemaEvolution =>
validateStoreAssignmentPolicy()
m.copy(
targetTable = cleanAttrMetadata(m.targetTable),
@@ -56,7 +57,8 @@ object ResolveRowLevelCommandAssignments extends Rule[LogicalPlan] {
notMatchedActions = alignActions(m.targetTable.output, m.notMatchedActions),
notMatchedBySourceActions = alignActions(m.targetTable.output, m.notMatchedBySourceActions))
- case m: MergeIntoTable if !m.skipSchemaResolution && m.resolved && !m.aligned =>
+ case m: MergeIntoTable if !m.skipSchemaResolution && m.resolved && !m.aligned
+ && !m.needSchemaEvolution =>
resolveAssignments(m)
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSetVariable.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSetVariable.scala
index 24b6b04de514b..ab44084357670 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSetVariable.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSetVariable.scala
@@ -33,6 +33,7 @@ import org.apache.spark.sql.types.IntegerType
*/
class ResolveSetVariable(val catalogManager: CatalogManager) extends Rule[LogicalPlan]
with ColumnResolutionHelper {
+ private val variableResolution = new VariableResolution(catalogManager.tempVariableManager)
override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
_.containsPattern(COMMAND), ruleId) {
@@ -40,7 +41,10 @@ class ResolveSetVariable(val catalogManager: CatalogManager) extends Rule[Logica
case setVariable: SetVariable if !setVariable.targetVariables.forall(_.resolved) =>
val resolvedVars = setVariable.targetVariables.map {
case u: UnresolvedAttribute =>
- lookupVariable(u.nameParts) match {
+ variableResolution.lookupVariable(
+ nameParts = u.nameParts,
+ resolvingExecuteImmediate = AnalysisContext.get.isExecuteImmediate
+ ) match {
case Some(variable) => variable.copy(canFold = false)
case _ => throw unresolvedVariableError(u.nameParts, Seq("SYSTEM", "SESSION"))
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableConstraints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableConstraints.scala
index 3b86b9580ae19..41631b24a83ed 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableConstraints.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableConstraints.scala
@@ -19,17 +19,22 @@ package org.apache.spark.sql.catalyst.analysis
import scala.collection.mutable
import org.apache.spark.sql.catalyst.expressions.{And, CheckInvariant, Expression, V2ExpressionUtils}
-import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, V2WriteCommand}
+import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, V2WriteCommand, WriteDelta}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.catalyst.trees.TreePattern.COMMAND
import org.apache.spark.sql.connector.catalog.CatalogManager
import org.apache.spark.sql.connector.catalog.constraints.Check
+import org.apache.spark.sql.connector.write.RowLevelOperation
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
class ResolveTableConstraints(val catalogManager: CatalogManager) extends Rule[LogicalPlan] {
override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
_.containsPattern(COMMAND), ruleId) {
+ // Deleting a delta of rows from an existing table doesn't produce any new rows, thus enforcing
+ // check constraints is unnecessary.
+ case w: WriteDelta if w.operation.command() == RowLevelOperation.Command.DELETE =>
+ w
case v2Write: V2WriteCommand
if v2Write.table.resolved && v2Write.query.resolved &&
!containsCheckInvariant(v2Write.query) && v2Write.outputResolved =>
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveWithCTE.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveWithCTE.scala
index 2a522e98a7683..0f6fcec467ce1 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveWithCTE.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveWithCTE.scala
@@ -59,6 +59,10 @@ object ResolveWithCTE extends Rule[LogicalPlan] {
cteDefMap.put(cteDef.id, cteDef)
}
cteDef
+ case cteDef if cteDef.hasSelfReferenceInAnchor || cteDef.hasSelfReferenceInSubCTE =>
+ throw new AnalysisException(
+ errorClass = "INVALID_RECURSIVE_CTE",
+ messageParameters = Map.empty)
case cteDef =>
// Multiple self-references are not allowed within one cteDef.
cteDef.child match {
@@ -306,7 +310,10 @@ object ResolveWithCTE extends Rule[LogicalPlan] {
columnNames: Option[Seq[String]]) = {
recursion.transformUpWithSubqueriesAndPruning(_.containsPattern(CTE)) {
case r: CTERelationRef if r.recursive && r.cteId == cteDefId =>
- val ref = UnionLoopRef(r.cteId, anchor.output.map(_.newInstance()), false)
+ // We mark the output of UnionLoopRef as nullable as it may become NULL in the following
+ // iterations.
+ val ref =
+ UnionLoopRef(r.cteId, anchor.output.map(_.newInstance().withNullability(true)), false)
columnNames.map(UnresolvedSubqueryColumnAliases(_, ref)).getOrElse(ref)
}
}
@@ -319,35 +326,42 @@ object ResolveWithCTE extends Rule[LogicalPlan] {
def checkIfSelfReferenceIsPlacedCorrectly(
plan: LogicalPlan,
cteId: Long,
- allowRecursiveRef: Boolean = true): Unit = plan match {
- case Join(left, right, Inner, _, _) =>
- checkIfSelfReferenceIsPlacedCorrectly(left, cteId, allowRecursiveRef)
- checkIfSelfReferenceIsPlacedCorrectly(right, cteId, allowRecursiveRef)
- case Join(left, right, Cross, _, _) =>
- checkIfSelfReferenceIsPlacedCorrectly(left, cteId, allowRecursiveRef)
- checkIfSelfReferenceIsPlacedCorrectly(right, cteId, allowRecursiveRef)
- case Join(left, right, LeftOuter, _, _) =>
- checkIfSelfReferenceIsPlacedCorrectly(left, cteId, allowRecursiveRef)
- checkIfSelfReferenceIsPlacedCorrectly(right, cteId, allowRecursiveRef = false)
- case Join(left, right, RightOuter, _, _) =>
- checkIfSelfReferenceIsPlacedCorrectly(left, cteId, allowRecursiveRef = false)
- checkIfSelfReferenceIsPlacedCorrectly(right, cteId, allowRecursiveRef)
- case Join(left, right, LeftSemi, _, _) =>
- checkIfSelfReferenceIsPlacedCorrectly(left, cteId, allowRecursiveRef)
- checkIfSelfReferenceIsPlacedCorrectly(right, cteId, allowRecursiveRef = false)
- case Join(left, right, LeftAnti, _, _) =>
- checkIfSelfReferenceIsPlacedCorrectly(left, cteId, allowRecursiveRef)
- checkIfSelfReferenceIsPlacedCorrectly(right, cteId, allowRecursiveRef = false)
- case Join(left, right, _, _, _) =>
- checkIfSelfReferenceIsPlacedCorrectly(left, cteId, allowRecursiveRef = false)
- checkIfSelfReferenceIsPlacedCorrectly(right, cteId, allowRecursiveRef = false)
- case Aggregate(_, _, child, _) =>
- checkIfSelfReferenceIsPlacedCorrectly(child, cteId, allowRecursiveRef = false)
- case r: UnionLoopRef if !allowRecursiveRef && r.loopId == cteId =>
- throw new AnalysisException(
- errorClass = "INVALID_RECURSIVE_REFERENCE.PLACE",
- messageParameters = Map.empty)
- case other =>
- other.children.foreach(checkIfSelfReferenceIsPlacedCorrectly(_, cteId, allowRecursiveRef))
+ allowRecursiveRef: Boolean = true): Unit = {
+ plan match {
+ case Join(left, right, Inner, _, _) =>
+ checkIfSelfReferenceIsPlacedCorrectly(left, cteId, allowRecursiveRef)
+ checkIfSelfReferenceIsPlacedCorrectly(right, cteId, allowRecursiveRef)
+ case Join(left, right, Cross, _, _) =>
+ checkIfSelfReferenceIsPlacedCorrectly(left, cteId, allowRecursiveRef)
+ checkIfSelfReferenceIsPlacedCorrectly(right, cteId, allowRecursiveRef)
+ case Join(left, right, LeftOuter, _, _) =>
+ checkIfSelfReferenceIsPlacedCorrectly(left, cteId, allowRecursiveRef)
+ checkIfSelfReferenceIsPlacedCorrectly(right, cteId, allowRecursiveRef = false)
+ case Join(left, right, RightOuter, _, _) =>
+ checkIfSelfReferenceIsPlacedCorrectly(left, cteId, allowRecursiveRef = false)
+ checkIfSelfReferenceIsPlacedCorrectly(right, cteId, allowRecursiveRef)
+ case Join(left, right, LeftSemi, _, _) =>
+ checkIfSelfReferenceIsPlacedCorrectly(left, cteId, allowRecursiveRef)
+ checkIfSelfReferenceIsPlacedCorrectly(right, cteId, allowRecursiveRef = false)
+ case Join(left, right, LeftAnti, _, _) =>
+ checkIfSelfReferenceIsPlacedCorrectly(left, cteId, allowRecursiveRef)
+ checkIfSelfReferenceIsPlacedCorrectly(right, cteId, allowRecursiveRef = false)
+ case Join(left, right, _, _, _) =>
+ checkIfSelfReferenceIsPlacedCorrectly(left, cteId, allowRecursiveRef = false)
+ checkIfSelfReferenceIsPlacedCorrectly(right, cteId, allowRecursiveRef = false)
+ case Aggregate(_, _, child, _) =>
+ checkIfSelfReferenceIsPlacedCorrectly(child, cteId, allowRecursiveRef = false)
+ case Window(_, _, _, child, _) =>
+ checkIfSelfReferenceIsPlacedCorrectly(child, cteId, allowRecursiveRef = false)
+ case Sort(_, _, child, _) =>
+ checkIfSelfReferenceIsPlacedCorrectly(child, cteId, allowRecursiveRef = false)
+ case r: UnionLoopRef if !allowRecursiveRef && r.loopId == cteId =>
+ throw new AnalysisException(
+ errorClass = "INVALID_RECURSIVE_REFERENCE.PLACE",
+ messageParameters = Map.empty)
+ case other =>
+ other.children.foreach(checkIfSelfReferenceIsPlacedCorrectly(_, cteId, allowRecursiveRef))
+ }
+ plan.subqueries.foreach(checkIfSelfReferenceIsPlacedCorrectly(_, cteId, allowRecursiveRef))
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteMergeIntoTable.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteMergeIntoTable.scala
index 7e2cf4f29807c..9e67aa156fa21 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteMergeIntoTable.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteMergeIntoTable.scala
@@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLite
import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner, JoinType, LeftAnti, LeftOuter, RightOuter}
import org.apache.spark.sql.catalyst.plans.logical.{AppendData, DeleteAction, Filter, HintInfo, InsertAction, Join, JoinHint, LogicalPlan, MergeAction, MergeIntoTable, MergeRows, NO_BROADCAST_AND_REPLICATION, Project, ReplaceData, UpdateAction, WriteDelta}
-import org.apache.spark.sql.catalyst.plans.logical.MergeRows.{Discard, Instruction, Keep, ROW_ID, Split}
+import org.apache.spark.sql.catalyst.plans.logical.MergeRows.{Copy, Delete, Discard, Insert, Instruction, Keep, ROW_ID, Split, Update}
import org.apache.spark.sql.catalyst.util.RowDeltaUtils.{OPERATION_COLUMN, WRITE_OPERATION, WRITE_WITH_METADATA_OPERATION}
import org.apache.spark.sql.connector.catalog.SupportsRowLevelOperations
import org.apache.spark.sql.connector.write.{RowLevelOperationTable, SupportsDelta}
@@ -45,8 +45,8 @@ object RewriteMergeIntoTable extends RewriteRowLevelCommand with PredicateHelper
override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
case m @ MergeIntoTable(aliasedTable, source, cond, matchedActions, notMatchedActions,
- notMatchedBySourceActions, _) if m.resolved && m.rewritable && m.aligned &&
- matchedActions.isEmpty && notMatchedActions.size == 1 &&
+ notMatchedBySourceActions, _) if m.resolved && m.rewritable && m.aligned &&
+ !m.needSchemaEvolution && matchedActions.isEmpty && notMatchedActions.size == 1 &&
notMatchedBySourceActions.isEmpty =>
EliminateSubqueryAliases(aliasedTable) match {
@@ -79,7 +79,8 @@ object RewriteMergeIntoTable extends RewriteRowLevelCommand with PredicateHelper
}
case m @ MergeIntoTable(aliasedTable, source, cond, matchedActions, notMatchedActions,
- notMatchedBySourceActions, _) if m.resolved && m.rewritable && m.aligned &&
+ notMatchedBySourceActions, _)
+ if m.resolved && m.rewritable && m.aligned && !m.needSchemaEvolution &&
matchedActions.isEmpty && notMatchedBySourceActions.isEmpty =>
EliminateSubqueryAliases(aliasedTable) match {
@@ -93,7 +94,7 @@ object RewriteMergeIntoTable extends RewriteRowLevelCommand with PredicateHelper
val notMatchedInstructions = notMatchedActions.map {
case InsertAction(cond, assignments) =>
- Keep(cond.getOrElse(TrueLiteral), assignments.map(_.value))
+ Keep(Insert, cond.getOrElse(TrueLiteral), assignments.map(_.value))
case other =>
throw new AnalysisException(
errorClass = "_LEGACY_ERROR_TEMP_3053",
@@ -120,7 +121,8 @@ object RewriteMergeIntoTable extends RewriteRowLevelCommand with PredicateHelper
}
case m @ MergeIntoTable(aliasedTable, source, cond, matchedActions, notMatchedActions,
- notMatchedBySourceActions, _) if m.resolved && m.rewritable && m.aligned =>
+ notMatchedBySourceActions, _)
+ if m.resolved && m.rewritable && m.aligned && !m.needSchemaEvolution =>
EliminateSubqueryAliases(aliasedTable) match {
case r @ DataSourceV2Relation(tbl: SupportsRowLevelOperations, _, _, _, _) =>
@@ -199,7 +201,7 @@ object RewriteMergeIntoTable extends RewriteRowLevelCommand with PredicateHelper
// as the last MATCHED and NOT MATCHED BY SOURCE instruction
// this logic is specific to data sources that replace groups of data
val carryoverRowsOutput = Literal(WRITE_WITH_METADATA_OPERATION) +: targetTable.output
- val keepCarryoverRowsInstruction = Keep(TrueLiteral, carryoverRowsOutput)
+ val keepCarryoverRowsInstruction = Keep(Copy, TrueLiteral, carryoverRowsOutput)
val matchedInstructions = matchedActions.map { action =>
toInstruction(action, metadataAttrs)
@@ -436,7 +438,7 @@ object RewriteMergeIntoTable extends RewriteRowLevelCommand with PredicateHelper
val rowValues = assignments.map(_.value)
val metadataValues = nullifyMetadataOnUpdate(metadataAttrs)
val output = Seq(Literal(WRITE_WITH_METADATA_OPERATION)) ++ rowValues ++ metadataValues
- Keep(cond.getOrElse(TrueLiteral), output)
+ Keep(Update, cond.getOrElse(TrueLiteral), output)
case DeleteAction(cond) =>
Discard(cond.getOrElse(TrueLiteral))
@@ -445,7 +447,7 @@ object RewriteMergeIntoTable extends RewriteRowLevelCommand with PredicateHelper
val rowValues = assignments.map(_.value)
val metadataValues = metadataAttrs.map(attr => Literal(null, attr.dataType))
val output = Seq(Literal(WRITE_OPERATION)) ++ rowValues ++ metadataValues
- Keep(cond.getOrElse(TrueLiteral), output)
+ Keep(Insert, cond.getOrElse(TrueLiteral), output)
case other =>
throw new AnalysisException(
@@ -471,15 +473,15 @@ object RewriteMergeIntoTable extends RewriteRowLevelCommand with PredicateHelper
case UpdateAction(cond, assignments) =>
val output = deltaUpdateOutput(assignments, metadataAttrs, originalRowIdValues)
- Keep(cond.getOrElse(TrueLiteral), output)
+ Keep(Update, cond.getOrElse(TrueLiteral), output)
case DeleteAction(cond) =>
val output = deltaDeleteOutput(rowAttrs, rowIdAttrs, metadataAttrs, originalRowIdValues)
- Keep(cond.getOrElse(TrueLiteral), output)
+ Keep(Delete, cond.getOrElse(TrueLiteral), output)
case InsertAction(cond, assignments) =>
val output = deltaInsertOutput(assignments, metadataAttrs, originalRowIdValues)
- Keep(cond.getOrElse(TrueLiteral), output)
+ Keep(Insert, cond.getOrElse(TrueLiteral), output)
case other =>
throw new AnalysisException(
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala
index ef425be42f981..c4549a189e8e1 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala
@@ -21,7 +21,6 @@ import scala.util.control.NonFatal
import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys._
-import org.apache.spark.internal.MDC
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys
import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, LogicalPlan}
@@ -239,7 +238,7 @@ object StreamingJoinHelper extends PredicateHelper with Logging {
collect(left, negate) ++ collect(right, negate)
case Subtract(left, right, _) =>
collect(left, negate) ++ collect(right, !negate)
- case TimeAdd(left, right, _) =>
+ case TimestampAddInterval(left, right, _) =>
collect(left, negate) ++ collect(right, negate)
case DatetimeSub(_, _, child) => collect(child, negate)
case UnaryMinus(child, _) =>
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala
index adf74c489ce1f..5243e17afe5da 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala
@@ -132,7 +132,11 @@ object TableOutputResolver extends SQLConfHelper with Logging {
case (valueType, colType) if DataType.equalsIgnoreCompatibleNullability(valueType, colType) =>
val canWriteExpr = canWrite(
tableName, valueType, colType, byName = true, conf, addError, colPath)
- if (canWriteExpr) checkNullability(value, col, conf, colPath) else value
+ if (canWriteExpr) {
+ applyColumnMetadata(checkNullability(value, col, conf, colPath), col)
+ } else {
+ value
+ }
case (valueType: StructType, colType: StructType) =>
val resolvedValue = resolveStructType(
tableName, value, valueType, col, colType,
@@ -180,12 +184,67 @@ object TableOutputResolver extends SQLConfHelper with Logging {
} else {
CharVarcharUtils.stringLengthCheck(casted, attr.dataType)
}
- Alias(exprWithStrLenCheck, attr.name)(explicitMetadata = Some(attr.metadata))
+ applyColumnMetadata(exprWithStrLenCheck, attr)
} else {
value
}
}
+
+ /**
+ * Add an [[Alias]] with the name and metadata from the given target table attribute.
+ *
+ * The metadata may be used by writers to get certain table properties.
+ * For example [[org.apache.spark.sql.catalyst.json.JacksonGenerator]]
+ * looks for default value metadata to control some behavior.
+ * This is not the best design, but it is the way at this time.
+ * We should change all the writers to pick up table configuration
+ * from the table directly. However, there are many third-party
+ * connectors that may rely on this behavior.
+ *
+ * We also must remove any [[CharVarcharUtils.CHAR_VARCHAR_TYPE_STRING_METADATA_KEY]]
+ * metadata from flowing out the top of the query.
+ * If we don't do this, the write operation will remain unresolved, or worse
+ * it may flip from resolved to unresolved. We assume that the read-side
+ * handling is performed lower in the query.
+ *
+ * Moreover, we cannot propagate other source metadata, like source table
+ * default value definitions without confusing writers with reader metadata.
+ * So we need to be sure we block the source metadata from propagating.
+ *
+ * See SPARK-52772 for a discussion on rewrites that caused trouble with
+ * going from resolved to unresolved.
+ */
+ private def applyColumnMetadata(expr: Expression, column: Attribute): NamedExpression = {
+ // We have dealt with the required write-side char/varchar processing.
+ // We do not want to transfer that information to the read-side.
+ // If we do, the write operation will fail to resolve.
+ val requiredMetadata = CharVarcharUtils.cleanMetadata(column.metadata)
+
+ // Make sure that the result has the requiredMetadata and only that.
+ // If the expr is an Attribute or NamedLambdaVariable with the proper name and metadata,
+ // it should remain stable, but we do not trust that other NamedAttributes will
+ // remain stable (namely Alias).
+ expr match {
+ case a: Attribute if a.name == column.name && a.metadata == requiredMetadata =>
+ a
+ case v: NamedLambdaVariable if v.name == column.name && v.metadata == requiredMetadata =>
+ v
+ case _ =>
+ // We cannot keep an Alias with the correct name and metadata because the
+ // metadata might be derived, and derived metadata is not stable upon rewrites.
+ // eg:
+ // Alias(cast(attr, attr.dataType), n).metadata is empty =>
+ // Alias(attr, n).metadata == attr.metadata.
+ val stripAlias = expr match {
+ case a: Alias => a.child
+ case _ => expr
+ }
+ Alias(stripAlias, column.name)(explicitMetadata = Some(requiredMetadata))
+ }
+ }
+
+
private def canWrite(
tableName: String,
valueType: DataType,
@@ -227,20 +286,14 @@ object TableOutputResolver extends SQLConfHelper with Logging {
tableName, newColPath.quoted
)
}
- defaultExpr
+ Some(applyColumnMetadata(defaultExpr.get, expectedCol))
} else if (matched.length > 1) {
throw QueryCompilationErrors.incompatibleDataToTableAmbiguousColumnNameError(
tableName, newColPath.quoted
)
} else {
matchedCols += matched.head.name
- val expectedName = expectedCol.name
- val matchedCol = matched.head match {
- // Save an Alias if we can change the name directly.
- case a: Attribute => a.withName(expectedName)
- case a: Alias => a.withName(expectedName)
- case other => other
- }
+ val matchedCol = matched.head
val actualExpectedCol = expectedCol.withDataType {
CharVarcharUtils.getRawType(expectedCol.metadata).getOrElse(expectedCol.dataType)
}
@@ -386,7 +439,7 @@ object TableOutputResolver extends SQLConfHelper with Logging {
} else {
struct
}
- Some(Alias(res, expected.name)())
+ Some(applyColumnMetadata(res, expected))
} else {
None
}
@@ -412,16 +465,15 @@ object TableOutputResolver extends SQLConfHelper with Logging {
resolveColumnsByPosition(tableName, Seq(param), Seq(fakeAttr), conf, addError, colPath)
}
if (res.length == 1) {
- if (res.head == param) {
- // If the element type is the same, we can reuse the input array directly.
- Some(
- Alias(nullCheckedInput, expected.name)(
- nonInheritableMetadataKeys =
- Seq(CharVarcharUtils.CHAR_VARCHAR_TYPE_STRING_METADATA_KEY)))
- } else {
- val func = LambdaFunction(res.head, Seq(param))
- Some(Alias(ArrayTransform(nullCheckedInput, func), expected.name)())
- }
+ val castedArray =
+ if (res.head == param) {
+ // If the element type is the same, we can reuse the input array directly.
+ nullCheckedInput
+ } else {
+ val func = LambdaFunction(res.head, Seq(param))
+ ArrayTransform(nullCheckedInput, func)
+ }
+ Some(applyColumnMetadata(castedArray, expected))
} else {
None
}
@@ -461,26 +513,25 @@ object TableOutputResolver extends SQLConfHelper with Logging {
if (resKey.length == 1 && resValue.length == 1) {
// If the key and value expressions have not changed, we just check original map field.
// Otherwise, we construct a new map by adding transformations to the keys and values.
- if (resKey.head == keyParam && resValue.head == valueParam) {
- Some(
- Alias(nullCheckedInput, expected.name)(
- nonInheritableMetadataKeys =
- Seq(CharVarcharUtils.CHAR_VARCHAR_TYPE_STRING_METADATA_KEY)))
- } else {
- val newKeys = if (resKey.head != keyParam) {
- val keyFunc = LambdaFunction(resKey.head, Seq(keyParam))
- ArrayTransform(MapKeys(nullCheckedInput), keyFunc)
- } else {
- MapKeys(nullCheckedInput)
- }
- val newValues = if (resValue.head != valueParam) {
- val valueFunc = LambdaFunction(resValue.head, Seq(valueParam))
- ArrayTransform(MapValues(nullCheckedInput), valueFunc)
+ val casted =
+ if (resKey.head == keyParam && resValue.head == valueParam) {
+ nullCheckedInput
} else {
- MapValues(nullCheckedInput)
+ val newKeys = if (resKey.head != keyParam) {
+ val keyFunc = LambdaFunction(resKey.head, Seq(keyParam))
+ ArrayTransform(MapKeys(nullCheckedInput), keyFunc)
+ } else {
+ MapKeys(nullCheckedInput)
+ }
+ val newValues = if (resValue.head != valueParam) {
+ val valueFunc = LambdaFunction(resValue.head, Seq(valueParam))
+ ArrayTransform(MapValues(nullCheckedInput), valueFunc)
+ } else {
+ MapValues(nullCheckedInput)
+ }
+ MapFromArrays(newKeys, newValues)
}
- Some(Alias(MapFromArrays(newKeys, newValues), expected.name)())
- }
+ Some(applyColumnMetadata(casted, expected))
} else {
None
}
@@ -525,12 +576,6 @@ object TableOutputResolver extends SQLConfHelper with Logging {
!Cast.canUpCast(cast.child.dataType, cast.dataType)
}
- private def isCompatible(tableAttr: Attribute, queryExpr: NamedExpression): Boolean = {
- DataTypeUtils.sameType(tableAttr.dataType, queryExpr.dataType) &&
- tableAttr.name == queryExpr.name &&
- tableAttr.metadata == queryExpr.metadata
- }
-
private def checkField(
tableName: String,
tableAttr: Attribute,
@@ -546,33 +591,32 @@ object TableOutputResolver extends SQLConfHelper with Logging {
} else {
tableAttr.dataType
}
- lazy val outputField = if (isCompatible(tableAttr, queryExpr)) {
- if (requiresNullChecks(queryExpr, tableAttr, conf)) {
- val assert = AssertNotNull(queryExpr, colPath)
- Some(Alias(assert, tableAttr.name)(explicitMetadata = Some(tableAttr.metadata)))
- } else {
- Some(queryExpr)
- }
- } else {
- val nullCheckedQueryExpr = checkNullability(queryExpr, tableAttr, conf, colPath)
- val udtUnwrapped = unwrapUDT(nullCheckedQueryExpr)
- val casted = cast(udtUnwrapped, attrTypeWithoutCharVarchar, conf, colPath.quoted)
- val exprWithStrLenCheck = if (conf.charVarcharAsString || !attrTypeHasCharVarchar) {
- casted
- } else {
- CharVarcharUtils.stringLengthCheck(casted, tableAttr.dataType)
- }
- // Renaming is needed for handling the following cases like
- // 1) Column names/types do not match, e.g., INSERT INTO TABLE tab1 SELECT 1, 2
- // 2) Target tables have column metadata
- Some(Alias(exprWithStrLenCheck, tableAttr.name)(explicitMetadata = Some(tableAttr.metadata)))
- }
val canWriteExpr = canWrite(
tableName, queryExpr.dataType, attrTypeWithoutCharVarchar,
byName, conf, addError, colPath)
- if (canWriteExpr) outputField else None
+ if (canWriteExpr) {
+ val prepared =
+ if (DataTypeUtils.sameType(tableAttr.dataType, queryExpr.dataType)) {
+ // If the types are an exact match, we can leave UDTs alone,
+ // we obviously do not need a cast, and the constraints of the target
+ // table char/varchar types must be met.
+ queryExpr
+ } else {
+ val udtUnwrapped = unwrapUDT(queryExpr)
+ val casted = cast(udtUnwrapped, attrTypeWithoutCharVarchar, conf, colPath.quoted)
+ if (conf.charVarcharAsString || !attrTypeHasCharVarchar) {
+ casted
+ } else {
+ CharVarcharUtils.stringLengthCheck(casted, tableAttr.dataType)
+ }
+ }
+ val nullChecked = checkNullability(prepared, tableAttr, conf, colPath)
+ Some(applyColumnMetadata(nullChecked, tableAttr))
+ } else {
+ None
+ }
}
private def unwrapUDT(expr: Expression): Expression = expr.dataType match {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionBase.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionBase.scala
index a8832aada0839..271e151e709cc 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionBase.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionBase.scala
@@ -42,6 +42,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{
Project,
ReplaceTable,
Union,
+ UnionLoop,
Unpivot
}
import org.apache.spark.sql.catalyst.rules.Rule
@@ -49,6 +50,7 @@ import org.apache.spark.sql.catalyst.trees.CurrentOrigin.withOrigin
import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper
import org.apache.spark.sql.connector.catalog.procedures.BoundProcedure
+import org.apache.spark.sql.errors.DataTypeErrors.cannotMergeIncompatibleDataTypesError
import org.apache.spark.sql.types.DataType
abstract class TypeCoercionBase extends TypeCoercionHelper {
@@ -247,6 +249,25 @@ abstract class TypeCoercionBase extends TypeCoercionHelper {
val attrMapping = s.children.head.output.zip(newChildren.head.output)
s.copy(children = newChildren) -> attrMapping
}
+
+ case s: UnionLoop
+ if s.childrenResolved && s.anchor.output.length == s.recursion.output.length
+ && !s.resolved =>
+ // If the anchor data type is wider than the recursion data type, we cast the recursion
+ // type to match the anchor type.
+ // On the other hand, we cannot cast the anchor type into a wider recursion type, as at
+ // this point the UnionLoopRefs inside the recursion are already resolved with the
+ // narrower anchor type.
+ val projectList = s.recursion.output.zip(s.anchor.output.map(_.dataType)).map {
+ case (attr, dt) =>
+ val widerType = findWiderTypeForTwo(attr.dataType, dt)
+ if (widerType.isDefined && widerType.get == dt) {
+ Alias(Cast(attr, dt), attr.name)()
+ } else {
+ throw cannotMergeIncompatibleDataTypesError(dt, attr.dataType)
+ }
+ }
+ s.copy(recursion = Project(projectList, s.recursion)) -> Nil
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionHelper.scala
index 390ff2f3114d9..0e7d44e98bfb8 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionHelper.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionHelper.scala
@@ -51,7 +51,7 @@ import org.apache.spark.sql.catalyst.expressions.{
SpecialFrameBoundary,
SpecifiedWindowFrame,
SubtractTimestamps,
- TimeAdd,
+ TimestampAddInterval,
WindowSpecDefinition
}
import org.apache.spark.sql.catalyst.expressions.aggregate.{Average, Sum}
@@ -671,7 +671,7 @@ abstract class TypeCoercionHelper {
case (e, _: DateType) => e
case (e, _: TimestampType) => e
case (e: Expression, t) if e.dataType != t && canCast(e.dataType, t) =>
- Cast(e, t)
+ Cast(child = e, dataType = t).withTimeZone(conf.sessionLocalTimeZone)
case _ => boundary
}
}
@@ -700,7 +700,8 @@ abstract class TypeCoercionHelper {
val newRight = castIfNotSameType(s.right, TimestampNTZType)
s.copy(left = newLeft, right = newRight)
- case t @ TimeAdd(StringTypeExpression(), _, _) => t.copy(start = Cast(t.start, TimestampType))
+ case t @ TimestampAddInterval(StringTypeExpression(), _, _) =>
+ t.copy(start = Cast(t.start, TimestampType))
case other => other
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala
index 55b5b8ddfe6a3..fd4e081c91b52 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.analysis
import java.util.Locale
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{ANALYSIS_ERROR, QUERY_PLAN}
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.ExtendedAnalysisException
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala
index 4e35bd313d370..f88e2d112b558 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ValidateSubqueryExpression.scala
@@ -17,7 +17,7 @@
package org.apache.spark.sql.catalyst.analysis
-import org.apache.spark.internal.{Logging, LogKeys, MDC}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.SubExprUtils._
import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/VariableResolution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/VariableResolution.scala
new file mode 100644
index 0000000000000..72af7c619a083
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/VariableResolution.scala
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.analysis
+
+import java.util.Locale
+
+import org.apache.spark.sql.catalyst.{SQLConfHelper, SqlScriptingContextManager}
+import org.apache.spark.sql.catalyst.catalog.TempVariableManager
+import org.apache.spark.sql.catalyst.expressions.{
+ Expression,
+ ExtractValue,
+ Literal,
+ VariableReference
+}
+import org.apache.spark.sql.catalyst.parser.SqlScriptingLabelContext.isForbiddenLabelOrForVariableName
+import org.apache.spark.sql.connector.catalog.{
+ CatalogManager,
+ Identifier
+}
+
+class VariableResolution(tempVariableManager: TempVariableManager) extends SQLConfHelper {
+
+ /**
+ * Resolves a `multipartName` to an [[Expression]] tree, supporting nested field access.
+ *
+ * This method implements a longest-match strategy similar to column resolution,
+ * preferring fully qualified variable names to avoid naming conflicts. It supports
+ * accessing nested fields within variables through dot notation.
+ *
+ * The resolution process works as follows:
+ * 1. Attempts to resolve the full name as a variable
+ * 2. If unsuccessful, treats the rightmost parts as nested field access
+ * 3. Continues until a variable is found or all combinations are exhausted
+ * 4. Wraps the result in ExtractValue expressions for nested field access
+ *
+ * @param nameParts The sequence of name parts representing the variable identifier
+ * (e.g., ["catalog", "schema", "variable", "field1", "field2"])
+ * @param resolvingView Whether this resolution is happening within a view context.
+ * When true, only variables explicitly referred to in the view definition are accessible.
+ * @param resolvingExecuteImmediate Whether this resolution is happening within an
+ * EXECUTE IMMEDIATE context. When true, local variables are not accessible, only session
+ * variables.
+ * @param referredTempVariableNames When resolving within a view, this contains the list of
+ * variable names that the view explicitly references and should have access to.
+ *
+ * @return Some(Expression) if a variable is successfully resolved, potentially wrapped in
+ * [[ExtractValue]] expressions for nested field access. None if no variable can be resolved
+ * from the given name parts.
+ */
+ def resolveMultipartName(
+ nameParts: Seq[String],
+ resolvingView: Boolean,
+ resolvingExecuteImmediate: Boolean,
+ referredTempVariableNames: Seq[Seq[String]]): Option[Expression] = {
+ var resolvedVariable: Option[Expression] = None
+ // We only support temp variables for now, so the variable name can at most have 3 parts.
+ var numInnerFields: Int = math.max(0, nameParts.length - 3)
+ // Follow the column resolution and prefer the longest match. This makes sure that users
+ // can always use fully qualified variable name to avoid name conflicts.
+ while (resolvedVariable.isEmpty && numInnerFields < nameParts.length) {
+ resolvedVariable = resolveVariable(
+ nameParts = nameParts.dropRight(numInnerFields),
+ resolvingView = resolvingView,
+ resolvingExecuteImmediate = resolvingExecuteImmediate,
+ referredTempVariableNames = referredTempVariableNames
+ )
+
+ if (resolvedVariable.isEmpty) {
+ numInnerFields += 1
+ }
+ }
+
+ resolvedVariable.map { variable =>
+ if (numInnerFields != 0) {
+ val nestedFields = nameParts.takeRight(numInnerFields)
+ nestedFields.foldLeft(variable: Expression) { (e, name) =>
+ ExtractValue(e, Literal(name), conf.resolver)
+ }
+ } else {
+ variable
+ }
+ }
+ }
+
+ /**
+ * Look up variable by nameParts.
+ * If in SQL Script, first check local variables, unless in EXECUTE IMMEDIATE
+ * (EXECUTE IMMEDIATE generated query cannot access local variables).
+ * if not found fall back to session variables.
+ * @param nameParts NameParts of the variable.
+ * @param resolvingExecuteImmediate Whether the current context is in EXECUTE IMMEDIATE.
+ * @return Reference to the variable.
+ */
+ def lookupVariable(
+ nameParts: Seq[String],
+ resolvingExecuteImmediate: Boolean): Option[VariableReference] = {
+ val namePartsCaseAdjusted = if (conf.caseSensitiveAnalysis) {
+ nameParts
+ } else {
+ nameParts.map(_.toLowerCase(Locale.ROOT))
+ }
+
+ SqlScriptingContextManager
+ .get()
+ .map(_.getVariableManager)
+ // If we are in EXECUTE IMMEDIATE lookup only session variables.
+ .filterNot(_ => resolvingExecuteImmediate)
+ // If variable name is qualified with session. treat it as a session variable.
+ .filterNot(
+ _ =>
+ nameParts.length > 2
+ || (nameParts.length == 2 && isForbiddenLabelOrForVariableName(nameParts.head))
+ )
+ .flatMap(_.get(namePartsCaseAdjusted))
+ .map { varDef =>
+ VariableReference(
+ nameParts,
+ FakeLocalCatalog,
+ Identifier.of(Array(varDef.identifier.namespace().last), namePartsCaseAdjusted.last),
+ varDef
+ )
+ }
+ .orElse(
+ if (maybeTempVariableName(nameParts)) {
+ tempVariableManager
+ .get(namePartsCaseAdjusted)
+ .map { varDef =>
+ VariableReference(
+ nameParts,
+ FakeSystemCatalog,
+ Identifier.of(Array(CatalogManager.SESSION_NAMESPACE), namePartsCaseAdjusted.last),
+ varDef
+ )
+ }
+ } else {
+ None
+ }
+ )
+ }
+
+ private def resolveVariable(
+ nameParts: Seq[String],
+ resolvingView: Boolean,
+ resolvingExecuteImmediate: Boolean,
+ referredTempVariableNames: Seq[Seq[String]]): Option[Expression] = {
+ if (resolvingView) {
+ if (referredTempVariableNames.contains(nameParts)) {
+ lookupVariable(nameParts = nameParts, resolvingExecuteImmediate = resolvingExecuteImmediate)
+ } else {
+ None
+ }
+ } else {
+ lookupVariable(nameParts = nameParts, resolvingExecuteImmediate = resolvingExecuteImmediate)
+ }
+ }
+
+ // The temp variables live in `SYSTEM.SESSION`, and the name can be qualified or not.
+ private def maybeTempVariableName(nameParts: Seq[String]): Boolean = {
+ nameParts.length == 1 || {
+ if (nameParts.length == 2) {
+ nameParts.head.equalsIgnoreCase(CatalogManager.SESSION_NAMESPACE)
+ } else if (nameParts.length == 3) {
+ nameParts(0).equalsIgnoreCase(CatalogManager.SYSTEM_CATALOG_NAME) &&
+ nameParts(1).equalsIgnoreCase(CatalogManager.SESSION_NAMESPACE)
+ } else {
+ false
+ }
+ }
+ }
+
+}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewResolution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewResolution.scala
index 371be97bebc0e..faa3b9081cbfd 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewResolution.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewResolution.scala
@@ -43,7 +43,13 @@ object ViewResolution {
view
)
}
- SQLConf.withExistingConf(View.effectiveSQLConf(view.desc.viewSQLConfigs, view.isTempView)) {
+ SQLConf.withExistingConf(
+ View.effectiveSQLConf(
+ configs = view.desc.viewSQLConfigs,
+ isTempView = view.isTempView,
+ createSparkVersion = view.desc.createVersion
+ )
+ ) {
resolveChild(view.child)
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/WindowResolution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/WindowResolution.scala
new file mode 100644
index 0000000000000..a477fe4398f65
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/WindowResolution.scala
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import org.apache.spark.sql.catalyst.expressions.{
+ AggregateWindowFunction,
+ CurrentRow,
+ FrameLessOffsetWindowFunction,
+ RangeFrame,
+ RankLike,
+ RowFrame,
+ SpecifiedWindowFrame,
+ UnboundedFollowing,
+ UnboundedPreceding,
+ UnspecifiedFrame,
+ WindowExpression,
+ WindowFunction,
+ WindowSpecDefinition
+}
+import org.apache.spark.sql.catalyst.expressions.aggregate.{
+ AggregateExpression,
+ ListAgg,
+ Median,
+ PercentileCont,
+ PercentileDisc
+}
+import org.apache.spark.sql.catalyst.util.TypeUtils.toSQLExpr
+import org.apache.spark.sql.errors.QueryCompilationErrors
+
+/**
+ * Utility object for resolving [[WindowExpression]].
+ *
+ * It ensures that window frame defintions and order specs are consistent between the
+ * [[WindowFunction]] and [[WindowSpecDefinition]], throwing errors if configurations are
+ * incompatible or missing.
+ */
+object WindowResolution {
+
+ /**
+ * Validates the window frame of a [[WindowExpression]].
+ *
+ * It enforces that the frame in [[WindowExpression.windowFunction]] matches the frame
+ * in [[WindowExpression.windowSpec]], alterantively it provides a default frame when it
+ * is unspecified.
+ */
+ def resolveFrame(windowExpression: WindowExpression): WindowExpression = windowExpression match {
+ case WindowExpression(
+ wf: FrameLessOffsetWindowFunction,
+ WindowSpecDefinition(_, _, f: SpecifiedWindowFrame)
+ ) if wf.frame != f =>
+ throw QueryCompilationErrors.cannotSpecifyWindowFrameError(wf.prettyName)
+
+ case WindowExpression(wf: WindowFunction, WindowSpecDefinition(_, _, f: SpecifiedWindowFrame))
+ if wf.frame != UnspecifiedFrame && wf.frame != f =>
+ throw QueryCompilationErrors.windowFrameNotMatchRequiredFrameError(f, wf.frame)
+
+ case WindowExpression(wf: WindowFunction, s @ WindowSpecDefinition(_, _, UnspecifiedFrame))
+ if wf.frame != UnspecifiedFrame =>
+ WindowExpression(wf, s.copy(frameSpecification = wf.frame))
+
+ case we @ WindowExpression(e, s @ WindowSpecDefinition(_, o, UnspecifiedFrame)) if e.resolved =>
+ val frame = if (o.nonEmpty) {
+ SpecifiedWindowFrame(RangeFrame, UnboundedPreceding, CurrentRow)
+ } else {
+ SpecifiedWindowFrame(RowFrame, UnboundedPreceding, UnboundedFollowing)
+ }
+ we.copy(windowSpec = s.copy(frameSpecification = frame))
+
+ case e => e
+ }
+
+ /**
+ * Ensures that [[WindowExpression.windowSpec.orderSpec]] is not missing.
+ *
+ * In case of [[RankLike]] window functions, it attaches the resolved order to the
+ * function to finalize it.
+ */
+ def resolveOrder(windowExpression: WindowExpression): WindowExpression = windowExpression match {
+ case WindowExpression(wf: WindowFunction, spec) if spec.orderSpec.isEmpty =>
+ throw QueryCompilationErrors.windowFunctionWithWindowFrameNotOrderedError(wf)
+
+ case WindowExpression(rank: RankLike, spec) if spec.resolved =>
+ val order = spec.orderSpec.map(_.child)
+ WindowExpression(rank.withOrder(order), spec)
+
+ case e => e
+ }
+
+ /**
+ * Validates a resolved [[WindowExpression]] to ensure it conforms to the allowed constraints.
+ *
+ * By checking the type and configuration of [[WindowExpression.windowFunction]] it enforces the
+ * following rules:
+ * - Disallows [[FrameLessOffsetWindowFunction]] (e.g. [[Lag]]) without defined ordering or
+ * one with a frame which is defined as something other than an offset frame (e.g.
+ * `ROWS BETWEEN` is logically incompatible with offset functions).
+ * - Disallows distinct aggregate expressions in window functions.
+ * - Disallows use of certain aggregate functions - [[ListAgg]], [[PercentileCont]],
+ * [[PercentileDisc]], [[Median]]
+ * - Allows only window functions of following types:
+ * - [[AggregateExpression]] (non-distinct)
+ * - [[FrameLessOffsetWindowFunction]]
+ * - [[AggregateWindowFunction]]
+ */
+ def validateResolvedWindowExpression(windowExpression: WindowExpression): Unit = {
+ checkWindowFunctionAndFrameMismatch(windowExpression)
+ checkWindowFunction(windowExpression)
+ }
+
+ def checkWindowFunctionAndFrameMismatch(windowExpression: WindowExpression): Unit = {
+ windowExpression match {
+ case _ @ WindowExpression(
+ windowFunction: FrameLessOffsetWindowFunction,
+ WindowSpecDefinition(_, order, frame: SpecifiedWindowFrame)
+ ) if order.isEmpty || !frame.isOffset =>
+ windowExpression.failAnalysis(
+ errorClass = "WINDOW_FUNCTION_AND_FRAME_MISMATCH",
+ messageParameters = Map(
+ "funcName" -> toSQLExpr(windowFunction),
+ "windowExpr" -> toSQLExpr(windowExpression)
+ )
+ )
+ case _ =>
+ }
+ }
+
+ def checkWindowFunction(windowExpression: WindowExpression): Unit = {
+ windowExpression.windowFunction match {
+ case AggregateExpression(_, _, true, _, _) =>
+ windowExpression.failAnalysis(
+ errorClass = "DISTINCT_WINDOW_FUNCTION_UNSUPPORTED",
+ messageParameters = Map("windowExpr" -> toSQLExpr(windowExpression))
+ )
+ case agg @ AggregateExpression(fun: ListAgg, _, _, _, _)
+ // listagg(...) WITHIN GROUP (ORDER BY ...) OVER (ORDER BY ...) is unsupported
+ if fun.orderingFilled && (windowExpression.windowSpec.orderSpec.nonEmpty ||
+ windowExpression.windowSpec.frameSpecification !=
+ SpecifiedWindowFrame(RowFrame, UnboundedPreceding, UnboundedFollowing)) =>
+ agg.failAnalysis(
+ errorClass = "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC",
+ messageParameters = Map("aggFunc" -> toSQLExpr(agg.aggregateFunction))
+ )
+ case agg @ AggregateExpression(_: PercentileCont | _: PercentileDisc | _: Median, _, _, _, _)
+ if windowExpression.windowSpec.orderSpec.nonEmpty ||
+ windowExpression.windowSpec.frameSpecification !=
+ SpecifiedWindowFrame(RowFrame, UnboundedPreceding, UnboundedFollowing) =>
+ agg.failAnalysis(
+ errorClass = "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC",
+ messageParameters = Map("aggFunc" -> toSQLExpr(agg.aggregateFunction))
+ )
+ case _: AggregateExpression | _: FrameLessOffsetWindowFunction | _: AggregateWindowFunction =>
+ case other =>
+ other.failAnalysis(
+ errorClass = "UNSUPPORTED_EXPR_FOR_WINDOW",
+ messageParameters = Map("sqlExpr" -> toSQLExpr(other))
+ )
+ }
+ }
+}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala
index 2b7eed0dc7950..b926cdf57f169 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala
@@ -51,7 +51,8 @@ class SubstituteExecuteImmediate(
val catalogManager: CatalogManager,
resolveChild: LogicalPlan => LogicalPlan,
checkAnalysis: LogicalPlan => Unit)
- extends Rule[LogicalPlan] with ColumnResolutionHelper {
+ extends Rule[LogicalPlan] {
+ private val variableResolution = new VariableResolution(catalogManager.tempVariableManager)
def resolveVariable(e: Expression): Expression = {
@@ -201,7 +202,10 @@ class SubstituteExecuteImmediate(
}
private def getVariableReference(expr: Expression, nameParts: Seq[String]): VariableReference = {
- lookupVariable(nameParts) match {
+ variableResolution.lookupVariable(
+ nameParts = nameParts,
+ resolvingExecuteImmediate = AnalysisContext.get.isExecuteImmediate
+ ) match {
case Some(variable) => variable
case _ =>
throw QueryCompilationErrors
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AggregateExpressionResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AggregateExpressionResolver.scala
index 4a01bf14fe4bc..b194a4e44a9d1 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AggregateExpressionResolver.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AggregateExpressionResolver.scala
@@ -18,11 +18,6 @@
package org.apache.spark.sql.catalyst.analysis.resolver
import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.catalyst.analysis.{
- AnsiTypeCoercion,
- CollationTypeCoercion,
- TypeCoercion
-}
import org.apache.spark.sql.catalyst.expressions.{Expression, OuterReference, SubExprUtils}
import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, ListAgg}
import org.apache.spark.sql.catalyst.util.toPrettySQL
@@ -41,11 +36,6 @@ class AggregateExpressionResolver(
private val traversals = expressionResolver.getExpressionTreeTraversals
- protected override val ansiTransformations: CoercesExpressionTypes.Transformations =
- AggregateExpressionResolver.ANSI_TYPE_COERCION_TRANSFORMATIONS
- protected override val nonAnsiTransformations: CoercesExpressionTypes.Transformations =
- AggregateExpressionResolver.TYPE_COERCION_TRANSFORMATIONS
-
private val expressionResolutionContextStack =
expressionResolver.getExpressionResolutionContextStack
private val subqueryRegistry = operatorResolver.getSubqueryRegistry
@@ -58,6 +48,7 @@ class AggregateExpressionResolver(
* resolving its children recursively and validating the resolved expression.
*/
override def resolve(aggregateExpression: AggregateExpression): Expression = {
+ expressionResolutionContextStack.peek().resolvingTreeUnderAggregateExpression = true
val aggregateExpressionWithChildrenResolved =
withResolvedChildren(aggregateExpression, expressionResolver.resolve _)
.asInstanceOf[AggregateExpression]
@@ -132,15 +123,13 @@ class AggregateExpressionResolver(
throwNestedAggregateFunction(aggregateExpression)
}
- val nonDeterministicChild =
- aggregateExpression.aggregateFunction.children.collectFirst {
- case child if !child.deterministic => child
+ aggregateExpression.aggregateFunction.children.foreach { child =>
+ if (!child.deterministic) {
+ throwAggregateFunctionWithNondeterministicExpression(
+ aggregateExpression,
+ child
+ )
}
- if (nonDeterministicChild.nonEmpty) {
- throwAggregateFunctionWithNondeterministicExpression(
- aggregateExpression,
- nonDeterministicChild.get
- )
}
}
@@ -249,23 +238,3 @@ class AggregateExpressionResolver(
)
}
}
-
-object AggregateExpressionResolver {
- // Ordering in the list of type coercions should be in sync with the list in [[TypeCoercion]].
- private val TYPE_COERCION_TRANSFORMATIONS: Seq[Expression => Expression] = Seq(
- CollationTypeCoercion.apply,
- TypeCoercion.InTypeCoercion.apply,
- TypeCoercion.FunctionArgumentTypeCoercion.apply,
- TypeCoercion.IfTypeCoercion.apply,
- TypeCoercion.ImplicitTypeCoercion.apply
- )
-
- // Ordering in the list of type coercions should be in sync with the list in [[AnsiTypeCoercion]].
- private val ANSI_TYPE_COERCION_TRANSFORMATIONS: Seq[Expression => Expression] = Seq(
- CollationTypeCoercion.apply,
- AnsiTypeCoercion.InTypeCoercion.apply,
- AnsiTypeCoercion.FunctionArgumentTypeCoercion.apply,
- AnsiTypeCoercion.IfTypeCoercion.apply,
- AnsiTypeCoercion.ImplicitTypeCoercion.apply
- )
-}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AggregateResolutionResult.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AggregateResolutionResult.scala
index d96185f642fd8..d4bb96e8d72f0 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AggregateResolutionResult.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AggregateResolutionResult.scala
@@ -20,14 +20,15 @@ package org.apache.spark.sql.catalyst.analysis.resolver
import java.util.HashSet
import org.apache.spark.sql.catalyst.expressions.{Alias, ExprId, NamedExpression}
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan}
/**
- * Stores the resulting operator, output list, grouping attributes and list of aliases from
- * aggregate list, obtained by resolving an [[Aggregate]] operator.
+ * Stores the resulting operator, output list, grouping attributes, list of aliases from
+ * aggregate list and base [[Aggregate]], obtained by resolving an [[Aggregate]] operator.
*/
case class AggregateResolutionResult(
operator: LogicalPlan,
outputList: Seq[NamedExpression],
- groupingAttributeIds: Option[HashSet[ExprId]],
- aggregateListAliases: Seq[Alias])
+ groupingAttributeIds: HashSet[ExprId],
+ aggregateListAliases: Seq[Alias],
+ baseAggregate: Aggregate)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AggregateResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AggregateResolver.scala
index fffd55b5897bd..7591452b76d21 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AggregateResolver.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AggregateResolver.scala
@@ -17,24 +17,18 @@
package org.apache.spark.sql.catalyst.analysis.resolver
-import java.util.{HashSet, LinkedHashMap}
+import java.util.HashSet
-import scala.jdk.CollectionConverters._
-
-import org.apache.spark.sql.catalyst.analysis.{
- AnalysisErrorAt,
- NondeterministicExpressionCollection,
- UnresolvedAttribute
-}
+import org.apache.spark.sql.catalyst.analysis.{AnalysisErrorAt, UnresolvedAttribute}
import org.apache.spark.sql.catalyst.expressions.{
Alias,
+ AliasHelper,
AttributeReference,
Expression,
ExprId,
- ExprUtils,
- NamedExpression
+ ExprUtils
}
-import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan}
/**
* Resolves an [[Aggregate]] by resolving its child, aggregate expressions and grouping
@@ -42,7 +36,8 @@ import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan, Proj
* related to [[Aggregate]] resolution.
*/
class AggregateResolver(operatorResolver: Resolver, expressionResolver: ExpressionResolver)
- extends TreeNodeResolver[Aggregate, LogicalPlan] {
+ extends TreeNodeResolver[Aggregate, LogicalPlan]
+ with AliasHelper {
private val scopes = operatorResolver.getNameScopes
private val lcaResolver = expressionResolver.getLcaResolver
@@ -50,17 +45,27 @@ class AggregateResolver(operatorResolver: Resolver, expressionResolver: Expressi
* Resolve [[Aggregate]] operator.
*
* 1. Resolve the child (inline table).
- * 2. Resolve aggregate expressions using [[ExpressionResolver.resolveAggregateExpressions]] and
+ * 2. Clear [[NameScope.availableAliases]]. Those are only relevant for the immediate aggregate
+ * expressions for output prioritization to work correctly in
+ * [[NameScope.tryResolveMultipartNameByOutput]].
+ * 3. Resolve aggregate expressions using [[ExpressionResolver.resolveAggregateExpressions]] and
* set [[NameScope.ordinalReplacementExpressions]] for grouping expressions resolution.
- * 3. If there's just one [[UnresolvedAttribute]] with a single-part name "ALL", expand it using
+ * 4. If there's just one [[UnresolvedAttribute]] with a single-part name "ALL", expand it using
* aggregate expressions which don't contain aggregate functions. There should not exist a
* column with that name in the lower operator's output, otherwise it takes precedence.
- * 4. Resolve grouping expressions using [[ExpressionResolver.resolveGroupingExpressions]]. This
+ * 5. Resolve grouping expressions using [[ExpressionResolver.resolveGroupingExpressions]]. This
* includes alias references to aggregate expressions, which is done in
* [[NameScope.resolveMultipartName]] and replacing [[UnresolvedOrdinals]] with corresponding
* expressions from aggregate list, done in [[OrdinalResolver]].
- * 5. Substitute non-deterministic expressions with derived attribute references to an
- * artificial [[Project]] list.
+ * 6. Remove all the unnecessary [[Alias]]es from the grouping (all the aliases) and aggregate
+ * (keep the outermost one) expressions. This is needed to stay compatible with the
+ * fixed-point implementation. For example:
+ *
+ * {{{ SELECT timestamp(col1:str) FROM VALUES('a') GROUP BY timestamp(col1:str); }}}
+ *
+ * Here we end up having inner [[Alias]]es in both the grouping and aggregate expressions
+ * lists which are uncomparable because they have different expression IDs (thus we have to
+ * strip them).
*
* If the resulting [[Aggregate]] contains lateral columns references, delegate the resolution of
* these columns to [[LateralColumnAliasResolver.handleLcaInAggregate]]. Otherwise, validate the
@@ -73,6 +78,8 @@ class AggregateResolver(operatorResolver: Resolver, expressionResolver: Expressi
val resolvedAggregate = try {
val resolvedChild = operatorResolver.resolve(unresolvedAggregate.child)
+ scopes.current.availableAliases.clear()
+
val resolvedAggregateExpressions = expressionResolver.resolveAggregateExpressions(
unresolvedAggregate.aggregateExpressions,
unresolvedAggregate
@@ -100,21 +107,25 @@ class AggregateResolver(operatorResolver: Resolver, expressionResolver: Expressi
)
}
- val partiallyResolvedAggregate = unresolvedAggregate.copy(
- groupingExpressions = resolvedGroupingExpressions,
- aggregateExpressions = resolvedAggregateExpressions.expressions,
+ val resolvedGroupingExpressionsWithoutAliases = resolvedGroupingExpressions.map(trimAliases)
+ val resolvedAggregateExpressionsWithoutAliases =
+ resolvedAggregateExpressions.expressions.map(trimNonTopLevelAliases)
+
+ val resolvedAggregate = unresolvedAggregate.copy(
+ groupingExpressions = resolvedGroupingExpressionsWithoutAliases,
+ aggregateExpressions = resolvedAggregateExpressionsWithoutAliases,
child = resolvedChild
)
- val resolvedAggregate = tryPullOutNondeterministic(partiallyResolvedAggregate)
-
if (resolvedAggregateExpressions.hasLateralColumnAlias) {
val aggregateWithLcaResolutionResult = lcaResolver.handleLcaInAggregate(resolvedAggregate)
AggregateResolutionResult(
operator = aggregateWithLcaResolutionResult.resolvedOperator,
outputList = aggregateWithLcaResolutionResult.outputList,
- groupingAttributeIds = None,
- aggregateListAliases = aggregateWithLcaResolutionResult.aggregateListAliases
+ groupingAttributeIds =
+ getGroupingAttributeIds(aggregateWithLcaResolutionResult.baseAggregate),
+ aggregateListAliases = aggregateWithLcaResolutionResult.aggregateListAliases,
+ baseAggregate = aggregateWithLcaResolutionResult.baseAggregate
)
} else {
// TODO: This validation function does a post-traversal. This is discouraged in single-pass
@@ -124,8 +135,9 @@ class AggregateResolver(operatorResolver: Resolver, expressionResolver: Expressi
AggregateResolutionResult(
operator = resolvedAggregate,
outputList = resolvedAggregate.aggregateExpressions,
- groupingAttributeIds = Some(getGroupingAttributeIds(resolvedAggregate)),
- aggregateListAliases = scopes.current.getTopAggregateExpressionAliases
+ groupingAttributeIds = getGroupingAttributeIds(resolvedAggregate),
+ aggregateListAliases = scopes.current.getTopAggregateExpressionAliases,
+ baseAggregate = resolvedAggregate
)
}
} finally {
@@ -134,8 +146,9 @@ class AggregateResolver(operatorResolver: Resolver, expressionResolver: Expressi
scopes.overwriteOutputAndExtendHiddenOutput(
output = resolvedAggregate.outputList.map(_.toAttribute),
- groupingAttributeIds = resolvedAggregate.groupingAttributeIds,
- aggregateListAliases = resolvedAggregate.aggregateListAliases
+ groupingAttributeIds = Some(resolvedAggregate.groupingAttributeIds),
+ aggregateListAliases = resolvedAggregate.aggregateListAliases,
+ baseAggregate = Some(resolvedAggregate.baseAggregate)
)
resolvedAggregate.operator
@@ -208,53 +221,6 @@ class AggregateResolver(operatorResolver: Resolver, expressionResolver: Expressi
}
}
- /**
- * In case there are non-deterministic expressions in either `groupingExpressions` or
- * `aggregateExpressions` replace them with attributes created out of corresponding
- * non-deterministic expression. Example:
- *
- * {{{ SELECT RAND() GROUP BY 1; }}}
- *
- * This query would have the following analyzed plan:
- * Aggregate(
- * groupingExpressions = [AttributeReference(_nonDeterministic)]
- * aggregateExpressions = [Alias(AttributeReference(_nonDeterministic), `rand()`)]
- * child = Project(
- * projectList = [Alias(Rand(...), `_nondeterministic`)]
- * child = OneRowRelation
- * )
- * )
- */
- private def tryPullOutNondeterministic(aggregate: Aggregate): Aggregate = {
- val nondeterministicToAttributes: LinkedHashMap[Expression, NamedExpression] =
- NondeterministicExpressionCollection.getNondeterministicToAttributes(
- aggregate.groupingExpressions
- )
-
- if (!nondeterministicToAttributes.isEmpty) {
- val newChild = Project(
- scopes.current.output ++ nondeterministicToAttributes.values.asScala.toSeq,
- aggregate.child
- )
- val resolvedAggregateExpressions = aggregate.aggregateExpressions.map { expression =>
- PullOutNondeterministicExpressionInExpressionTree(expression, nondeterministicToAttributes)
- }
- val resolvedGroupingExpressions = aggregate.groupingExpressions.map { expression =>
- PullOutNondeterministicExpressionInExpressionTree(
- expression,
- nondeterministicToAttributes
- )
- }
- aggregate.copy(
- groupingExpressions = resolvedGroupingExpressions,
- aggregateExpressions = resolvedAggregateExpressions,
- child = newChild
- )
- } else {
- aggregate
- }
- }
-
private def canGroupByAll(expressions: Seq[Expression]): Boolean = {
val isOrderByAll = expressions match {
case Seq(unresolvedAttribute: UnresolvedAttribute) =>
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AggregateWithLcaResolutionResult.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AggregateWithLcaResolutionResult.scala
index 535c41cad39ec..15d4ac4f5ceb3 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AggregateWithLcaResolutionResult.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AggregateWithLcaResolutionResult.scala
@@ -18,7 +18,7 @@
package org.apache.spark.sql.catalyst.analysis.resolver
import org.apache.spark.sql.catalyst.expressions.{Alias, NamedExpression}
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan}
/**
* Stores the result of resolution of lateral column aliases in an [[Aggregate]].
@@ -26,8 +26,11 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
* @param outputList The output list of the resolved operator.
* @param aggregateListAliases List of aliases from aggregate list and all artificially inserted
* [[Project]] nodes.
+ * @param baseAggregate [[Aggregate]] node constructed by [[LateralColumnAliasResolver]] while
+ * resolving lateral column references in [[Aggregate]].
*/
case class AggregateWithLcaResolutionResult(
resolvedOperator: LogicalPlan,
outputList: Seq[NamedExpression],
- aggregateListAliases: Seq[Alias])
+ aggregateListAliases: Seq[Alias],
+ baseAggregate: Aggregate)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AliasResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AliasResolver.scala
index 83329c0fe464d..b56281fa12ba8 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AliasResolver.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AliasResolver.scala
@@ -17,13 +17,14 @@
package org.apache.spark.sql.catalyst.analysis.resolver
-import org.apache.spark.sql.catalyst.analysis.{AliasResolution, MultiAlias, UnresolvedAlias}
+import org.apache.spark.sql.catalyst.analysis.{AliasResolution, UnresolvedAlias}
import org.apache.spark.sql.catalyst.expressions.{
Alias,
Expression,
NamedExpression,
OuterReference
}
+import org.apache.spark.sql.errors.QueryCompilationErrors
/**
* Resolver class that resolves unresolved aliases and handles user-specified aliases.
@@ -45,25 +46,32 @@ class AliasResolver(expressionResolver: ExpressionResolver)
* we create a new [[Alias]] using the [[AutoGeneratedAliasProvider]]. Here we allow inner
* aliases to persist until the end of single-pass resolution, after which they will be removed
* in the post-processing phase.
+ *
+ * Resulting [[Alias]] must be added to the list of `availableAliases` in the current
+ * [[NameScope]].
*/
override def resolve(unresolvedAlias: UnresolvedAlias): NamedExpression =
- scopes.current.lcaRegistry.withNewLcaScope {
+ scopes.current.lcaRegistry.withNewLcaScope(
+ isTopLevelAlias = expressionResolutionContextStack.peek().isTopOfProjectList
+ ) {
val aliasWithResolvedChildren =
withResolvedChildren(unresolvedAlias, expressionResolver.resolve _)
.asInstanceOf[UnresolvedAlias]
- val resolvedAlias =
+ val resolvedNode =
AliasResolution.resolve(aliasWithResolvedChildren).asInstanceOf[NamedExpression]
- resolvedAlias match {
- case multiAlias: MultiAlias =>
- throw new ExplicitlyUnsupportedResolverFeature(
- s"unsupported expression: ${multiAlias.getClass.getName}"
- )
+ resolvedNode match {
case alias: Alias =>
- expressionResolver.getExpressionIdAssigner.mapExpression(alias)
+ val resultAlias = expressionResolver.getExpressionIdAssigner.mapExpression(alias)
+ scopes.current.availableAliases.add(resultAlias.exprId)
+ resultAlias
case outerReference: OuterReference =>
autoGeneratedAliasProvider.newAlias(outerReference)
+ case _ =>
+ throw QueryCompilationErrors.unsupportedSinglePassAnalyzerFeature(
+ s"${resolvedNode.getClass} expression resolution"
+ )
}
}
@@ -77,18 +85,21 @@ class AliasResolver(expressionResolver: ExpressionResolver)
* those aliases. See [[ExpressionIdAssigner.mapExpression]] doc for more details.
*/
def handleResolvedAlias(alias: Alias): Alias = {
- val resolvedAlias = scopes.current.lcaRegistry.withNewLcaScope {
+ val resolvedAlias = scopes.current.lcaRegistry.withNewLcaScope(
+ isTopLevelAlias = expressionResolutionContextStack.peek().isTopOfProjectList
+ ) {
val aliasWithResolvedChildren =
withResolvedChildren(alias, expressionResolver.resolve _).asInstanceOf[Alias]
- val mappedAlias = expressionResolver.getExpressionIdAssigner.mapExpression(
+ val resultAlias = expressionResolver.getExpressionIdAssigner.mapExpression(
originalExpression = aliasWithResolvedChildren,
prioritizeOldDuplicateAliasId =
expressionResolutionContextStack.peek().resolvingGroupingExpressions
)
- scopes.current.availableAliases.add(mappedAlias.exprId)
- mappedAlias
+ scopes.current.availableAliases.add(resultAlias.exprId)
+
+ resultAlias
}
collapseAlias(resolvedAlias)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AutoGeneratedAliasProvider.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AutoGeneratedAliasProvider.scala
index 2a49581b3499b..5fd5a5ff7870e 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AutoGeneratedAliasProvider.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/AutoGeneratedAliasProvider.scala
@@ -62,7 +62,7 @@ class AutoGeneratedAliasProvider(expressionIdAssigner: ExpressionIdAssigner) {
name: Option[String] = None,
explicitMetadata: Option[Metadata] = None,
skipExpressionIdAssigner: Boolean = false): Alias = {
- var alias = Alias(
+ val alias = Alias(
child = child,
name = name.getOrElse(toPrettySQL(child))
)(
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/BinaryArithmeticResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/BinaryArithmeticResolver.scala
index d88a450a5acda..0668c0a6513b7 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/BinaryArithmeticResolver.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/BinaryArithmeticResolver.scala
@@ -17,19 +17,11 @@
package org.apache.spark.sql.catalyst.analysis.resolver
-import org.apache.spark.sql.catalyst.analysis.{
- AnsiStringPromotionTypeCoercion,
- AnsiTypeCoercion,
- BinaryArithmeticWithDatetimeResolver,
- DecimalPrecisionTypeCoercion,
- DivisionTypeCoercion,
- IntegralDivisionTypeCoercion,
- StringPromotionTypeCoercion,
- TypeCoercion
-}
+import org.apache.spark.sql.catalyst.analysis.BinaryArithmeticWithDatetimeResolver
import org.apache.spark.sql.catalyst.expressions.{
Add,
BinaryArithmetic,
+ Cast,
DateAdd,
Divide,
Expression,
@@ -37,7 +29,7 @@ import org.apache.spark.sql.catalyst.expressions.{
Subtract,
SubtractDates
}
-import org.apache.spark.sql.types.{DateType, StringType}
+import org.apache.spark.sql.types._
/**
* [[BinaryArithmeticResolver]] is invoked by [[ExpressionResolver]] in order to resolve
@@ -63,7 +55,7 @@ import org.apache.spark.sql.types.{DateType, StringType}
*
* Cast(
* DatetimeSub(
- * TimeAdd(
+ * TimestampAddInterval(
* Literal('4 11:11', StringType),
* UnaryMinus(
* Literal(Interval('4 22:12' DAY TO MINUTE), DayTimeIntervalType(0,2))
@@ -77,8 +69,8 @@ import org.apache.spark.sql.types.{DateType, StringType}
* top-most node itself is not resolved recursively in order to avoid recursive calls to
* [[BinaryArithmeticResolver]] and other sub-resolvers. To prevent a case where we resolve the
* same node twice, we need to mark nodes that will act as a limit for the downwards traversal by
- * applying a [[ExpressionResolver.SINGLE_PASS_SUBTREE_BOUNDARY]] tag to them. These children
- * along with all the nodes below them are guaranteed to be resolved at this point. When
+ * applying a [[ResolverTag.SINGLE_PASS_SUBTREE_BOUNDARY]] tag to them. These children along with
+ * all the nodes below them are guaranteed to be resolved at this point. When
* [[ExpressionResolver]] reaches one of the tagged nodes, it returns identity rather than
* resolving it. Finally, after resolving the subtree, we need to resolve the top-most node itself,
* which in this case means applying a timezone, if necessary.
@@ -90,11 +82,6 @@ class BinaryArithmeticResolver(expressionResolver: ExpressionResolver)
private val traversals = expressionResolver.getExpressionTreeTraversals
- protected override val ansiTransformations: CoercesExpressionTypes.Transformations =
- BinaryArithmeticResolver.ANSI_TYPE_COERCION_TRANSFORMATIONS
- protected override val nonAnsiTransformations: CoercesExpressionTypes.Transformations =
- BinaryArithmeticResolver.TYPE_COERCION_TRANSFORMATIONS
-
override def resolve(unresolvedBinaryArithmetic: BinaryArithmetic): Expression = {
val binaryArithmeticWithResolvedChildren: BinaryArithmetic =
withResolvedChildren(unresolvedBinaryArithmetic, expressionResolver.resolve _)
@@ -117,8 +104,9 @@ class BinaryArithmeticResolver(expressionResolver: ExpressionResolver)
* of nodes.
*/
private def transformBinaryArithmeticNode(binaryArithmetic: BinaryArithmetic): Expression = {
+ val binaryArithmeticWithNullReplaced: Expression = replaceNullType(binaryArithmetic)
val binaryArithmeticWithDateTypeReplaced: Expression =
- replaceDateType(binaryArithmetic)
+ replaceDateType(binaryArithmeticWithNullReplaced)
val binaryArithmeticWithTypeCoercion: Expression =
coerceExpressionTypes(
expression = binaryArithmeticWithDateTypeReplaced,
@@ -154,26 +142,29 @@ class BinaryArithmeticResolver(expressionResolver: ExpressionResolver)
BinaryArithmeticWithDatetimeResolver.resolve(arithmetic)
case other => other
}
-}
-object BinaryArithmeticResolver {
- // Ordering in the list of type coercions should be in sync with the list in [[TypeCoercion]].
- private val TYPE_COERCION_TRANSFORMATIONS: Seq[Expression => Expression] = Seq(
- StringPromotionTypeCoercion.apply,
- DecimalPrecisionTypeCoercion.apply,
- DivisionTypeCoercion.apply,
- IntegralDivisionTypeCoercion.apply,
- TypeCoercion.ImplicitTypeCoercion.apply,
- TypeCoercion.DateTimeOperationsTypeCoercion.apply
- )
-
- // Ordering in the list of type coercions should be in sync with the list in [[AnsiTypeCoercion]].
- private val ANSI_TYPE_COERCION_TRANSFORMATIONS: Seq[Expression => Expression] = Seq(
- AnsiStringPromotionTypeCoercion.apply,
- DecimalPrecisionTypeCoercion.apply,
- DivisionTypeCoercion.apply,
- IntegralDivisionTypeCoercion.apply,
- AnsiTypeCoercion.ImplicitTypeCoercion.apply,
- AnsiTypeCoercion.AnsiDateTimeOperationsTypeCoercion.apply
- )
+ /**
+ * Replaces NullType by a compatible type in arithmetic expressions over Datetime operands.
+ * This avoids recursive calls of [[BinaryArithmeticWithDatetimeResolver]] which converts
+ * unacceptable nulls of `NullType` to an expected types of datetime expressions at the
+ * first step, and replacing arithmetic `Add` and `Subtract` by the same datetime expressions
+ * on the following steps.
+ */
+ private def replaceNullType(expression: Expression): Expression = expression match {
+ case a @ Add(l, r, _) => (l.dataType, r.dataType) match {
+ case (_: DatetimeType, _: NullType) =>
+ a.copy(right = Cast(a.right, DayTimeIntervalType.DEFAULT))
+ case (_: NullType, _: DatetimeType) =>
+ a.copy(left = Cast(a.left, DayTimeIntervalType.DEFAULT))
+ case _ => a
+ }
+ case s @ Subtract(l, r, _) => (l.dataType, r.dataType) match {
+ case (_: NullType, _: DatetimeType) =>
+ s.copy(left = Cast(s.left, s.right.dataType))
+ case (_: DatetimeType, _: NullType) =>
+ s.copy(right = Cast(s.right, s.left.dataType))
+ case _ => s
+ }
+ case other => other
+ }
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TryExtractOrdinal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/CandidatesForResolution.scala
similarity index 67%
rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TryExtractOrdinal.scala
rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/CandidatesForResolution.scala
index 42766a78e248f..3b8ec61bf5ee8 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TryExtractOrdinal.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/CandidatesForResolution.scala
@@ -17,18 +17,10 @@
package org.apache.spark.sql.catalyst.analysis.resolver
-import org.apache.spark.sql.catalyst.expressions.{Expression, IntegerLiteral}
+import org.apache.spark.sql.catalyst.expressions.Attribute
/**
- * Try to extract ordinal from an expression. Return `Some(ordinal)` if the type of the expression
- * is [[IntegerLitera]], `None` otherwise.
+ * [[CandidatesForResolution]] is used by the [[NameScope]] during multipart name resolution to
+ * prioritize attributes from different types of operator output (main, hidden, metadata).
*/
-object TryExtractOrdinal {
- def apply(expression: Expression): Option[Int] = {
- expression match {
- case IntegerLiteral(literal) =>
- Some(literal)
- case other => None
- }
- }
-}
+case class CandidatesForResolution(attributes: Seq[Attribute], outputType: OutputType.OutputType)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/CoercesExpressionTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/CoercesExpressionTypes.scala
index 0fc6a6742edad..34be99a1abc0d 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/CoercesExpressionTypes.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/CoercesExpressionTypes.scala
@@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.analysis.{
TypeCoercion
}
import org.apache.spark.sql.catalyst.expressions.{Cast, Expression}
+import org.apache.spark.sql.catalyst.trees.CurrentOrigin.withOrigin
/**
* [[CoercesExpressionTypes]] is extended by resolvers that need to apply type coercion.
@@ -57,27 +58,42 @@ trait CoercesExpressionTypes extends SQLConfHelper {
*
* In the end, we apply [[DefaultCollationTypeCoercion]].
* See [[DefaultCollationTypeCoercion]] doc for more info.
+ *
+ * Additionally, we copy the tags and origin in case the call to this method didn't come from
+ * [[ExpressionResolver]], where they are copied generically.
*/
def coerceExpressionTypes(
expression: Expression,
expressionTreeTraversal: ExpressionTreeTraversal): Expression = {
- val coercedExpressionOnce = applyTypeCoercion(
- expression = expression,
- expressionTreeTraversal = expressionTreeTraversal
- )
- // This is a hack necessary because fixed-point analyzer sometimes requires multiple passes to
- // resolve type coercion. Instead, in single pass, we apply type coercion twice on the same
- // node in order to ensure that types are resolved.
- val coercedExpressionTwice = applyTypeCoercion(
- expression = coercedExpressionOnce,
- expressionTreeTraversal = expressionTreeTraversal
- )
+ withOrigin(expression.origin) {
+ val coercedExpressionOnce = applyTypeCoercion(
+ expression = expression,
+ expressionTreeTraversal = expressionTreeTraversal
+ )
+
+ // If the expression isn't changed by the first iteration of type coercion,
+ // second iteration won't be effective either.
+ val expressionAfterTypeCoercion = if (coercedExpressionOnce.eq(expression)) {
+ coercedExpressionOnce
+ } else {
+ // This is a hack necessary because fixed-point analyzer sometimes requires multiple passes
+ // to resolve type coercion. Instead, in single pass, we apply type coercion twice on the
+ // same node in order to ensure that types are resolved.
+ applyTypeCoercion(
+ expression = coercedExpressionOnce,
+ expressionTreeTraversal = expressionTreeTraversal
+ )
+ }
+
+ val coercionResult = expressionTreeTraversal.defaultCollation match {
+ case Some(defaultCollation) =>
+ DefaultCollationTypeCoercion(expressionAfterTypeCoercion, defaultCollation)
+ case None =>
+ expressionAfterTypeCoercion
+ }
- expressionTreeTraversal.defaultCollation match {
- case Some(defaultCollation) =>
- DefaultCollationTypeCoercion(coercedExpressionTwice, defaultCollation)
- case None =>
- coercedExpressionTwice
+ coercionResult.copyTagsFrom(expression)
+ coercionResult
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ConditionalExpressionResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ConditionalExpressionResolver.scala
deleted file mode 100644
index 0d847cf09adf0..0000000000000
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ConditionalExpressionResolver.scala
+++ /dev/null
@@ -1,63 +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.catalyst.analysis.resolver
-
-import org.apache.spark.sql.catalyst.analysis.{AnsiTypeCoercion, TypeCoercion}
-import org.apache.spark.sql.catalyst.expressions.{ConditionalExpression, Expression}
-
-/**
- * Resolver for [[If]], [[CaseWhen]] and [[Coalesce]] expressions.
- */
-class ConditionalExpressionResolver(expressionResolver: ExpressionResolver)
- extends TreeNodeResolver[ConditionalExpression, Expression]
- with ResolvesExpressionChildren
- with CoercesExpressionTypes {
-
- private val traversals = expressionResolver.getExpressionTreeTraversals
-
- protected override val ansiTransformations: CoercesExpressionTypes.Transformations =
- ConditionalExpressionResolver.ANSI_TYPE_COERCION_TRANSFORMATIONS
- protected override val nonAnsiTransformations: CoercesExpressionTypes.Transformations =
- ConditionalExpressionResolver.TYPE_COERCION_TRANSFORMATIONS
-
- override def resolve(unresolvedConditionalExpression: ConditionalExpression): Expression = {
- val conditionalExpressionWithResolvedChildren =
- withResolvedChildren(unresolvedConditionalExpression, expressionResolver.resolve _)
-
- coerceExpressionTypes(
- expression = conditionalExpressionWithResolvedChildren,
- expressionTreeTraversal = traversals.current
- )
- }
-}
-
-object ConditionalExpressionResolver {
- // Ordering in the list of type coercions should be in sync with the list in [[TypeCoercion]].
- private val TYPE_COERCION_TRANSFORMATIONS: Seq[Expression => Expression] = Seq(
- TypeCoercion.CaseWhenTypeCoercion.apply,
- TypeCoercion.FunctionArgumentTypeCoercion.apply,
- TypeCoercion.IfTypeCoercion.apply
- )
-
- // Ordering in the list of type coercions should be in sync with the list in [[AnsiTypeCoercion]].
- private val ANSI_TYPE_COERCION_TRANSFORMATIONS: Seq[Expression => Expression] = Seq(
- AnsiTypeCoercion.CaseWhenTypeCoercion.apply,
- AnsiTypeCoercion.FunctionArgumentTypeCoercion.apply,
- AnsiTypeCoercion.IfTypeCoercion.apply
- )
-}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/CreateNamedStructResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/CreateNamedStructResolver.scala
deleted file mode 100644
index d0e4ecea25cb3..0000000000000
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/CreateNamedStructResolver.scala
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.catalyst.analysis.resolver
-
-import org.apache.spark.sql.catalyst.expressions.{Alias, CreateNamedStruct, Expression}
-
-/**
- * Resolves [[CreateNamedStruct]] nodes by recursively resolving children. If [[CreateNamedStruct]]
- * is not directly under an [[Alias]], removes aliases from struct fields. Otherwise, let
- * [[AliasResolver]] handle the removal.
- */
-class CreateNamedStructResolver(expressionResolver: ExpressionResolver)
- extends TreeNodeResolver[CreateNamedStruct, Expression]
- with ResolvesExpressionChildren {
-
- override def resolve(createNamedStruct: CreateNamedStruct): Expression = {
- val createNamedStructWithResolvedChildren =
- withResolvedChildren(createNamedStruct, expressionResolver.resolve)
- .asInstanceOf[CreateNamedStruct]
- CreateNamedStructResolver.cleanupAliases(createNamedStructWithResolvedChildren)
- }
-}
-
-object CreateNamedStructResolver {
-
- /**
- * For a query like:
- *
- * {{{ SELECT STRUCT(1 AS a, 2 AS b) }}}
- *
- * [[CreateNamedStruct]] will be: CreateNamedStruct(Seq("a", Alias(1, "a"), "b", Alias(2, "b")))
- *
- * Because inner aliases are not expected in the analyzed logical plan, we need to remove them
- * here. However, we only do so if [[CreateNamedStruct]] is not directly under an [[Alias]], in
- * which case the removal will be handled by [[AliasResolver]]. This is because in single-pass,
- * [[Alias]] is resolved after [[CreateNamedStruct]] and in order to compute the correct output
- * name, it needs to know complete structure of the child.
- */
- def cleanupAliases(createNamedStruct: CreateNamedStruct): CreateNamedStruct = {
- createNamedStruct
- .withNewChildren(createNamedStruct.children.map {
- case a: Alias if a.metadata.isEmpty => a.child
- case other => other
- })
- .asInstanceOf[CreateNamedStruct]
- }
-}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ExpressionIdAssigner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ExpressionIdAssigner.scala
index 71bc7f24d5f28..9cd4c525b48c2 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ExpressionIdAssigner.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ExpressionIdAssigner.scala
@@ -326,9 +326,17 @@ class ExpressionIdAssigner {
* child mappings will have collisions during this merge operation. We need to decide which of
* the new IDs get the priority for the old ID. This is done based on the IDs that are actually
* outputted into the multi-child operator. This information is provided with `newOutputIds`.
- * If the new ID is present in that set, we treat it as a P0 over the IDs that are hidden in the
- * branch. Also, we iterate over child mappings from right to left, prioritizing IDs from the
- * left, because that's how operators like [[Union]] propagate IDs upwards.
+ *
+ * The principles:
+ * 1. If the destination ID is present in `newOutputIds`, we treat it as a higher priority over
+ * the ID that is "hidden" in the logical plan branch.
+ * 2. If both destination IDs are present in `newOutputIds`, we prioritize the identity mapping -
+ * the new ID which is equal to the old ID, and not the "remapping". This is valid in SQL
+ * because we are dealing with a fully unresolved plan and the remapping is not needed.
+ * DataFrame queries that contain a self-join or a self-union and are referencing the same
+ * attribute from both branches will fail (which is expected).
+ * 3. We iterate over child mappings from right to left, prioritizing IDs from the left, because
+ * that's how multi-child operators like [[Join]] or [[Union]] propagate IDs upwards.
*
* Example 1:
* {{{
@@ -360,8 +368,19 @@ class ExpressionIdAssigner {
* df2.join(df1, df2("b") === df1("a"))
* }}}
*
- * This is used by multi child operators like [[Join]] or [[Union]] to propagate mapped
- * expression IDs upwards.
+ * Example 3:
+ * {{{
+ * -- In this query CTE references a table which is also present in a JOIN. First, CTE definition
+ * -- is analyzed with `t1` inside. Let's say it outputs col1#0. Once we get to a left JOIN child,
+ * -- which is also `t1`, we know that expression IDs in `t1` have to be regenerated to col#1
+ * -- because it's a duplicate relation. After resolving the JOIN, we are left with (#0 -> #0),
+ * -- (#1 -> #1) and (#0 -> #1) mappings. Also, JOIN outputs both #0 and #1. This is an example
+ * -- of principle 2. when identity (#0 -> #0) and (#1 -> #1) mappings have to be prioritized,
+ * -- because (#0 -> #1) is a remapping and not needed in SQL.
+ * SELECT * FROM (
+ * WITH cte1 AS (SELECT * FROM t1) SELECT t1.col1 FROM t1 JOIN cte1 USING (col1)
+ * );
+ * }}}
*
* When `mergeIntoExisting` is true, we merge child mappings into an existing mapping entry
* instead of creating a new one. This setting is used when resolving [[LateralJoin]]s.
@@ -380,15 +399,19 @@ class ExpressionIdAssigner {
throw SparkException.internalError("No child mappings to create new current mapping")
}
- val priorityMapping = new ExpressionIdAssigner.PriorityMapping(newOutputIds.size)
+ val newMapping = if (mergeIntoExisting) {
+ currentStackEntry.mapping.get
+ } else {
+ new ExpressionIdAssigner.Mapping
+ }
while (!currentStackEntry.childMappings.isEmpty) {
val nextMapping = currentStackEntry.childMappings.pop()
nextMapping.forEach {
case (oldId, remappedId) =>
- updatePriorityMapping(
- priorityMapping = priorityMapping,
+ updateNewMapping(
+ newMapping = newMapping,
oldId = oldId,
remappedId = remappedId,
newOutputIds = newOutputIds
@@ -396,17 +419,6 @@ class ExpressionIdAssigner {
}
}
- val newMapping = if (mergeIntoExisting) {
- currentStackEntry.mapping.get
- } else {
- new ExpressionIdAssigner.Mapping
- }
-
- priorityMapping.forEach {
- case (oldId, priority) =>
- newMapping.put(oldId, priority.pick())
- }
-
setCurrentMapping(newMapping)
}
@@ -606,27 +618,31 @@ class ExpressionIdAssigner {
}
/**
- * Update the priority mapping for the given `oldId` and `remappedId`. If the `remappedId` is
- * contained in the `newOutputIds`, we treat it as a P0 over the IDs that are not exposed from
- * the operator branch. Otherwise, we treat it as a P1.
+ * Update `newMapping` with the `oldId -> remappedId` mapping, based on the principles described
+ * in [[createMappingFromChildMappings]]:
+ * 1. If no mapping from `oldId` exists, we create it
+ * 2. If the mapping from `oldId` already exists but is not present in `newOutputIds`, we
+ * deprioritize old mapping in favor of new one
+ * 3. If the mapping from `oldId` already exists and is present in `newOutputIds` and the new
+ * mapping is the identity one, we deprioritize old mapping in favor of new one
+ * 4. Otherwise we keep the existing mapping
*/
- private def updatePriorityMapping(
- priorityMapping: ExpressionIdAssigner.PriorityMapping,
+ private def updateNewMapping(
+ newMapping: ExpressionIdAssigner.Mapping,
oldId: ExprId,
remappedId: ExprId,
newOutputIds: Set[ExprId]): Unit = {
- if (newOutputIds.contains(remappedId)) {
- priorityMapping.merge(
- oldId,
- ExpressionIdPriority(p0 = Some(remappedId)),
- (priority, _) => priority.copy(p0 = Some(remappedId))
- )
- } else {
- priorityMapping.merge(
- oldId,
- ExpressionIdPriority(p1 = Some(remappedId)),
- (priority, _) => priority.copy(p1 = Some(remappedId))
- )
+ newMapping.get(oldId) match {
+ case null =>
+ newMapping.put(oldId, remappedId)
+
+ case knownRemappedId if !newOutputIds.contains(knownRemappedId) =>
+ newMapping.put(oldId, remappedId)
+
+ case knownRemappedId if newOutputIds.contains(remappedId) && remappedId == oldId =>
+ newMapping.put(oldId, remappedId)
+
+ case _ =>
}
}
}
@@ -641,8 +657,6 @@ object ExpressionIdAssigner {
type Stack = ArrayDeque[StackEntry]
- type PriorityMapping = HashMap[ExprId, ExpressionIdPriority]
-
/**
* Assert that `outputs` don't have conflicting expression IDs.
*/
@@ -696,15 +710,3 @@ object ExpressionIdAssigner {
}
}
}
-
-/**
- * [[ExpressionIdPriority]] is used by the [[ExpressionIdAssigner]] when merging child mappings
- * of a multi-child operator to determine which new ID gets picked in case of an old ID collision.
- */
-case class ExpressionIdPriority(p0: Option[ExprId] = None, p1: Option[ExprId] = None) {
- def pick(): ExprId = p0.getOrElse {
- p1.getOrElse {
- throw SparkException.internalError("No expression ID to pick")
- }
- }
-}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ExpressionResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ExpressionResolver.scala
index cfeaa54a39f60..2b78b92518b91 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ExpressionResolver.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ExpressionResolver.scala
@@ -40,7 +40,6 @@ import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction}
import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, LogicalPlan, Sort}
import org.apache.spark.sql.catalyst.trees.CurrentOrigin
-import org.apache.spark.sql.catalyst.trees.TreeNodeTag
import org.apache.spark.sql.catalyst.util.CollationFactory
import org.apache.spark.sql.errors.QueryCompilationErrors
@@ -126,7 +125,6 @@ class ExpressionResolver(
private val aliasResolver = new AliasResolver(this)
private val timezoneAwareExpressionResolver = new TimezoneAwareExpressionResolver(this)
- private val conditionalExpressionResolver = new ConditionalExpressionResolver(this)
private val binaryArithmeticResolver = new BinaryArithmeticResolver(this)
private val limitLikeExpressionValidator = new LimitLikeExpressionValidator
private val aggregateExpressionResolver = new AggregateExpressionResolver(resolver, this)
@@ -136,11 +134,10 @@ class ExpressionResolver(
aggregateExpressionResolver,
binaryArithmeticResolver
)
- private val timeAddResolver = new TimeAddResolver(this)
- private val unaryMinusResolver = new UnaryMinusResolver(this)
private val subqueryExpressionResolver = new SubqueryExpressionResolver(this, resolver)
private val ordinalResolver = new OrdinalResolver(this)
private val lcaResolver = new LateralColumnAliasResolver(this)
+ private val semiStructuredExtractResolver = new SemiStructuredExtractResolver(this)
/**
* Get the expression tree traversal stack.
@@ -262,16 +259,16 @@ class ExpressionResolver(
subqueryExpressionResolver.resolveScalarSubquery(unresolvedScalarSubquery)
case unresolvedListQuery: ListQuery =>
subqueryExpressionResolver.resolveListQuery(unresolvedListQuery)
- case unresolvedTimeAdd: TimeAdd =>
- timeAddResolver.resolve(unresolvedTimeAdd)
+ case unresolvedTimestampAdd: TimestampAddInterval =>
+ resolveExpressionGenericallyWithTimezoneWithTypeCoercion(unresolvedTimestampAdd)
case unresolvedUnaryMinus: UnaryMinus =>
- unaryMinusResolver.resolve(unresolvedUnaryMinus)
+ resolveExpressionGenericallyWithTypeCoercion(unresolvedUnaryMinus)
case createNamedStruct: CreateNamedStruct =>
resolveExpressionGenerically(createNamedStruct)
case sortOrder: SortOrder =>
resolveExpressionGenerically(sortOrder)
case unresolvedConditionalExpression: ConditionalExpression =>
- conditionalExpressionResolver.resolve(unresolvedConditionalExpression)
+ resolveExpressionGenericallyWithTypeCoercion(unresolvedConditionalExpression)
case getViewColumnByNameAndOrdinal: GetViewColumnByNameAndOrdinal =>
resolveGetViewColumnByNameAndOrdinal(getViewColumnByNameAndOrdinal)
case getTimeField: GetTimeField =>
@@ -286,6 +283,8 @@ class ExpressionResolver(
resolveUpCast(unresolvedUpCast)
case unresolvedCollation: UnresolvedCollation =>
resolveCollation(unresolvedCollation)
+ case semiStructuredExtract: SemiStructuredExtract =>
+ semiStructuredExtractResolver.resolve(semiStructuredExtract)
case expression: Expression =>
resolveExpressionGenericallyWithTypeCoercion(expression)
}
@@ -584,15 +583,10 @@ class ExpressionResolver(
aliasResolver.resolve(unresolvedAlias)
case unresolvedAttribute: UnresolvedAttribute =>
resolveAttribute(unresolvedAttribute)
- case unresolvedStar: UnresolvedStar =>
- // We don't support edge cases of star usage, e.g. `WHERE col1 IN (*)`
- throw new ExplicitlyUnsupportedResolverFeature("Star outside of Project list")
case attributeReference: AttributeReference =>
handleResolvedAttributeReference(attributeReference)
case outerReference: OuterReference =>
handleResolvedOuterReference(outerReference)
- case _: UnresolvedNamedLambdaVariable =>
- throw new ExplicitlyUnsupportedResolverFeature("Lambda variables")
case _ =>
withPosition(unresolvedNamedExpression) {
throwUnsupportedSinglePassAnalyzerFeature(unresolvedNamedExpression)
@@ -646,6 +640,9 @@ class ExpressionResolver(
* In case that attribute is resolved as a literal function (i.e. result is [[CurrentDate]]),
* perform additional resolution on it.
*
+ * In case result of the previous step is a recursive data type, we coerce it to stay compatible
+ * with the fixed-point analyzer.
+ *
* If the attribute is at the top of the project list (which is indicated by
* [[ExpressionResolutionContext.isTopOfProjectList]]), we preserve the [[Alias]] or remove it
* otherwise.
@@ -667,7 +664,6 @@ class ExpressionResolver(
.resolvingGroupingExpressions && traversals.current.groupByAliases
),
canResolveNameByHiddenOutput = canResolveNameByHiddenOutput,
- shouldPreferTableColumnsOverAliases = shouldPreferTableColumnsOverAliases,
shouldPreferHiddenOutput = traversals.current.isFilterOnTopOfAggregate,
canResolveNameByHiddenOutputInSubquery =
subqueryRegistry.currentScope.aggregateExpressionsExtractor.isDefined,
@@ -689,7 +685,7 @@ class ExpressionResolver(
case Some(lateralAttributeReference) =>
scopes.current.lcaRegistry
.markAttributeLaterallyReferenced(lateralAttributeReference)
- candidate.setTagValue(ExpressionResolver.SINGLE_PASS_IS_LCA, ())
+ candidate.setTagValue(ResolverTag.SINGLE_PASS_IS_LCA, ())
expressionResolutionContext.hasLateralColumnAlias = true
case None =>
}
@@ -703,11 +699,16 @@ class ExpressionResolver(
case other => other
}
+ val coercedCandidate = candidateOrLiteralFunction match {
+ case extractValue: ExtractValue => coerceRecursiveDataTypes(extractValue)
+ case other => other
+ }
+
val properlyAliasedExpressionTree =
if (expressionResolutionContext.isTopOfProjectList && nameTarget.aliasName.isDefined) {
- Alias(candidateOrLiteralFunction, nameTarget.aliasName.get)()
+ Alias(coercedCandidate, nameTarget.aliasName.get)()
} else {
- candidateOrLiteralFunction
+ coercedCandidate
}
properlyAliasedExpressionTree match {
@@ -718,17 +719,41 @@ class ExpressionResolver(
}
}
+ /**
+ * Coerces recursive types ([[ExtractValue]] expressions) in a bottom up manner. For example:
+ *
+ * {{{
+ * CREATE OR REPLACE TABLE t(col MAP);
+ * SELECT col.field FROM t;
+ * }}}
+ *
+ * In this example we need to cast inner field from `String` to `BIGINT`, thus analyzed plan
+ * should look like:
+ *
+ * {{{
+ * Project [col#x[cast(field as bigint)] AS field#x]
+ * +- SubqueryAlias spark_catalog.default.t
+ * +- Relation spark_catalog.default.t[col#x] parquet
+ * }}}
+ *
+ * This is needed to stay compatible with the fixed-point implementation.
+ */
+ private def coerceRecursiveDataTypes(extractValue: ExtractValue): Expression = {
+ extractValue.transformUp {
+ case field => coerceExpressionTypes(field, traversals.current)
+ }
+ }
+
private def isFilterOnTopOfAggregate(parentOperator: LogicalPlan): Boolean = {
parentOperator match {
- case _ @Filter(_, _: Aggregate) => true
+ case _: Filter if scopes.current.baseAggregate.isDefined => true
case _ => false
}
}
private def isSortOnTopOfAggregate(parentOperator: LogicalPlan): Boolean = {
parentOperator match {
- case _ @Sort(_, _, _: Aggregate, _) => true
- case _ @Sort(_, _, _ @Filter(_, _: Aggregate), _) => true
+ case _: Sort if scopes.current.baseAggregate.isDefined => true
case _ => false
}
}
@@ -738,11 +763,6 @@ class ExpressionResolver(
case other => false
}
- private def shouldPreferTableColumnsOverAliases = traversals.current.parentOperator match {
- case _: Sort => true
- case _ => false
- }
-
/**
* [[AttributeReference]] is already resolved if it's passed to us from DataFrame `col(...)`
* function, for example.
@@ -1027,8 +1047,12 @@ class ExpressionResolver(
TypeCoercionValidation.failOnTypeCheckResult(resolvedExpression)
}
- if (!resolvedExpression.resolved) {
- throwSinglePassFailedToResolveExpression(resolvedExpression)
+ resolvedExpression match {
+ case runtimeReplaceable: RuntimeReplaceable if !runtimeReplaceable.replacement.resolved =>
+ throwFailedToResolveRuntimeReplaceableExpression(runtimeReplaceable)
+ case expression if !expression.resolved =>
+ throwSinglePassFailedToResolveExpression(resolvedExpression)
+ case _ =>
}
validateExpressionUnderSupportedOperator(resolvedExpression)
@@ -1066,10 +1090,12 @@ class ExpressionResolver(
context = expression.origin.getQueryContext,
summary = expression.origin.context.summary()
)
-}
-object ExpressionResolver {
- private val AMBIGUOUS_SELF_JOIN_METADATA = Seq("__dataset_id", "__col_position")
- val SINGLE_PASS_SUBTREE_BOUNDARY = TreeNodeTag[Unit]("single_pass_subtree_boundary")
- val SINGLE_PASS_IS_LCA = TreeNodeTag[Unit]("single_pass_is_lca")
+ private def throwFailedToResolveRuntimeReplaceableExpression(
+ runtimeReplaceable: RuntimeReplaceable) = {
+ throw SparkException.internalError(
+ s"Cannot resolve the runtime replaceable expression ${toSQLExpr(runtimeReplaceable)}. " +
+ s"The replacement is unresolved: ${toSQLExpr(runtimeReplaceable.replacement)}."
+ )
+ }
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/FilterResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/FilterResolver.scala
index 599270b28b1d4..4489278f0b238 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/FilterResolver.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/FilterResolver.scala
@@ -65,8 +65,7 @@ class FilterResolver(resolver: Resolver, expressionResolver: ExpressionResolver)
retainOriginalOutput(
operator = finalFilter,
missingExpressions = missingAttributes,
- output = scopes.current.output,
- hiddenOutput = scopes.current.hiddenOutput
+ scopes = scopes
)
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/GroupingAndAggregateExpressionsExtractor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/GroupingAndAggregateExpressionsExtractor.scala
index e28526959f851..4929ffc57b6da 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/GroupingAndAggregateExpressionsExtractor.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/GroupingAndAggregateExpressionsExtractor.scala
@@ -33,13 +33,23 @@ import org.apache.spark.sql.catalyst.plans.logical.Aggregate
class GroupingAndAggregateExpressionsExtractor(
aggregate: Aggregate,
autoGeneratedAliasProvider: AutoGeneratedAliasProvider) {
- private val aliasChildToAliasInAggregateExpressions = new IdentityHashMap[Expression, Alias]
+
+ /**
+ * Maps children of aliases from aggregate list to their parents or to `None` if the expression
+ * doesn't have an alias. This map only accounts for the first appearance of the expression. For
+ * example, for `SELECT col1, col1 AS a` map entry should be (col1 -> None), but for
+ * `SELECT col1 AS a, col1` map entry should be (col1 -> Some(a)).
+ */
+ private val aliasChildToAliasInAggregateExpressions =
+ new IdentityHashMap[Expression, Option[Alias]]
private val aggregateExpressionsSemanticComparator = new SemanticComparator(
aggregate.aggregateExpressions.map {
case alias: Alias =>
- aliasChildToAliasInAggregateExpressions.put(alias.child, alias)
+ aliasChildToAliasInAggregateExpressions.putIfAbsent(alias.child, Some(alias))
alias.child
- case other => other
+ case other =>
+ aliasChildToAliasInAggregateExpressions.put(other, None)
+ other
}
)
@@ -53,43 +63,48 @@ class GroupingAndAggregateExpressionsExtractor(
* (grouping expressions used for extraction) and `extractedAggregateExpressionAliases` (aliases
* of [[AggregateExpression]]s that are transformed to attributes during extraction) in order to
* insert missing attributes to below operators.
+ *
+ * When an expression exists in both grouping and aggregate expressions (for example, when there
+ * are lateral column references in [[Aggregate]], LCA algorithm will copy grouping expressions
+ * to aggregate list prior to entering this method), we still extract grouping expression but
+ * don't add it later if it is not necessary.
*/
def extractReferencedGroupingAndAggregateExpressions(
expression: Expression,
referencedGroupingExpressions: mutable.ArrayBuffer[NamedExpression],
extractedAggregateExpressionAliases: mutable.ArrayBuffer[Alias]): Expression = {
- collectFirstAggregateExpression(expression) match {
- case (Some(attribute: Attribute), _)
- if !aliasChildToAliasInAggregateExpressions.containsKey(attribute) =>
- attribute
- case (Some(expression), alias) =>
- alias match {
- case None =>
- throw SparkException.internalError(
- s"No parent alias for expression $expression while extracting aggregate" +
- s"expressions in Sort operator."
- )
- case Some(alias) =>
- alias.toAttribute
- }
- case (None, _) if groupingExpressionsSemanticComparator.exists(expression) =>
- expression match {
- case attribute: Attribute =>
- referencedGroupingExpressions += attribute
- attribute
- case other =>
- val alias = autoGeneratedAliasProvider.newAlias(child = other)
- referencedGroupingExpressions += alias
- alias.toAttribute
- }
- case _ =>
- expression match {
- case aggregateExpression: AggregateExpression =>
- val alias = autoGeneratedAliasProvider.newAlias(child = aggregateExpression)
- extractedAggregateExpressionAliases += alias
- alias.toAttribute
- case other => other
- }
+ val aggregateExpressionWithAlias = collectFirstAggregateExpression(expression)
+ val isGroupingExpression = groupingExpressionsSemanticComparator.exists(expression)
+ if (isGroupingExpression) {
+ val groupingExpressionReference = aggregateExpressionWithAlias match {
+ case (Some(attribute: Attribute), None) => attribute
+ case (Some(_), Some(alias)) => alias.toAttribute
+ case _ =>
+ expression match {
+ case attribute: Attribute => attribute
+ case other => autoGeneratedAliasProvider.newAlias(child = other)
+ }
+ }
+ referencedGroupingExpressions += groupingExpressionReference
+ groupingExpressionReference.toAttribute
+ } else {
+ aggregateExpressionWithAlias match {
+ case (Some(attribute: Attribute), None) => attribute
+ case (Some(_), Some(alias)) => alias.toAttribute
+ case (Some(expression), None) =>
+ throw SparkException.internalError(
+ s"No parent alias for expression $expression while extracting aggregate" +
+ s"expressions in an operator."
+ )
+ case _ =>
+ expression match {
+ case aggregateExpression: AggregateExpression =>
+ val alias = autoGeneratedAliasProvider.newAlias(child = aggregateExpression)
+ extractedAggregateExpressionAliases += alias
+ alias.toAttribute
+ case other => other
+ }
+ }
}
}
@@ -103,7 +118,7 @@ class GroupingAndAggregateExpressionsExtractor(
aggregateExpressionsSemanticComparator.collectFirst(expression)
referencedAggregateExpression match {
case Some(expression) =>
- (Some(expression), Option(aliasChildToAliasInAggregateExpressions.get(expression)))
+ (Some(expression), aliasChildToAliasInAggregateExpressions.get(expression))
case None =>
(None, None)
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/HavingResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/HavingResolver.scala
index 4bbcd43a0377a..ec0907807d5d0 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/HavingResolver.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/HavingResolver.scala
@@ -22,13 +22,14 @@ import scala.collection.mutable
import org.apache.spark.SparkException
import org.apache.spark.sql.catalyst.analysis.UnresolvedHaving
import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression}
-import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, LogicalPlan}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, LogicalPlan, Project}
/**
* Resolves [[UnresolvedHaving]] node and its condition.
*/
class HavingResolver(resolver: Resolver, expressionResolver: ExpressionResolver)
extends TreeNodeResolver[UnresolvedHaving, LogicalPlan]
+ with RewritesAliasesInTopLcaProject
with ResolvesNameByHiddenOutput
with ValidatesFilter {
private val scopes: NameScopeStack = resolver.getNameScopes
@@ -53,29 +54,48 @@ class HavingResolver(resolver: Resolver, expressionResolver: ExpressionResolver)
val partiallyResolvedHaving =
Filter(condition = unresolvedHaving.havingCondition, child = resolvedChild)
- val resolvedCondition = expressionResolver.resolveExpressionTreeInOperator(
+ val partiallyResolvedCondition = expressionResolver.resolveExpressionTreeInOperator(
partiallyResolvedHaving.condition,
partiallyResolvedHaving
)
- val (finalCondition, missingExpressions) = resolvedChild match {
- case _ if scopes.current.hasLcaInAggregate =>
- throw new ExplicitlyUnsupportedResolverFeature(
- "Lateral column alias in Aggregate below HAVING"
+ val (resolvedCondition, missingExpressions) = resolvedChild match {
+ case _ @(_: Project | _: Aggregate) if scopes.current.baseAggregate.isDefined =>
+ handleAggregateBelowHaving(
+ scopes.current.baseAggregate.get,
+ partiallyResolvedCondition
)
- case aggregate: Aggregate =>
- handleAggregateBelowHaving(aggregate, resolvedCondition)
case other =>
throw SparkException.internalError(
s"Unexpected operator ${other.getClass.getSimpleName} under HAVING"
)
}
+ val (resolvedConditionWithAliasReplacement, filteredMissingExpressions) =
+ tryReplaceSortOrderOrHavingConditionWithAlias(resolvedCondition, scopes, missingExpressions)
+
+ val deduplicatedMissingExpressions = deduplicateMissingExpressions(filteredMissingExpressions)
+
val resolvedChildWithMissingAttributes =
- insertMissingExpressions(resolvedChild, missingExpressions.toSeq)
+ insertMissingExpressions(resolvedChild, deduplicatedMissingExpressions)
+
+ val isChildChangedByMissingExpressions = !resolvedChildWithMissingAttributes.eq(resolvedChild)
+
+ val (finalChild, finalCondition) = resolvedChildWithMissingAttributes match {
+ case project: Project if scopes.current.baseAggregate.isDefined =>
+ val (newProject, newExpressions) = rewriteNamedExpressionsInTopLcaProject(
+ projectToRewrite = project,
+ baseAggregate = scopes.current.baseAggregate.get,
+ expressionsToRewrite = Seq(resolvedConditionWithAliasReplacement),
+ rewriteCandidates = missingExpressions,
+ autoGeneratedAliasProvider = autoGeneratedAliasProvider
+ )
+ (newProject, newExpressions.head)
+ case other => (other, resolvedCondition)
+ }
val resolvedHaving = partiallyResolvedHaving.copy(
- child = resolvedChildWithMissingAttributes,
+ child = finalChild,
condition = finalCondition
)
@@ -85,12 +105,15 @@ class HavingResolver(resolver: Resolver, expressionResolver: ExpressionResolver)
resolvedFilter = resolvedHaving
)
- retainOriginalOutput(
- operator = resolvedHaving,
- missingExpressions = missingExpressions.toSeq,
- output = scopes.current.output,
- hiddenOutput = scopes.current.hiddenOutput
- )
+ if (isChildChangedByMissingExpressions) {
+ retainOriginalOutput(
+ operator = resolvedHaving,
+ missingExpressions = missingExpressions.toSeq,
+ scopes = scopes
+ )
+ } else {
+ resolvedHaving
+ }
}
/**
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/HybridAnalyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/HybridAnalyzer.scala
index c21d1aacadeaf..0117b3fc2fb55 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/HybridAnalyzer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/HybridAnalyzer.scala
@@ -75,21 +75,17 @@ class HybridAnalyzer(
private val sampleRateGenerator = new Random()
def apply(plan: LogicalPlan, tracker: QueryPlanningTracker): LogicalPlan = {
- val passedResolvedGuard = resolverGuard.apply(plan)
-
val dualRun =
conf.getConf(SQLConf.ANALYZER_DUAL_RUN_LEGACY_AND_SINGLE_PASS_RESOLVER) &&
- passedResolvedGuard &&
- checkDualRunSampleRate()
+ checkDualRunSampleRate() &&
+ checkResolverGuard(plan)
withTrackedAnalyzerBridgeState(dualRun) {
if (dualRun) {
resolveInDualRun(plan, tracker)
} else if (conf.getConf(SQLConf.ANALYZER_SINGLE_PASS_RESOLVER_ENABLED)) {
resolveInSinglePass(plan, tracker)
- } else if (passedResolvedGuard && conf.getConf(
- SQLConf.ANALYZER_SINGLE_PASS_RESOLVER_ENABLED_TENTATIVELY
- )) {
+ } else if (conf.getConf(SQLConf.ANALYZER_SINGLE_PASS_RESOLVER_ENABLED_TENTATIVELY)) {
resolveInSinglePassTentatively(plan, tracker)
} else {
resolveInFixedPoint(plan, tracker)
@@ -97,10 +93,6 @@ class HybridAnalyzer(
}
}
- def getSinglePassResolutionDuration: Option[Long] = singlePassResolutionDuration
-
- def getFixedPointResolutionDuration: Option[Long] = fixedPointResolutionDuration
-
/**
* Call `body` in the context of tracked [[AnalyzerBridgeState]]. Set the new bridge state
* depending on whether we are in dual-run mode or not:
@@ -206,10 +198,21 @@ class HybridAnalyzer(
private def resolveInSinglePassTentatively(
plan: LogicalPlan,
tracker: QueryPlanningTracker): LogicalPlan = {
- try {
- resolveInSinglePass(plan, tracker)
- } catch {
- case _: ExplicitlyUnsupportedResolverFeature =>
+ val singlePassResult = if (checkResolverGuard(plan)) {
+ try {
+ Some(resolveInSinglePass(plan, tracker))
+ } catch {
+ case _: ExplicitlyUnsupportedResolverFeature =>
+ None
+ }
+ } else {
+ None
+ }
+
+ singlePassResult match {
+ case Some(result) =>
+ result
+ case None =>
resolveInFixedPoint(plan, tracker)
}
}
@@ -256,6 +259,16 @@ class HybridAnalyzer(
}
}
+ private def checkResolverGuard(plan: LogicalPlan): Boolean = {
+ try {
+ resolverGuard.apply(plan)
+ } catch {
+ case e: Throwable
+ if !conf.getConf(SQLConf.ANALYZER_SINGLE_PASS_RESOLVER_EXPOSE_RESOLVER_GUARD_FAILURE) =>
+ false
+ }
+ }
+
/**
* Normalizes the logical plan using [[NormalizePlan]].
*
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/JoinResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/JoinResolver.scala
index ff752bd76f41e..c718a8ba3782c 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/JoinResolver.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/JoinResolver.scala
@@ -208,7 +208,7 @@ class JoinResolver(resolver: Resolver, expressionResolver: ExpressionResolver)
scopes.current.hiddenOutput.filter(_.qualifiedAccessOnly)
val newProjectList =
- if (unresolvedJoin.getTagValue(Resolver.TOP_LEVEL_OPERATOR).isEmpty) {
+ if (unresolvedJoin.getTagValue(ResolverTag.TOP_LEVEL_OPERATOR).isEmpty) {
newOutputList ++ qualifiedAccessOnlyColumnsFromHiddenOutput
} else {
newOutputList
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/KeyTransformingMap.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/KeyTransformingMap.scala
index ae56c6ed04193..7815b3b0c79cd 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/KeyTransformingMap.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/KeyTransformingMap.scala
@@ -17,16 +17,17 @@
package org.apache.spark.sql.catalyst.analysis.resolver
-import java.util.{Collection, HashMap, Iterator}
+import java.util.{Collection, Iterator, LinkedHashMap}
import java.util.Map.Entry
import java.util.function.Function
/**
* The [[KeyTransformingMap]] is a partial implementation of [[mutable.Map]] that transforms input
* keys with a custom [[mapKey]] method.
+ * It preserves the order of insertion by using the [[LinkedHashMap]] as an underlying map.
*/
private abstract class KeyTransformingMap[K, V] {
- private val impl = new HashMap[K, V]
+ private val impl = new LinkedHashMap[K, V]
def get(key: K): Option[V] = Option(impl.get(mapKey(key)))
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/LateralColumnAliasProhibitedRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/LateralColumnAliasProhibitedRegistry.scala
index bc0f11f5bd6de..e6e084115adec 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/LateralColumnAliasProhibitedRegistry.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/LateralColumnAliasProhibitedRegistry.scala
@@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute}
* idempotent.
*/
class LateralColumnAliasProhibitedRegistry extends LateralColumnAliasRegistry {
- def withNewLcaScope(body: => Alias): Alias = body
+ def withNewLcaScope(isTopLevelAlias: Boolean)(body: => Alias): Alias = body
def getAttribute(attributeName: String): Option[Attribute] =
throwLcaResolutionNotEnabled()
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/LateralColumnAliasRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/LateralColumnAliasRegistry.scala
index 45a38417a8eed..fa539a4d9110f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/LateralColumnAliasRegistry.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/LateralColumnAliasRegistry.scala
@@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute}
* LCA resolution is disabled by [[SQLConf.LATERAL_COLUMN_ALIAS_IMPLICIT_ENABLED]].
*/
abstract class LateralColumnAliasRegistry {
- def withNewLcaScope(body: => Alias): Alias
+ def withNewLcaScope(isTopLevelAlias: Boolean)(body: => Alias): Alias
def getAttribute(attributeName: String): Option[Attribute]
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/LateralColumnAliasRegistryImpl.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/LateralColumnAliasRegistryImpl.scala
index c685b098db2d2..94520455ee8d3 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/LateralColumnAliasRegistryImpl.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/LateralColumnAliasRegistryImpl.scala
@@ -81,13 +81,15 @@ class LateralColumnAliasRegistryImpl(attributes: Seq[Attribute])
/**
* Creates a new LCA resolution scope for each [[Alias]] resolution. Executes the lambda and
- * registers the resolved alias for later LCA resolution.
+ * registers top-level resolved aliases for later LCA resolution.
*/
- def withNewLcaScope(body: => Alias): Alias = {
+ def withNewLcaScope(isTopLevelAlias: Boolean)(body: => Alias): Alias = {
currentAttributeDependencyLevelStack.push(0)
try {
val resolvedAlias = body
- registerAlias(resolvedAlias)
+ if (isTopLevelAlias) {
+ registerAlias(resolvedAlias)
+ }
resolvedAlias
} finally {
currentAttributeDependencyLevelStack.pop()
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/LateralColumnAliasResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/LateralColumnAliasResolver.scala
index 1ffaf73fc356a..5d268f087aa92 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/LateralColumnAliasResolver.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/LateralColumnAliasResolver.scala
@@ -57,9 +57,13 @@ class LateralColumnAliasResolver(expressionResolver: ExpressionResolver) extends
* - In order to be able to resolve [[Sort]] on top of an [[Aggregate]] that has LCAs, we need
* to collect all aliases from [[Aggregate]], as well as any aliases from artificially inserted
* [[Project]] nodes.
+ * - Collects all aliases from newly created [[Aggregate]] and [[Project]] nodes and adds them
+ * to `aliasesToCollect`
*/
def handleLcaInAggregate(resolvedAggregate: Aggregate): AggregateWithLcaResolutionResult = {
- extractLcaAndReplaceAggWithProject(resolvedAggregate) match {
+ val aliasesToCollect = new ArrayBuffer[Alias]
+
+ extractLcaAndReplaceAggWithProject(resolvedAggregate, aliasesToCollect) match {
case _ @Project(projectList: Seq[_], aggregate: Aggregate) =>
// TODO: This validation function does a post-traversal. This is discouraged in single-pass
// Analyzer.
@@ -76,21 +80,17 @@ class LateralColumnAliasResolver(expressionResolver: ExpressionResolver) extends
scope = scopes.current,
originalProjectList = projectList,
firstIterationProjectList = aggregate.aggregateExpressions.map(_.toAttribute),
- remappedAliases = Some(remappedAliases)
+ remappedAliases = Some(remappedAliases),
+ aliasesToCollect = aliasesToCollect
)
- val aggregateListAliases =
- scopes.current.lcaRegistry.getAliasDependencyLevels().asScala.flatMap(_.asScala).toSeq
-
- scopes.overwriteCurrent(
- output = Some(finalProject.projectList.map(_.toAttribute)),
- hasLcaInAggregate = true
- )
+ scopes.overwriteCurrent(output = Some(finalProject.projectList.map(_.toAttribute)))
AggregateWithLcaResolutionResult(
resolvedOperator = finalProject,
outputList = finalProject.projectList,
- aggregateListAliases = aggregateListAliases
+ aggregateListAliases = aliasesToCollect.toSeq,
+ baseAggregate = aggregate
)
case _ =>
throw SparkException.internalError(
@@ -128,6 +128,8 @@ class LateralColumnAliasResolver(expressionResolver: ExpressionResolver) extends
* full definitions ( `attr` as `name` ) have already been defined on lower levels.
* - If an attribute is never referenced, it does not show up in multi-level project lists, but
* instead only in the top-most [[Project]].
+ * - Additionally, collect all aliases from newly created [[Project]] nodes and add them to
+ * `aliasesToCollect`.
*
* For previously given query, following above rules, resolved [[Project]] would look like:
*
@@ -142,7 +144,8 @@ class LateralColumnAliasResolver(expressionResolver: ExpressionResolver) extends
scope: NameScope,
originalProjectList: Seq[NamedExpression],
firstIterationProjectList: Seq[NamedExpression],
- remappedAliases: Option[HashMap[ExprId, Alias]] = None): Project = {
+ remappedAliases: Option[HashMap[ExprId, Alias]] = None,
+ aliasesToCollect: ArrayBuffer[Alias] = ArrayBuffer.empty): Project = {
val aliasDependencyMap = scope.lcaRegistry.getAliasDependencyLevels()
val (finalChildPlan, _) = aliasDependencyMap.asScala.foldLeft(
(resolvedChild, firstIterationProjectList)
@@ -159,6 +162,12 @@ class LateralColumnAliasResolver(expressionResolver: ExpressionResolver) extends
if (referencedAliases.nonEmpty) {
val newProjectList = currentProjectList.map(_.toAttribute) ++ referencedAliases
+
+ newProjectList.foreach {
+ case alias: Alias => aliasesToCollect += alias
+ case _ =>
+ }
+
(Project(newProjectList, currentPlan), newProjectList)
} else {
(currentPlan, currentProjectList)
@@ -173,6 +182,11 @@ class LateralColumnAliasResolver(expressionResolver: ExpressionResolver) extends
}
}
+ finalProjectList.foreach {
+ case alias: Alias => aliasesToCollect += alias
+ case _ =>
+ }
+
Project(finalProjectList, finalChildPlan)
}
@@ -184,6 +198,8 @@ class LateralColumnAliasResolver(expressionResolver: ExpressionResolver) extends
* [[NamedExpression]] we don't need to alias it again.
* - Places a [[Project]] on top of the new [[Aggregate]] operator, where the project list will
* be created from [[Alias]] references to original aggregate expressions.
+ * - Additionally, collect aliases from newly created aggregate expressions and add them to
+ * `aliasesToCollect`.
*
* For example, for a query like:
*
@@ -198,7 +214,9 @@ class LateralColumnAliasResolver(expressionResolver: ExpressionResolver) extends
* The [[Project]] is unresolved, which is fine, because it will later be resolved as if we only
* had a lateral alias reference in [[Project]] and not [[Aggregate]].
*/
- private def extractLcaAndReplaceAggWithProject(aggregate: Aggregate): Project = {
+ private def extractLcaAndReplaceAggWithProject(
+ aggregate: Aggregate,
+ aliasesToCollect: ArrayBuffer[Alias]): Project = {
val newAggregateExpressions = new LinkedHashSet[NamedExpression]
val extractedExpressionAliases = new HashMap[Expression, NamedExpression]()
val groupingExpressionSemanticComparator = new SemanticComparator(aggregate.groupingExpressions)
@@ -212,9 +230,16 @@ class LateralColumnAliasResolver(expressionResolver: ExpressionResolver) extends
newAggregateExpressions = newAggregateExpressions
).asInstanceOf[NamedExpression]
)
+
+ val newAggregateExpressionsSeq = newAggregateExpressions.asScala.toSeq
+ newAggregateExpressionsSeq.foreach {
+ case alias: Alias => aliasesToCollect += alias
+ case _ =>
+ }
+
val result = Project(
projectList = extractedExpressions,
- child = aggregate.copy(aggregateExpressions = newAggregateExpressions.asScala.toSeq)
+ child = aggregate.copy(aggregateExpressions = newAggregateExpressionsSeq)
)
result
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/LimitExpressionResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/LimitExpressionResolver.scala
deleted file mode 100644
index d25112d78c6e7..0000000000000
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/LimitExpressionResolver.scala
+++ /dev/null
@@ -1,110 +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.catalyst.analysis.resolver
-
-import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.catalyst.expressions.Expression
-import org.apache.spark.sql.types.IntegerType
-
-/**
- * The [[LimitExpressionResolver]] is a resolver that resolves a [[LocalLimit]] or [[GlobalLimit]]
- * expression and performs all the necessary validation.
- */
-class LimitExpressionResolver extends TreeNodeResolver[Expression, Expression] {
-
- /**
- * Resolve a limit expression of [[GlobalLimit]] or [[LocalLimit]] and perform validation.
- */
- override def resolve(unresolvedLimitExpression: Expression): Expression = {
- validateLimitExpression(unresolvedLimitExpression, expressionName = "limit")
- unresolvedLimitExpression
- }
-
- /**
- * Validate a resolved limit expression of [[GlobalLimit]] or [[LocalLimit]]:
- * - The expression has to be foldable
- * - The result data type has to be [[IntegerType]]
- * - The evaluated expression has to be non-null
- * - The evaluated expression has to be positive
- *
- * The `foldable` check is implemented in some expressions
- * as a recursive expression tree traversal.
- * It is not an ideal approach for the single-pass [[ExpressionResolver]],
- * but __is__ practical, since:
- * - We have to call `eval` here anyway, and it's recursive
- * - In practice `LIMIT` expression trees are very small
- */
- private def validateLimitExpression(expression: Expression, expressionName: String): Unit = {
- if (!expression.foldable) {
- throwInvalidLimitLikeExpressionIsUnfoldable(expressionName, expression)
- }
- if (expression.dataType != IntegerType) {
- throwInvalidLimitLikeExpressionDataType(expressionName, expression)
- }
- expression.eval() match {
- case null =>
- throwInvalidLimitLikeExpressionIsNull(expressionName, expression)
- case value: Int if value < 0 =>
- throwInvalidLimitLikeExpressionIsNegative(expressionName, expression, value)
- case _ =>
- }
- }
-
- private def throwInvalidLimitLikeExpressionIsUnfoldable(
- name: String,
- expression: Expression): Nothing =
- throw new AnalysisException(
- errorClass = "INVALID_LIMIT_LIKE_EXPRESSION.IS_UNFOLDABLE",
- messageParameters = Map(
- "name" -> name,
- "expr" -> toSQLExpr(expression)
- ),
- origin = expression.origin
- )
-
- private def throwInvalidLimitLikeExpressionDataType(
- name: String,
- expression: Expression): Nothing =
- throw new AnalysisException(
- errorClass = "INVALID_LIMIT_LIKE_EXPRESSION.DATA_TYPE",
- messageParameters = Map(
- "name" -> name,
- "expr" -> toSQLExpr(expression),
- "dataType" -> toSQLType(expression.dataType)
- ),
- origin = expression.origin
- )
-
- private def throwInvalidLimitLikeExpressionIsNull(name: String, expression: Expression): Nothing =
- throw new AnalysisException(
- errorClass = "INVALID_LIMIT_LIKE_EXPRESSION.IS_NULL",
- messageParameters = Map("name" -> name, "expr" -> toSQLExpr(expression)),
- origin = expression.origin
- )
-
- private def throwInvalidLimitLikeExpressionIsNegative(
- name: String,
- expression: Expression,
- value: Int): Nothing =
- throw new AnalysisException(
- errorClass = "INVALID_LIMIT_LIKE_EXPRESSION.IS_NEGATIVE",
- messageParameters =
- Map("name" -> name, "expr" -> toSQLExpr(expression), "v" -> toSQLValue(value, IntegerType)),
- origin = expression.origin
- )
-}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/MetadataResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/MetadataResolver.scala
index bccc038f87eff..12ca6b4b333d6 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/MetadataResolver.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/MetadataResolver.scala
@@ -17,17 +17,9 @@
package org.apache.spark.sql.catalyst.analysis.resolver
-import org.apache.spark.sql.catalyst.analysis.{
- FunctionResolution,
- RelationResolution,
- UnresolvedRelation
-}
-import org.apache.spark.sql.catalyst.plans.logical.{
- AnalysisHelper,
- LogicalPlan,
- SubqueryAlias,
- UnresolvedWith
-}
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.analysis.{RelationResolution, UnresolvedRelation}
+import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.trees.TreePattern.{UNRESOLVED_RELATION, UNRESOLVED_WITH}
import org.apache.spark.sql.connector.catalog.CatalogManager
@@ -45,9 +37,9 @@ import org.apache.spark.sql.connector.catalog.CatalogManager
class MetadataResolver(
override val catalogManager: CatalogManager,
override val relationResolution: RelationResolution,
- functionResolution: FunctionResolution,
override val extensions: Seq[ResolverExtension] = Seq.empty)
- extends RelationMetadataProvider
+ extends SQLConfHelper
+ with RelationMetadataProvider
with DelegatesResolutionToExtensions {
override val relationsWithResolvedMetadata = new RelationsWithResolvedMetadata
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/NameScope.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/NameScope.scala
index aed5b767b2066..3ccae116cb187 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/NameScope.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/NameScope.scala
@@ -38,6 +38,7 @@ import org.apache.spark.sql.catalyst.expressions.{
NamedExpression,
OuterReference
}
+import org.apache.spark.sql.catalyst.plans.logical.Aggregate
import org.apache.spark.sql.catalyst.util._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.Metadata
@@ -141,8 +142,9 @@ import org.apache.spark.sql.types.Metadata
* to this [[NameScope]]. If the [[Aggregate]] has lateral column references, this list contains
* both the aliases from [[Aggregate]] as well as all aliases from artificially inserted
* [[Project]] nodes.
- * @param hasLcaInAggregate Flag that indicates whether there is a lateral column alias reference
- * in the [[Aggregate]] corresponding to this [[NameScope]].
+ * @param baseAggregate [[Aggregate]] node that is either a resolved [[Aggregate]] corresponding to
+ * this node or base [[Aggregate]] constructed when resolving lateral column references in
+ * [[Aggregate]].
*/
class NameScope(
val output: Seq[Attribute] = Seq.empty,
@@ -150,7 +152,7 @@ class NameScope(
val isSubqueryRoot: Boolean = false,
val availableAliases: HashSet[ExprId] = new HashSet[ExprId],
val aggregateListAliases: Seq[Alias] = Seq.empty,
- val hasLcaInAggregate: Boolean = false,
+ val baseAggregate: Option[Aggregate] = None,
planLogger: PlanLogger = new PlanLogger
) extends SQLConfHelper {
@@ -223,21 +225,21 @@ class NameScope(
/**
* Returns new [[NameScope]] which preserves all the immutable [[NameScope]] properties but
* overwrites `output`, `hiddenOutput`, `availableAliases`, `aggregateListAliases` and
- * `hasLcaInAggregate` if provided. Mutable state like `lcaRegistry` is not preserved.
+ * `baseAggregate` if provided. Mutable state like `lcaRegistry` is not preserved.
*/
def overwrite(
output: Option[Seq[Attribute]] = None,
hiddenOutput: Option[Seq[Attribute]] = None,
availableAliases: Option[HashSet[ExprId]] = None,
aggregateListAliases: Seq[Alias] = Seq.empty,
- hasLcaInAggregate: Boolean = false): NameScope = {
+ baseAggregate: Option[Aggregate] = None): NameScope = {
new NameScope(
output = output.getOrElse(this.output),
hiddenOutput = hiddenOutput.getOrElse(this.hiddenOutput),
isSubqueryRoot = isSubqueryRoot,
availableAliases = availableAliases.getOrElse(this.availableAliases),
aggregateListAliases = aggregateListAliases,
- hasLcaInAggregate = hasLcaInAggregate || this.hasLcaInAggregate,
+ baseAggregate = baseAggregate,
planLogger = planLogger
)
}
@@ -300,6 +302,26 @@ class NameScope(
def getOrdinalReplacementExpressions: Option[OrdinalReplacementExpressions] =
ordinalReplacementExpressions
+ /**
+ * Returns attribute with `expressionId` if `output` contains it. This is used to preserve
+ * nullability for resolved [[AttributeReference]].
+ */
+ def getAttributeById(expressionId: ExprId): Option[Attribute] =
+ Option(outputById.get(expressionId))
+
+ /**
+ * Returns attribute with `expressionId` if `hiddenOutput` contains it.
+ */
+ def getHiddenAttributeById(expressionId: ExprId): Option[Attribute] =
+ Option(hiddenAttributesById.get(expressionId))
+
+ /**
+ * Return all the explicitly outputted expression IDs. Hidden or metadata output are not included.
+ */
+ def getOutputIds: Set[ExprId] = {
+ output.map(_.exprId).toSet
+ }
+
/**
* Expand the [[UnresolvedStar]]. The expected use case for this method is star expansion inside
* [[Project]].
@@ -351,6 +373,28 @@ class NameScope(
)
}
+ /**
+ * Find attributes in this [[NameScope]] that match a provided one-part `name`.
+ *
+ * This method is simpler and more lightweight than [[resolveMultipartName]], because here we
+ * just return all the attributes matched by the one-part `name`. This is only suitable
+ * for situations where name _resolution_ is not required (e.g. accessing struct fields
+ * from the lower operator's output).
+ *
+ * For example, this method is used to look up attributes to match a specific [[View]] schema.
+ * See [[ExpressionResolver.resolveGetViewColumnByNameAndOrdinal]] for more info on view column
+ * lookup.
+ *
+ * We are relying on a simple [[IdentifierMap]] to perform that work, since we just need to match
+ * one-part name from the lower operator's output here.
+ */
+ def findAttributesByName(name: String): Seq[Attribute] = {
+ attributesByName.get(name) match {
+ case Some(attributes) => attributes.toSeq
+ case None => Seq.empty
+ }
+ }
+
/**
* Resolve multipart name into a [[NameTarget]]. [[NameTarget]]'s `candidates` may contain
* simple [[AttributeReference]]s if it's a column or alias, or [[ExtractValue]] expressions if
@@ -445,19 +489,6 @@ class NameScope(
* Even though there is ambiguity with the name `col1`, the [[SortOrder]] expression should be
* resolved as a table column from the project list and not throw [[AMBIGUOUS_REFERENCE]].
*
- * On the other hand, in the following example:
- *
- * {{{
- * val df = sql("SELECT 1 AS col1, col1 FROM VALUES(1)")
- * df.select("col1")
- * }}}
- *
- * Resolution of name `col1` in the second [[Project]] produce [[AMBIGUOUS_REFERENCE]] error.
- *
- * In order to achieve this we are using [[shouldPreferTableColumnsOverAliases]] flag which
- * should be set to true when the parent operator is [[Sort]] and only when we are resolving by
- * `output` (we don't consider this flag for `metadataOutput` or `hiddenOutput`).
- *
* The names in [[Aggregate.groupingExpressions]] can reference
* [[Aggregate.aggregateExpressions]] aliases. `canReferenceAggregateExpressionAliases` will be
* true when we are resolving the grouping expressions.
@@ -484,17 +515,35 @@ class NameScope(
* SELECT COUNT(col1) FROM t1 GROUP BY col1 ORDER BY MAX(col2);
* }}}
*
- * We are relying on the [[AttributeSeq]] to perform that work, since it requires complex
- * resolution logic involving nested field extraction and multipart name matching.
+ * Spark is being smart about name resolution and prioritizes candidates from output levels that
+ * can actually be resolved, even though that output level might not be the first choice.
+ * For example, ORDER BY clause prefers attributes from SELECT list (namely, aliases) over table
+ * columns from below. However, if attributes on the SELECT level have name ambiguity or other
+ * issues, Spark will try to resolve the name using the table columns from below. Examples:
+ *
+ * {{{
+ * CREATE TABLE t1 (col1 INT);
+ * CREATE TABLE t2 (col1 STRUCT);
+ *
+ * -- Main output is ambiguous, so col1 from t1 is used for sorting.
+ * SELECT 1 AS col1, 2 AS col1 FROM t1 ORDER BY col1;
+ *
+ * -- col1 from main output does not have `field`, so struct field of col1 from t2 is used for
+ * -- sorting.
+ * SELECT 1 AS col1 FROM t2 ORDER BY col1.field;
+ * }}}
+ *
+ * This is achieved using candidate prioritization mechanism in [[pickSuitableCandidates]].
*
- * Also, see [[AttributeSeq.resolve]] for more details.
+ * We are relying on the [[AttributeSeq]] to perform name resolution, since it requires complex
+ * resolution logic involving nested field extraction and multipart name matching. See
+ * [[AttributeSeq.resolve]] for more details.
*/
def resolveMultipartName(
multipartName: Seq[String],
canLaterallyReferenceColumn: Boolean = false,
canReferenceAggregateExpressionAliases: Boolean = false,
canResolveNameByHiddenOutput: Boolean = false,
- shouldPreferTableColumnsOverAliases: Boolean = false,
shouldPreferHiddenOutput: Boolean = false,
canReferenceAggregatedAccessOnlyAttributes: Boolean = false): NameTarget = {
val resolvedMultipartName: ResolvedMultipartName =
@@ -502,7 +551,6 @@ class NameScope(
multipartName = multipartName,
nameComparator = nameComparator,
canResolveNameByHiddenOutput = canResolveNameByHiddenOutput,
- shouldPreferTableColumnsOverAliases = shouldPreferTableColumnsOverAliases,
shouldPreferHiddenOutput = shouldPreferHiddenOutput,
canReferenceAggregatedAccessOnlyAttributes = canReferenceAggregatedAccessOnlyAttributes
).orElse(tryResolveMultipartNameAsLiteralFunction(multipartName))
@@ -536,149 +584,104 @@ class NameScope(
}
/**
- * Find attributes in this [[NameScope]] that match a provided one-part `name`.
- *
- * This method is simpler and more lightweight than [[resolveMultipartName]], because here we
- * just return all the attributes matched by the one-part `name`. This is only suitable
- * for situations where name _resolution_ is not required (e.g. accessing struct fields
- * from the lower operator's output).
- *
- * For example, this method is used to look up attributes to match a specific [[View]] schema.
- * See [[ExpressionResolver.resolveGetViewColumnByNameAndOrdinal]] for more info on view column
- * lookup.
- *
- * We are relying on a simple [[IdentifierMap]] to perform that work, since we just need to match
- * one-part name from the lower operator's output here.
- */
- def findAttributesByName(name: String): Seq[Attribute] = {
- attributesByName.get(name) match {
- case Some(attributes) => attributes.toSeq
- case None => Seq.empty
- }
- }
-
- /**
- * Returns attribute with `expressionId` if `output` contains it. This is used to preserve
- * nullability for resolved [[AttributeReference]].
- */
- def getAttributeById(expressionId: ExprId): Option[Attribute] =
- Option(outputById.get(expressionId))
-
- /**
- * Returns attribute with `expressionId` if `hiddenOutput` contains it.
- */
- def getHiddenAttributeById(expressionId: ExprId): Option[Attribute] =
- Option(hiddenAttributesById.get(expressionId))
-
- /**
- * Return all the explicitly outputted expression IDs. Hidden or metadata output are not included.
- */
- def getOutputIds: Set[ExprId] = {
- output.map(_.exprId).toSet
- }
-
- /**
- * Resolution by attributes available in the current [[NameScope]] is done in the following way:
- * - First, we resolve the name using all the available attributes in the current scope
- * - For all the candidates that are found, we lookup the expression IDs in the maps created
- * when [[NameScope]] is updated to distinguish attributes resolved using the main output,
- * hidden output and metadata output (for hidden output, we use
- * `canReferenceAggregatedAccessOnlyAttributes` flag to determine if all the attributes can be
- * used or only the ones that are not tagged as `aggregatedAccessOnly`).
- * - We prioritize the hidden output over the other ones if `shouldPreferHiddenOutput` is set to
- * true. This is done in case of HAVING where attributes from grouping expressions of the
- * underlying [[Aggregate]] are preferred over aliases from operator below. Example:
- *
- * {{{ SELECT 1 AS col1 FROM VALUES(1, 2) GROUP BY col1 HAVING col1 = 1; }}}
- *
- * Plan would be:
- * Project [col1#2]
- * +- Filter (col1#1 = 1)
- * +- Aggregate [col1#1], [a AS col1#2, col1#1]
- * +- LocalRelation [col1#1, col2#3]
- *
- * Otherwise, we prioritize main output over the metadata output and metadata output
- * over the hidden output.
- * - If `shouldPreferTableColumnsOverAliases` is set to true, we prefer the table columns over
- * the aliases which can be used for name resolution.
- * - If we didn't find any candidates this way we fallback to other ways of resolution described
- * in `resolveMultipartName` doc.
+ * Try resolve [[multipartName]] using attributes from a relevant operator output. This algorithm
+ * splits candidates from [[attributesForResolution]] into several groups and picks the best match
+ * ensuring that there's no choice ambiguity.
+ *
+ * Detailed flow:
+ * 1. Match the given [[multipartName]] using
+ * [[attributesForResolution.getCandidatesForResolution]] and get a subset of candidates for
+ * that name.
+ * 2. If nested fields were inferred during the name matching process, we are dealing with
+ * struct/map/array field/element extraction. Further narrow down those attributes that are
+ * suitable for field extraction using [[ExtractValue.isExtractable]]. We can safely do this
+ * right away, because nested fields cannot be applied to non-recursive data types.
+ * 3. Triage the candidates into several groups: main output, metadata output and hidden output.
+ * Main output is the topmost output of a relevant operator (actual SELECT list). Metadata
+ * output is a special qualified-access only output which originates from [[NaturalJoin]] or
+ * [[UsingJoin]] and can only be accessed by a qualified multipart name. If we have it, it
+ * means that [[attributesForResolution.getCandidatesForResolution]] inferred a qualified
+ * attribute name. Hidden output is only used if [[canResolveNameByHiddenOutput]] is specified
+ * (in ORDER BY and HAVING clauses). These attributes can sometimes be accessed from below in
+ * relation to the relevant operator - the attributes are not explicitly mentioned in a SELECT
+ * clause, but SQL language rules still allow referencing them. Not all hidden attributes can
+ * be referenced if we are dealing with an [[Aggregate]] - only those that are part of grouping
+ * expressions, or if we are resolving a name under an aggregate function (if
+ * [[canReferenceAggregatedAccessOnlyAttributes]] is specified).
+ * 4. Infer the right resolution priority depending on [[canResolveNameByHiddenOutput]] and
+ * [[shouldPreferHiddenOutput]] flag values. These flags are set depending on the operator
+ * in which context we are currently resolving the [[multipartName]]. For example, ORDER BY
+ * clause prefers attributes from SELECT list over lower attributes from the table, but HAVING
+ * clause has the opposite rules.
+ * 5. Pick the best suitable candidates using [[pickSuitableCandidates]]. We prioritize candidates
+ * that have exactly 1 match for the [[multipartName]], because other options would fail.
+ * If there was a single match, we return [[ResolvedMultipartName]] with that attribute, and
+ * multipart name resolution process succeeds. If none of the options are suitable, we fall
+ * back to the main output and either return [[ResolvedMultipartName]] with multiple candidates
+ * from that main output to throw a descriptive [[AMBIGUOUS_REFERENCE]] error later or return
+ * [[None]] to continue the name resolution process using other sources.
+ *
+ * This algorithm is incomplete and completely covers just the SQL scenarios. DataFrame
+ * programs can prioritize several layers of [[Project]] outputs if several nested
+ * `.select(...)` calls have conflicting attributes.
*/
private def tryResolveMultipartNameByOutput(
multipartName: Seq[String],
nameComparator: NameComparator,
canResolveNameByHiddenOutput: Boolean,
- shouldPreferTableColumnsOverAliases: Boolean,
shouldPreferHiddenOutput: Boolean,
canReferenceAggregatedAccessOnlyAttributes: Boolean): Option[ResolvedMultipartName] = {
- val (candidates, nestedFields) =
- attributesForResolution.getCandidatesForResolution(multipartName, nameComparator)
-
- val hiddenOutputCandidates = candidates.filter { element =>
- !outputById.containsKey(element.exprId) &&
- (canReferenceAggregatedAccessOnlyAttributes || !element.aggregatedAccessOnly)
- }
+ val (candidates, nestedFields) = getCandidatesForResolution(multipartName)
- val (currentCandidates: Seq[Attribute], resolutionType: String) =
- if (shouldPreferHiddenOutput && hiddenOutputCandidates.nonEmpty) {
- (hiddenOutputCandidates, "hidden")
- } else {
- val outputCandidates = candidates.filter { element =>
- outputById.containsKey(element.exprId)
- }
+ val mainOutputCandidates = getMainOutputCandidates(candidates)
+ val metadataOutputCandidates = getMetadataOutputCandidates(candidates)
- if (outputCandidates.nonEmpty) {
- (outputCandidates, "normal")
- } else {
- val metadataOutputCandidates =
- candidates.filter { element =>
- !outputById.containsKey(element.exprId) && element.qualifiedAccessOnly
- }
+ val resolutionOrder = if (canResolveNameByHiddenOutput) {
+ val hiddenOutputCandidates =
+ getHiddenOutputCandidates(candidates, canReferenceAggregatedAccessOnlyAttributes)
- if (metadataOutputCandidates.nonEmpty) {
- (metadataOutputCandidates, "metadata")
- } else {
- if (canResolveNameByHiddenOutput &&
- !shouldPreferHiddenOutput &&
- hiddenOutputCandidates.nonEmpty) {
- (hiddenOutputCandidates, "hidden")
- } else {
- (Seq.empty, "")
- }
- }
- }
+ if (shouldPreferHiddenOutput) {
+ Seq(
+ CandidatesForResolution(hiddenOutputCandidates, OutputType.Hidden),
+ CandidatesForResolution(mainOutputCandidates, OutputType.Main),
+ CandidatesForResolution(metadataOutputCandidates, OutputType.Metadata)
+ )
+ } else {
+ Seq(
+ CandidatesForResolution(mainOutputCandidates, OutputType.Main),
+ CandidatesForResolution(metadataOutputCandidates, OutputType.Metadata),
+ CandidatesForResolution(hiddenOutputCandidates, OutputType.Hidden)
+ )
}
+ } else {
+ Seq(
+ CandidatesForResolution(mainOutputCandidates, OutputType.Main),
+ CandidatesForResolution(metadataOutputCandidates, OutputType.Metadata)
+ )
+ }
+
+ val suitableCandidates = pickSuitableCandidates(
+ resolutionOrder = resolutionOrder,
+ fallbackCandidates = CandidatesForResolution(mainOutputCandidates, OutputType.Main)
+ )
val resolvedCandidates = attributesForResolution.resolveCandidates(
multipartName,
nameComparator,
- currentCandidates,
+ suitableCandidates.attributes,
nestedFields
)
if (resolvedCandidates.nonEmpty) {
- val candidatesWithPreferredColumnsOverAliases = if (shouldPreferTableColumnsOverAliases) {
- val (aliasCandidates, nonAliasCandidates) =
- resolvedCandidates.partition(candidate => availableAliases.contains(candidate.exprId))
-
- if (nonAliasCandidates.nonEmpty) {
- nonAliasCandidates
- } else {
- aliasCandidates
- }
- } else {
- resolvedCandidates
- }
-
planLogger.logNameResolutionEvent(
multipartName,
- candidatesWithPreferredColumnsOverAliases,
- s"From $resolutionType output"
+ resolvedCandidates,
+ s"From ${suitableCandidates.outputType} output"
)
Some(
ResolvedMultipartName(
- candidates = candidatesWithPreferredColumnsOverAliases,
+ candidates = resolvedCandidates,
referencedAttribute = None
)
)
@@ -687,6 +690,54 @@ class NameScope(
}
}
+ private def getCandidatesForResolution(
+ multipartName: Seq[String]): (Seq[Attribute], Seq[String]) = {
+ val (candidates, nestedFields) =
+ attributesForResolution.getCandidatesForResolution(multipartName, nameComparator)
+
+ val filteredCandidates = if (nestedFields.nonEmpty) {
+ candidates.filter { attribute =>
+ ExtractValue.isExtractable(attribute, nestedFields, nameComparator)
+ }
+ } else {
+ candidates
+ }
+
+ (filteredCandidates, nestedFields)
+ }
+
+ private def getMainOutputCandidates(candidates: Seq[Attribute]): Seq[Attribute] = {
+ candidates.filter { attribute =>
+ outputById.containsKey(attribute.exprId)
+ }
+ }
+
+ private def getMetadataOutputCandidates(candidates: Seq[Attribute]): Seq[Attribute] = {
+ candidates.filter { element =>
+ !outputById.containsKey(element.exprId) && element.qualifiedAccessOnly
+ }
+ }
+
+ private def getHiddenOutputCandidates(
+ candidates: Seq[Attribute],
+ canReferenceAggregatedAccessOnlyAttributes: Boolean): Seq[Attribute] = {
+ candidates.filter { attribute =>
+ !availableAliases.contains(attribute.exprId) &&
+ (canReferenceAggregatedAccessOnlyAttributes || !attribute.aggregatedAccessOnly)
+ }
+ }
+
+ private def pickSuitableCandidates(
+ resolutionOrder: Seq[CandidatesForResolution],
+ fallbackCandidates: CandidatesForResolution): CandidatesForResolution = {
+ resolutionOrder
+ .collectFirst {
+ case candidates if candidates.attributes.size == 1 =>
+ candidates
+ }
+ .getOrElse(fallbackCandidates)
+ }
+
private def tryResolveMultipartNameAsLiteralFunction(
multipartName: Seq[String]): Option[ResolvedMultipartName] = {
val literalFunction = LiteralFunctionResolution.resolve(multipartName).toSeq
@@ -841,8 +892,8 @@ class NameScopeStack(planLogger: PlanLogger = new PlanLogger) extends SQLConfHel
/**
* Completely overwrite the current scope state with operator `output`, `hiddenOutput`,
- * `availableAliases`, `aggregateListAliases` and `hasLcaInAggregate`. If `hiddenOutput`,
- * `availableAliases` or `hasLcaInAggregate` are not provided, preserve the previous values.
+ * `availableAliases`, `aggregateListAliases` and `baseAggregate`. If `hiddenOutput`,
+ * `availableAliases` or `baseAggregate` are not provided, preserve the previous values.
* Additionally, update nullabilities of attributes in hidden output from new output, so that if
* attribute was nullable in either old hidden output or new output, it must stay nullable in new
* hidden output as well.
@@ -869,7 +920,7 @@ class NameScopeStack(planLogger: PlanLogger = new PlanLogger) extends SQLConfHel
hiddenOutput: Option[Seq[Attribute]] = None,
availableAliases: Option[HashSet[ExprId]] = None,
aggregateListAliases: Seq[Alias] = Seq.empty,
- hasLcaInAggregate: Boolean = false): Unit = {
+ baseAggregate: Option[Aggregate] = None): Unit = {
val hiddenOutputWithUpdatedNullabilities = updateHiddenOutputProperties(
output.getOrElse(stack.peek().output),
hiddenOutput.getOrElse(stack.peek().hiddenOutput)
@@ -880,15 +931,15 @@ class NameScopeStack(planLogger: PlanLogger = new PlanLogger) extends SQLConfHel
hiddenOutput = Some(hiddenOutputWithUpdatedNullabilities),
availableAliases = availableAliases,
aggregateListAliases = aggregateListAliases,
- hasLcaInAggregate = hasLcaInAggregate
+ baseAggregate = baseAggregate
)
stack.push(newScope)
}
/**
- * Overwrites `output`, `groupingAttributeIds` and `aggregateListAliases` of the current
- * [[NameScope]] entry and:
+ * Overwrites `output`, `groupingAttributeIds`, `aggregateListAliases` and `baseAggregate` of the
+ * current [[NameScope]] entry and:
* 1. extends hidden output with the provided output (only attributes that are not in the hidden
* output are added). This is done because resolution of arguments can be done through certain
* operators by hidden output. This use case is specific to Dataframe programs. Example:
@@ -916,7 +967,8 @@ class NameScopeStack(planLogger: PlanLogger = new PlanLogger) extends SQLConfHel
def overwriteOutputAndExtendHiddenOutput(
output: Seq[Attribute],
groupingAttributeIds: Option[HashSet[ExprId]] = None,
- aggregateListAliases: Seq[Alias] = Seq.empty): Unit = {
+ aggregateListAliases: Seq[Alias] = Seq.empty,
+ baseAggregate: Option[Aggregate] = None): Unit = {
val prevScope = stack.pop
val hiddenOutputWithUpdatedProperties: Seq[Attribute] = updateHiddenOutputProperties(
@@ -932,7 +984,8 @@ class NameScopeStack(planLogger: PlanLogger = new PlanLogger) extends SQLConfHel
val newScope = prevScope.overwrite(
output = Some(output),
hiddenOutput = Some(hiddenOutput),
- aggregateListAliases = aggregateListAliases
+ aggregateListAliases = aggregateListAliases,
+ baseAggregate = baseAggregate
)
stack.push(newScope)
@@ -989,11 +1042,11 @@ class NameScopeStack(planLogger: PlanLogger = new PlanLogger) extends SQLConfHel
/**
* After finishing the resolution after [[pushScope]], the caller needs to call [[popScope]] to
- * clear the stack. We propagate `hiddenOutput`, `availableAliases` and `hasLcaInAggregate`
- * upwards because of name resolution by overwriting their current values with the popped ones.
- * This is not done in case [[pushScope]] and [[popScope]] were called in the context of subquery
- * resolution (which is indicated by `isSubqueryRoot` flag), because we don't want to overwrite
- * the existing `hiddenOutput` of the main plan.
+ * clear the stack. We propagate `hiddenOutput`, `availableAliases` upwards because of name
+ * resolution by overwriting their current values with the popped ones. This is not done in case
+ * [[pushScope]] and [[popScope]] were called in the context of subquery resolution (which is
+ * indicated by `isSubqueryRoot` flag), because we don't want to overwrite the existing
+ * `hiddenOutput` of the main plan.
*/
def popScope(): Unit = {
val childScope = stack.pop()
@@ -1002,8 +1055,7 @@ class NameScopeStack(planLogger: PlanLogger = new PlanLogger) extends SQLConfHel
stack.push(
currentScope.overwrite(
hiddenOutput = Some(childScope.hiddenOutput),
- availableAliases = Some(childScope.availableAliases),
- hasLcaInAggregate = childScope.hasLcaInAggregate
+ availableAliases = Some(childScope.availableAliases)
)
)
}
@@ -1082,7 +1134,6 @@ class NameScopeStack(planLogger: PlanLogger = new PlanLogger) extends SQLConfHel
canLaterallyReferenceColumn = canLaterallyReferenceColumn,
canReferenceAggregateExpressionAliases = canReferenceAggregateExpressionAliases,
canResolveNameByHiddenOutput = canResolveNameByHiddenOutput,
- shouldPreferTableColumnsOverAliases = shouldPreferTableColumnsOverAliases,
shouldPreferHiddenOutput = shouldPreferHiddenOutput,
canReferenceAggregatedAccessOnlyAttributes = canReferenceAggregatedAccessOnlyAttributes
)
@@ -1103,7 +1154,9 @@ class NameScopeStack(planLogger: PlanLogger = new PlanLogger) extends SQLConfHel
if (nameTarget.candidates.nonEmpty) {
nameTarget.copy(
isOuterReference = true,
- candidates = nameTarget.candidates.map(wrapCandidateInOuterReference)
+ candidates = nameTarget.candidates.map { candidate =>
+ wrapCandidateInOuterReference(candidate, outer)
+ }
)
} else {
nameTargetFromCurrentScope
@@ -1136,18 +1189,60 @@ class NameScopeStack(planLogger: PlanLogger = new PlanLogger) extends SQLConfHel
/**
* Wrap candidate in [[OuterReference]]. If the root is not an [[Attribute]], but an
* [[ExtractValue]] (struct/map/array field reference) we find the actual [[Attribute]] and wrap
- * it in [[OuterReference]].
+ * it in [[OuterReference]]. In case found [[Attribute]] is aliased in the outer scope, we
+ * replace it with an [[Attribute]] created from the [[Alias]].
*/
- private def wrapCandidateInOuterReference(candidate: Expression): Expression = candidate match {
- case candidate: Attribute =>
- OuterReference(candidate)
- case extractValue: ExtractValue =>
- extractValue.transformUp {
- case attribute: Attribute => OuterReference(attribute)
- case other => other
+ private def wrapCandidateInOuterReference(
+ candidate: Expression,
+ outerScope: NameScope): Expression = {
+ candidate match {
+ case extractValue: ExtractValue =>
+ extractValue.transformUp {
+ case attribute: Attribute =>
+ tryReplaceOuterReferenceAttributeWithAlias(attribute, outerScope)
+ case other => other
+ }
+ case attribute: Attribute =>
+ tryReplaceOuterReferenceAttributeWithAlias(attribute, outerScope)
+ case other => other
+ }
+ }
+
+ /**
+ * Try to replace an [[Attribute]] with an [[Attribute]] created out of the [[Alias]] from the
+ * outer scope. For example:
+ *
+ * {{{ SELECT col1 AS alias FROM VALUES('a') GROUP BY col1 HAVING (SELECT col1 = 'a'); }}}
+ *
+ * Plan should be:
+ *
+ * {{{
+ * Filter cast(scalar-subquery#2 [alias#1] as boolean)
+ * +- Project [(outer(alias#1) = a) AS (outer(col1) = a)#3]
+ * +- OneRowRelation
+ * +- Aggregate [col1#0], [col1#0 AS alias#1]
+ * +- LocalRelation [col1#0]
+ * }}}
+ *
+ * As it can be seen, we replace `outer(col1)` with `outer(alias)` but keep the original
+ * [[Attribute]] in the name (to be compatible with the fixed-point implementation).
+ */
+ private def tryReplaceOuterReferenceAttributeWithAlias(
+ attribute: Attribute,
+ outerScope: NameScope): OuterReference = {
+ val replacedAttribute = outerScope.aggregateListAliases
+ .collectFirst {
+ case alias if alias.child.semanticEquals(attribute) => alias.toAttribute
}
- case _ =>
- candidate
+ .getOrElse(attribute)
+
+ val outerReference = OuterReference(replacedAttribute)
+ outerReference.setTagValue(
+ OuterReference.SINGLE_PASS_SQL_STRING_OVERRIDE,
+ toPrettySQL(OuterReference(attribute))
+ )
+
+ outerReference
}
/**
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/OperatorWithUncomparableTypeValidator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/OperatorWithUncomparableTypeValidator.scala
new file mode 100644
index 0000000000000..6afa9e379297d
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/OperatorWithUncomparableTypeValidator.scala
@@ -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.
+ */
+
+package org.apache.spark.sql.catalyst.analysis.resolver
+
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.plans.logical.{Distinct, LogicalPlan, SetOperation}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.{DataType, MapType, VariantType}
+
+/**
+ * [[OperatorWithUncomparableTypeValidator]] performs the validation of a logical plan to ensure
+ * that it (if it is [[Distinct]] or [[SetOperation]]) does not contain any uncomparable types:
+ * [[VariantType]], [[MapType]], [[GeometryType]] or [[GeographyType]].
+ */
+object OperatorWithUncomparableTypeValidator {
+
+ /**
+ * Validates that the provided logical plan does not contain any uncomparable types:
+ * [[VariantType]], [[MapType]], [[GeometryType]] or [[GeographyType]] (throws a specific
+ * user-facing error if it does). Operators that are not supported are [[Distinct]] and
+ * [[SetOperation]] ([[Union]], [[Except]], [[Intersect]]).
+ */
+ def validate(operator: LogicalPlan, output: Seq[Attribute]): Unit = {
+ operator match {
+ case unsupportedOperator @ (_: SetOperation | _: Distinct) =>
+
+ output.foreach { element =>
+ if (hasMapType(element.dataType)) {
+ throwUnsupportedSetOperationOnMapType(element, unsupportedOperator)
+ }
+
+ if (hasVariantType(element.dataType)) {
+ throwUnsupportedSetOperationOnVariantType(element, unsupportedOperator)
+ }
+ }
+ case _ =>
+ }
+ }
+
+ private def hasMapType(dt: DataType): Boolean = {
+ dt.existsRecursively(_.isInstanceOf[MapType])
+ }
+
+ private def hasVariantType(dt: DataType): Boolean = {
+ dt.existsRecursively(_.isInstanceOf[VariantType])
+ }
+
+ private def throwUnsupportedSetOperationOnMapType(
+ mapCol: Attribute,
+ unresolvedPlan: LogicalPlan): Unit = {
+ throw QueryCompilationErrors.unsupportedSetOperationOnMapType(
+ mapCol = mapCol,
+ origin = unresolvedPlan.origin
+ )
+ }
+
+ private def throwUnsupportedSetOperationOnVariantType(
+ variantCol: Attribute,
+ unresolvedPlan: LogicalPlan): Unit = {
+ throw QueryCompilationErrors.unsupportedSetOperationOnVariantType(
+ variantCol = variantCol,
+ origin = unresolvedPlan.origin
+ )
+ }
+}
diff --git a/connect-examples/server-library-example/common/src/main/protobuf/relations.proto b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/OutputType.scala
similarity index 73%
rename from connect-examples/server-library-example/common/src/main/protobuf/relations.proto
rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/OutputType.scala
index 1ebf0e640bef1..04152870e0014 100644
--- a/connect-examples/server-library-example/common/src/main/protobuf/relations.proto
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/OutputType.scala
@@ -15,20 +15,14 @@
* limitations under the License.
*/
-syntax = 'proto3';
+package org.apache.spark.sql.catalyst.analysis.resolver
-option java_multiple_files = true;
-option java_package = "org.apache.connect.examples.serverlibrary.proto";
-
-import "base.proto";
-
-message CustomRelation {
- oneof relation_type {
- Scan scan = 1;
- }
-}
+/**
+ * [[OutputType]] represents different types of output used during multipart name resolution in the
+ * [[NameScope]].
+ */
+object OutputType extends Enumeration {
+ type OutputType = Value
-message Scan {
- // (Required) Table to scan.
- CustomTable table = 1;
+ val Main, Hidden, Metadata = Value
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PlanLogger.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PlanLogger.scala
index 3c9fb4bea4913..d1cd7adbe994a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PlanLogger.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PlanLogger.scala
@@ -17,7 +17,7 @@
package org.apache.spark.sql.catalyst.analysis.resolver
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{MESSAGE, QUERY_PLAN}
import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PlanRewriter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PlanRewriter.scala
index 73a83fd8c3eec..544d8180fa1fe 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PlanRewriter.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PlanRewriter.scala
@@ -59,7 +59,7 @@ class PlanRewriter(
val planWithRewrittenSubqueries =
plan.transformAllExpressionsWithPruning(_.containsPattern(PLAN_EXPRESSION)) {
case subqueryExpression: SubqueryExpression =>
- val rewrittenSubqueryPlan = rewrite(subqueryExpression.plan)
+ val rewrittenSubqueryPlan = doRewriteWithSubqueries(subqueryExpression.plan)
subqueryExpression.withNewPlan(rewrittenSubqueryPlan)
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ProducesUnresolvedSubtree.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ProducesUnresolvedSubtree.scala
index 576cb98bbabe8..61f5457b42688 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ProducesUnresolvedSubtree.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ProducesUnresolvedSubtree.scala
@@ -25,8 +25,8 @@ import org.apache.spark.sql.catalyst.expressions.Expression
* called produces partially-unresolved subtrees. In order to resolve the subtree a callback
* resolver is called recursively. This callback must ensure that no node is resolved twice in
* order to not break the single-pass invariant. This is done by tagging the limits of this
- * traversal with [[ExpressionResolver.SINGLE_PASS_SUBTREE_BOUNDARY]] tag. This tag is applied to
- * the original expression's children, which are guaranteed to be resolved at the time of given
+ * traversal with [[ResolverTag.SINGLE_PASS_SUBTREE_BOUNDARY]] tag. This tag is applied to the
+ * original expression's children, which are guaranteed to be resolved at the time of given
* expression's resolution. When callback resolver encounters the node that is tagged, it should
* return identity instead of trying to resolve it.
*/
@@ -40,20 +40,20 @@ trait ProducesUnresolvedSubtree extends ResolvesExpressionChildren {
*
* If the result of the callback is the same object as the source `expression`, we don't perform
* the downwards traversal. This is both more optimal and a fail-safe mechanism in case we
- * accidentally lose the [[ExpressionResolver.SINGLE_PASS_SUBTREE_BOUNDARY]] tag.
+ * accidentally lose the [[ResolverTag.SINGLE_PASS_SUBTREE_BOUNDARY]] tag.
*/
protected def withResolvedSubtree(
expression: Expression,
expressionResolver: Expression => Expression)(body: => Expression): Expression = {
expression.children.foreach { child =>
- child.setTagValue(ExpressionResolver.SINGLE_PASS_SUBTREE_BOUNDARY, ())
+ child.setTagValue(ResolverTag.SINGLE_PASS_SUBTREE_BOUNDARY, ())
}
val resultExpression = body
if (resultExpression.eq(expression)) {
expression.children.foreach { child =>
- child.unsetTagValue(ExpressionResolver.SINGLE_PASS_SUBTREE_BOUNDARY)
+ child.unsetTagValue(ResolverTag.SINGLE_PASS_SUBTREE_BOUNDARY)
}
resultExpression
} else {
@@ -68,9 +68,9 @@ trait ProducesUnresolvedSubtree extends ResolvesExpressionChildren {
*/
protected def tryPopSinglePassSubtreeBoundary(unresolvedExpression: Expression): Boolean = {
if (unresolvedExpression
- .getTagValue(ExpressionResolver.SINGLE_PASS_SUBTREE_BOUNDARY)
+ .getTagValue(ResolverTag.SINGLE_PASS_SUBTREE_BOUNDARY)
.isDefined) {
- unresolvedExpression.unsetTagValue(ExpressionResolver.SINGLE_PASS_SUBTREE_BOUNDARY)
+ unresolvedExpression.unsetTagValue(ResolverTag.SINGLE_PASS_SUBTREE_BOUNDARY)
true
} else {
false
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ProjectResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ProjectResolver.scala
index 076ba6019d786..9e6522968b5b4 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ProjectResolver.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ProjectResolver.scala
@@ -60,12 +60,19 @@ class ProjectResolver(operatorResolver: Resolver, expressionResolver: Expression
*
* After the subtree and project-list expressions are resolved in the child scope we overwrite
* current scope with resolved operators output to expose new names to the parent operators.
+ *
+ * We need to clear [[NameScope.availableAliases]]. Those are only relevant for the immediate
+ * project list for output prioritization to work correctly in
+ * [[NameScope.tryResolveMultipartNameByOutput]].
*/
override def resolve(unresolvedProject: Project): LogicalPlan = {
scopes.pushScope()
val (resolvedOperator, resolvedProjectList) = try {
val resolvedChild = operatorResolver.resolve(unresolvedProject.child)
+
+ scopes.current.availableAliases.clear()
+
val childReferencedAttributes = expressionResolver.getLastReferencedAttributes
val resolvedProjectList =
expressionResolver.resolveProjectList(unresolvedProject.projectList, unresolvedProject)
@@ -87,7 +94,8 @@ class ProjectResolver(operatorResolver: Resolver, expressionResolver: Expression
expressions = aggregateWithLcaResolutionResult.outputList,
hasAggregateExpressions = false,
hasLateralColumnAlias = false,
- aggregateListAliases = aggregateWithLcaResolutionResult.aggregateListAliases
+ aggregateListAliases = aggregateWithLcaResolutionResult.aggregateListAliases,
+ baseAggregate = Some(aggregateWithLcaResolutionResult.baseAggregate)
)
(aggregateWithLcaResolutionResult.resolvedOperator, projectList)
} else {
@@ -95,8 +103,10 @@ class ProjectResolver(operatorResolver: Resolver, expressionResolver: Expression
// single-pass Analyzer.
ExprUtils.assertValidAggregation(aggregate)
- val resolvedAggregateList =
- resolvedProjectList.copy(aggregateListAliases = scopes.current.aggregateListAliases)
+ val resolvedAggregateList = resolvedProjectList.copy(
+ aggregateListAliases = scopes.current.aggregateListAliases,
+ baseAggregate = Some(aggregate)
+ )
(aggregate, resolvedAggregateList)
}
@@ -119,7 +129,8 @@ class ProjectResolver(operatorResolver: Resolver, expressionResolver: Expression
scopes.overwriteOutputAndExtendHiddenOutput(
output = resolvedProjectList.expressions.map(namedExpression => namedExpression.toAttribute),
- aggregateListAliases = resolvedProjectList.aggregateListAliases
+ aggregateListAliases = resolvedProjectList.aggregateListAliases,
+ baseAggregate = resolvedProjectList.baseAggregate
)
resolvedOperator
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolutionCheckRunner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolutionCheckRunner.scala
new file mode 100644
index 0000000000000..79a3f9cf1707c
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolutionCheckRunner.scala
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.analysis.resolver
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.SubqueryExpression
+import org.apache.spark.sql.catalyst.plans.logical.{AnalysisHelper, LogicalPlan}
+import org.apache.spark.sql.catalyst.trees.TreePattern.PLAN_EXPRESSION
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * The [[ResolutionCheckRunner]] is used to run `resolutionChecks` on the logical plan.
+ *
+ * Important note: these checks are not always idempotent, and sometimes perform heavy network
+ * operations.
+ */
+class ResolutionCheckRunner(resolutionChecks: Seq[LogicalPlan => Unit]) extends SQLConfHelper {
+
+ /**
+ * Runs the resolution checks on `plan`. Invokes all the checks for every subquery plan, and
+ * eventually for the main query plan.
+ */
+ def runWithSubqueries(plan: LogicalPlan): Unit = {
+ if (conf.getConf(SQLConf.ANALYZER_SINGLE_PASS_RESOLVER_RUN_EXTENDED_RESOLUTION_CHECKS)) {
+ AnalysisHelper.allowInvokingTransformsInAnalyzer {
+ doRunWithSubqueries(plan)
+ }
+ }
+ }
+
+ private def doRunWithSubqueries(plan: LogicalPlan): Unit = {
+ val planWithRewrittenSubqueries =
+ plan.transformAllExpressionsWithPruning(_.containsPattern(PLAN_EXPRESSION)) {
+ case subqueryExpression: SubqueryExpression =>
+ doRunWithSubqueries(subqueryExpression.plan)
+
+ subqueryExpression
+ }
+
+ run(planWithRewrittenSubqueries)
+ }
+
+ private def run(plan: LogicalPlan): Unit = {
+ for (check <- resolutionChecks) {
+ check(plan)
+ }
+ }
+}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolutionValidator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolutionValidator.scala
index b621e396a8839..326ee8463a79f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolutionValidator.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolutionValidator.scala
@@ -101,6 +101,8 @@ class ResolutionValidator {
validateJoin(join)
case repartition: Repartition =>
validateRepartition(repartition)
+ case sample: Sample =>
+ validateSample(sample)
// [[LogicalRelation]], [[HiveTableRelation]] and other specific relations can't be imported
// because of a potential circular dependency, so we match a generic Catalyst
// [[MultiInstanceRelation]] instead.
@@ -269,6 +271,10 @@ class ResolutionValidator {
validate(repartition.child)
}
+ private def validateSample(sample: Sample): Unit = {
+ validate(sample.child)
+ }
+
private def validateJoin(join: Join) = {
attributeScopeStack.pushScope()
try {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolvedProjectList.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolvedProjectList.scala
index 6c436d4176d22..9bbce99bb1130 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolvedProjectList.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolvedProjectList.scala
@@ -18,6 +18,7 @@
package org.apache.spark.sql.catalyst.analysis.resolver
import org.apache.spark.sql.catalyst.expressions.{Alias, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.Aggregate
/**
* Structure used to return results of the resolved project list.
@@ -28,9 +29,12 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, NamedExpression}
* - hasLateralColumnAlias: True if the resolved project list contains any lateral column aliases.
* - aggregateListAliases: List of aliases in aggregate list if there are aggregate expressions in
* the [[Project]].
+ * - baseAggregate: Base [[Aggregate]] node constructed by [[LateralColumnAliasResolver]] while
+ * resolving lateral column references in [[Aggregate]].
*/
case class ResolvedProjectList(
expressions: Seq[NamedExpression],
hasAggregateExpressions: Boolean,
hasLateralColumnAlias: Boolean,
- aggregateListAliases: Seq[Alias])
+ aggregateListAliases: Seq[Alias],
+ baseAggregate: Option[Aggregate] = None)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/Resolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/Resolver.scala
index d9d698b1fecac..75d23f29ecfc5 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/Resolver.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/Resolver.scala
@@ -21,7 +21,6 @@ import java.util.HashSet
import org.apache.spark.SparkException
import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.catalyst.EvaluateUnresolvedInlineTable
import org.apache.spark.sql.catalyst.analysis.{
withPosition,
AnalysisErrorAt,
@@ -43,7 +42,7 @@ import org.apache.spark.sql.catalyst.expressions.{
}
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.trees.CurrentOrigin
-import org.apache.spark.sql.catalyst.trees.TreeNodeTag
+import org.apache.spark.sql.catalyst.util.EvaluateUnresolvedInlineTable
import org.apache.spark.sql.connector.catalog.CatalogManager
import org.apache.spark.sql.errors.QueryCompilationErrors
@@ -112,7 +111,6 @@ class Resolver(
private var relationMetadataProvider: RelationMetadataProvider = new MetadataResolver(
catalogManager,
relationResolution,
- functionResolution,
metadataResolverExtensions
)
@@ -173,7 +171,7 @@ class Resolver(
planLogger.logPlanResolutionEvent(planAfterSubstitution, "Main resolution")
- planAfterSubstitution.setTagValue(Resolver.TOP_LEVEL_OPERATOR, ())
+ planAfterSubstitution.setTagValue(ResolverTag.TOP_LEVEL_OPERATOR, ())
resolve(planAfterSubstitution)
}
@@ -257,6 +255,8 @@ class Resolver(
resolveSupervisingCommand(supervisingCommand)
case repartition: Repartition =>
resolveRepartition(repartition)
+ case sample: Sample =>
+ resolveSample(sample)
case _ =>
tryDelegateResolutionToExtension(unresolvedPlan).getOrElse {
handleUnmatchedOperator(unresolvedPlan)
@@ -476,12 +476,18 @@ class Resolver(
/**
* [[Distinct]] operator doesn't require any special resolution.
+ * We validate results of the resolution using the [[OperatorWithUncomparableTypeValidator]]
+ * ([[MapType]], [[VariantType]], [[GeometryType]] and [[GeographyType]] are not supported
+ * under [[Distinct]] operator).
*
* `hiddenOutput` and `availableAliases` are reset when [[Distinct]] is reached during tree
* traversal.
*/
private def resolveDistinct(unresolvedDistinct: Distinct): LogicalPlan = {
val resolvedDistinct = unresolvedDistinct.copy(child = resolve(unresolvedDistinct.child))
+
+ OperatorWithUncomparableTypeValidator.validate(resolvedDistinct, scopes.current.output)
+
scopes.overwriteCurrent(
hiddenOutput = Some(scopes.current.output),
availableAliases = Some(new HashSet[ExprId])
@@ -660,6 +666,14 @@ class Resolver(
repartition.copy(child = resolve(repartition.child))
}
+ /**
+ * Resolve [[Sample]] operator. Its resolution doesn't require any specific logic (besides
+ * child resolution).
+ */
+ private def resolveSample(sample: Sample): LogicalPlan = {
+ sample.copy(child = resolve(sample.child))
+ }
+
private def createCteRelationRef(name: String, cteRelationDef: CTERelationDef): LogicalPlan = {
SubqueryAlias(
identifier = name,
@@ -771,11 +785,6 @@ class Resolver(
object Resolver {
- /**
- * Marks the operator as the top-most operator in a query or a view.
- */
- val TOP_LEVEL_OPERATOR = TreeNodeTag[Unit]("top_level_operator")
-
/**
* Create a new instance of the [[RelationResolution]].
*/
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolverGuard.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolverGuard.scala
index 75215524d2144..5b28d5369e387 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolverGuard.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolverGuard.scala
@@ -19,7 +19,11 @@ package org.apache.spark.sql.catalyst.analysis.resolver
import java.util.Locale
-import org.apache.spark.sql.catalyst.{FunctionIdentifier, SQLConfHelper, SqlScriptingContextManager}
+import org.apache.spark.sql.catalyst.{
+ FunctionIdentifier,
+ SQLConfHelper,
+ SqlScriptingContextManager
+}
import org.apache.spark.sql.catalyst.analysis.{
FunctionRegistry,
GetViewColumnByNameAndOrdinal,
@@ -143,6 +147,8 @@ class ResolverGuard(catalogManager: CatalogManager) extends SQLConfHelper {
checkRepartition(repartition)
case having: UnresolvedHaving =>
checkHaving(having)
+ case sample: Sample =>
+ checkSample(sample)
case _ =>
false
}
@@ -168,8 +174,6 @@ class ResolverGuard(catalogManager: CatalogManager) extends SQLConfHelper {
checkUnresolvedCast(unresolvedCast)
case unresolvedUpCast: UpCast =>
checkUnresolvedUpCast(unresolvedUpCast)
- case unresolvedStar: UnresolvedStar =>
- checkUnresolvedStar(unresolvedStar)
case unresolvedAlias: UnresolvedAlias =>
checkUnresolvedAlias(unresolvedAlias)
case unresolvedAttribute: UnresolvedAttribute =>
@@ -194,6 +198,8 @@ class ResolverGuard(catalogManager: CatalogManager) extends SQLConfHelper {
checkUnresolvedFunction(unresolvedFunction)
case getViewColumnByNameAndOrdinal: GetViewColumnByNameAndOrdinal =>
checkGetViewColumnBynameAndOrdinal(getViewColumnByNameAndOrdinal)
+ case semiStructuredExtract: SemiStructuredExtract =>
+ checkSemiStructuredExtract(semiStructuredExtract)
case expression if isGenerallySupportedExpression(expression) =>
expression.children.forall(checkExpression)
case _ =>
@@ -219,13 +225,23 @@ class ResolverGuard(catalogManager: CatalogManager) extends SQLConfHelper {
}
private def checkProject(project: Project) = {
- checkOperator(project.child) && project.projectList.forall(checkExpression)
+ checkOperator(project.child) && project.projectList.forall {
+ case _: UnresolvedStar =>
+ true
+ case other =>
+ checkExpression(other)
+ }
}
private def checkAggregate(aggregate: Aggregate) = {
checkOperator(aggregate.child) &&
aggregate.groupingExpressions.forall(checkExpression) &&
- aggregate.aggregateExpressions.forall(checkExpression)
+ aggregate.aggregateExpressions.forall {
+ case _: UnresolvedStar =>
+ true
+ case other =>
+ checkExpression(other)
+ }
}
private def checkJoin(join: Join) = {
@@ -267,7 +283,8 @@ class ResolverGuard(catalogManager: CatalogManager) extends SQLConfHelper {
private def checkUnresolvedInlineTable(unresolvedInlineTable: UnresolvedInlineTable) =
unresolvedInlineTable.rows.forall(_.forall(checkExpression))
- private def checkUnresolvedRelation(unresolvedRelation: UnresolvedRelation) = true
+ private def checkUnresolvedRelation(unresolvedRelation: UnresolvedRelation) =
+ !unresolvedRelation.isStreaming
private def checkResolvedInlineTable(resolvedInlineTable: ResolvedInlineTable) =
resolvedInlineTable.rows.forall(_.forall(checkExpression))
@@ -306,8 +323,6 @@ class ResolverGuard(catalogManager: CatalogManager) extends SQLConfHelper {
private def checkUnresolvedUpCast(upCast: UpCast) = checkExpression(upCast.child)
- private def checkUnresolvedStar(unresolvedStar: UnresolvedStar) = true
-
private def checkUnresolvedAlias(unresolvedAlias: UnresolvedAlias) =
checkExpression(unresolvedAlias.child)
@@ -331,6 +346,7 @@ class ResolverGuard(catalogManager: CatalogManager) extends SQLConfHelper {
}
private def checkUnresolvedFunction(unresolvedFunction: UnresolvedFunction) =
+ unresolvedFunction.nameParts.size == 1 &&
!ResolverGuard.UNSUPPORTED_FUNCTION_NAMES.contains(unresolvedFunction.nameParts.head) &&
// UDFs are not supported
FunctionRegistry.functionSet.contains(
@@ -358,6 +374,9 @@ class ResolverGuard(catalogManager: CatalogManager) extends SQLConfHelper {
private def checkGetViewColumnBynameAndOrdinal(
getViewColumnByNameAndOrdinal: GetViewColumnByNameAndOrdinal) = true
+ private def checkSemiStructuredExtract(semiStructuredExtract: SemiStructuredExtract) =
+ checkExpression(semiStructuredExtract.child)
+
private def checkRepartition(repartition: Repartition) = {
checkOperator(repartition.child)
}
@@ -365,6 +384,10 @@ class ResolverGuard(catalogManager: CatalogManager) extends SQLConfHelper {
private def checkHaving(having: UnresolvedHaving) =
checkExpression(having.havingCondition) && checkOperator(having.child)
+ private def checkSample(sample: Sample) = {
+ checkOperator(sample.child)
+ }
+
/**
* Most of the expressions come from resolving the [[UnresolvedFunction]], but here we have some
* popular expressions allowlist for two reasons:
@@ -417,8 +440,8 @@ class ResolverGuard(catalogManager: CatalogManager) extends SQLConfHelper {
_: RegExpCount | _: RegExpSubStr | _: RegExpInStr =>
true
// JSON
- case _: JsonToStructs | _: StructsToJson | _: SchemaOfJson | _: JsonObjectKeys |
- _: LengthOfJsonArray =>
+ case _: GetJsonObject | _: JsonTuple | _: JsonToStructs | _: StructsToJson |
+ _: SchemaOfJson | _: JsonObjectKeys | _: LengthOfJsonArray =>
true
// CSV
case _: SchemaOfCsv | _: StructsToCsv | _: CsvToStructs =>
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolverMetricTracker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolverMetricTracker.scala
index 680360836eb7d..0e92684dc2a45 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolverMetricTracker.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolverMetricTracker.scala
@@ -24,7 +24,6 @@ import org.apache.spark.sql.catalyst.rules.QueryExecutionMetering
* Trait for tracking and logging timing metrics for single-pass resolver.
*/
trait ResolverMetricTracker {
- private val profilerGroup: String = getClass.getSimpleName
/**
* Log top-level timing metrics for single-pass analyzer. In order to utilize existing logging
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolverRunner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolverRunner.scala
index 37d41919f1323..fa06d39f13ccb 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolverRunner.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolverRunner.scala
@@ -18,7 +18,11 @@
package org.apache.spark.sql.catalyst.analysis.resolver
import org.apache.spark.sql.catalyst.{QueryPlanningTracker, SQLConfHelper}
-import org.apache.spark.sql.catalyst.analysis.{AnalysisContext, CleanupAliases}
+import org.apache.spark.sql.catalyst.analysis.{
+ AnalysisContext,
+ CleanupAliases,
+ PullOutNondeterministic
+}
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.internal.SQLConf
@@ -41,7 +45,8 @@ class ResolverRunner(
*/
private val planRewriteRules: Seq[Rule[LogicalPlan]] = Seq(
PruneMetadataColumns,
- CleanupAliases
+ CleanupAliases,
+ PullOutNondeterministic
)
/**
@@ -50,6 +55,11 @@ class ResolverRunner(
*/
private val planRewriter = new PlanRewriter(planRewriteRules, extendedRewriteRules)
+ /**
+ * `resolutionCheckRunner` is used to run `extendedResolutionChecks` on the resolved plan.
+ */
+ private val resolutionCheckRunner = new ResolutionCheckRunner(extendedResolutionChecks)
+
/**
* Entry point for the resolver. This method performs following 4 steps:
* - Resolves the plan in a bottom-up using [[Resolver]], single-pass manner.
@@ -69,7 +79,7 @@ class ResolverRunner(
runValidator(rewrittenPlan)
- runExtendedResolutionChecks(rewrittenPlan)
+ resolutionCheckRunner.runWithSubqueries(rewrittenPlan)
rewrittenPlan
}
@@ -82,12 +92,4 @@ class ResolverRunner(
validator.validatePlan(plan)
}
}
-
- private def runExtendedResolutionChecks(plan: LogicalPlan): Unit = {
- if (conf.getConf(SQLConf.ANALYZER_SINGLE_PASS_RESOLVER_RUN_EXTENDED_RESOLUTION_CHECKS)) {
- for (check <- extendedResolutionChecks) {
- check(plan)
- }
- }
- }
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PullOutNondeterministicExpressionInExpressionTree.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolverTag.scala
similarity index 50%
rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PullOutNondeterministicExpressionInExpressionTree.scala
rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolverTag.scala
index 3272c6975075c..6de583e1a1c12 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PullOutNondeterministicExpressionInExpressionTree.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolverTag.scala
@@ -17,28 +17,34 @@
package org.apache.spark.sql.catalyst.analysis.resolver
-import java.util.LinkedHashMap
-
-import org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.trees.TreeNodeTag
/**
- * Pull out nondeterministic expressions in an expression tree and replace them with the
- * corresponding attributes in the `nondeterministicToAttributes` map.
+ * Object used to store single-pass resolver related tags.
*/
-object PullOutNondeterministicExpressionInExpressionTree {
- def apply[ExpressionType <: Expression](
- expression: ExpressionType,
- nondeterministicToAttributes: LinkedHashMap[Expression, NamedExpression]): ExpressionType = {
- expression
- .transform {
- case childExpression =>
- nondeterministicToAttributes.get(childExpression) match {
- case null =>
- childExpression
- case namedExpression =>
- namedExpression.toAttribute
- }
- }
- .asInstanceOf[ExpressionType]
- }
+object ResolverTag {
+
+ /**
+ * Tag used to mark [[Project]] nodes added for expression ID deduplication.
+ */
+ val PROJECT_FOR_EXPRESSION_ID_DEDUPLICATION =
+ TreeNodeTag[Unit]("project_for_expression_id_deduplication")
+
+ /**
+ * Tag used to mark a node after resolving it to avoid traversing into its subtree twice.
+ */
+ val SINGLE_PASS_SUBTREE_BOUNDARY =
+ TreeNodeTag[Unit]("single_pass_subtree_boundary")
+
+ /**
+ * Tag used to determine whether a node is an LCA.
+ */
+ val SINGLE_PASS_IS_LCA =
+ TreeNodeTag[Unit]("single_pass_is_lca")
+
+ /**
+ * Tag used to mark the operator as the top-most operator in a query or a view.
+ */
+ val TOP_LEVEL_OPERATOR =
+ TreeNodeTag[Unit]("top_level_operator")
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolvesNameByHiddenOutput.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolvesNameByHiddenOutput.scala
index 06a93910f2c59..ca6f4a63c88ef 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolvesNameByHiddenOutput.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ResolvesNameByHiddenOutput.scala
@@ -21,22 +21,11 @@ import java.util.HashSet
import scala.collection.mutable
-import org.apache.spark.sql.catalyst.expressions.{
- Attribute,
- AttributeReference,
- ExprId,
- NamedExpression,
- PipeOperator
-}
-import org.apache.spark.sql.catalyst.plans.logical.{
- Aggregate,
- Distinct,
- LogicalPlan,
- Project,
- SubqueryAlias,
- UnaryNode
-}
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.{ExprId, NamedExpression, PipeOperator}
+import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.util._
+import org.apache.spark.sql.internal.SQLConf
/**
* [[ResolvesNameByHiddenOutput]] is used by resolvers for operators that are able to resolve
@@ -175,7 +164,7 @@ import org.apache.spark.sql.catalyst.util._
*
* In the plan you can see that `col2` is added to the lower [[Project.projectList]].
*/
-trait ResolvesNameByHiddenOutput {
+trait ResolvesNameByHiddenOutput extends SQLConfHelper {
/**
* Insert the missing expressions in the output list of the operator. Recursively call
@@ -233,6 +222,20 @@ trait ResolvesNameByHiddenOutput {
case other => other
}
+ /**
+ * Deduplicates missing expressions by [[ExprId]].
+ */
+ def deduplicateMissingExpressions(
+ missingExpressions: Seq[NamedExpression]): Seq[NamedExpression] = {
+ val duplicateMissingExpressions = new HashSet[ExprId]
+ missingExpressions.collect {
+ case expression: NamedExpression
+ if !duplicateMissingExpressions.contains(expression.exprId) =>
+ duplicateMissingExpressions.add(expression.exprId)
+ expression
+ }
+ }
+
private def expandOperatorsOutputList(
operator: UnaryNode,
operatorOutput: Seq[NamedExpression],
@@ -242,24 +245,27 @@ trait ResolvesNameByHiddenOutput {
missingExpressions = missingExpressions
)
- val missingAttributes = filteredMissingExpressions.collect {
- case attribute: AttributeReference => attribute
- }
-
- val expandedChild = insertMissingExpressions(operator.child, missingAttributes)
+ if (filteredMissingExpressions.nonEmpty) {
+ val (metadataCols, nonMetadataCols) =
+ operatorOutput.partition(_.toAttribute.qualifiedAccessOnly)
- val (metadataCols, nonMetadataCols) =
- operatorOutput.partition(_.toAttribute.qualifiedAccessOnly)
+ operator match {
+ case aggregate: Aggregate =>
+ val newAggregateList = nonMetadataCols ++ filteredMissingExpressions ++ metadataCols
+ aggregate.copy(aggregateExpressions = newAggregateList)
+ case project: Project =>
+ val expandedChild = insertMissingExpressions(
+ operator = operator.child,
+ missingExpressions = filteredMissingExpressions
+ )
+ val newProjectList =
+ nonMetadataCols ++ filteredMissingExpressions.map(_.toAttribute) ++ metadataCols
- val newOutputList = nonMetadataCols ++ filteredMissingExpressions ++ metadataCols
- val newOperator = operator match {
- case aggregate: Aggregate =>
- aggregate.copy(aggregateExpressions = newOutputList, child = expandedChild)
- case project: Project =>
- project.copy(projectList = newOutputList, child = expandedChild)
+ project.copy(projectList = newProjectList, child = expandedChild)
+ }
+ } else {
+ operator
}
-
- newOperator
}
private def filterMissingExpressions(
@@ -300,12 +306,43 @@ trait ResolvesNameByHiddenOutput {
* because they may be needed in upper operators (if not, they will be pruned away in
* [[PruneMetadataColumns]]). Other hidden attributes are thrown away, because we cannot
* reference them from the new [[Project]] (they are not outputted from below).
+ *
+ * If [[SQLConf.SINGLE_PASS_RESOLVER_PREVENT_USING_ALIASES_FROM_NON_DIRECT_CHILDREN]] is set to
+ * true, we need to overwrite the current scope and clear `aggregateListAliases` and
+ * `baseAggregate`. This is needed in order to prevent later replacement of Sort/Having
+ * expressions using semantically equal aliased expressions from non-direct children. For
+ * example, in the following query:
+ *
+ * {{{ SELECT col1 AS a FROM VALUES(1,2) GROUP BY col1, col2 HAVING col2 > 1 ORDER BY col1; }}}
+ *
+ * With flag set to false, analyzed plan will be:
+ *
+ * Sort [a#3 ASC NULLS FIRST], true
+ * +- Project [a#3]
+ * +- Filter (col2#2 > 1)
+ * +- Aggregate [col1#1, col2#2], [col1#1 AS a#3, col2#2, col1#1]
+ * +- LocalRelation [col1#1, col2#2]
+ *
+ * Instead of using missing attribute `col1#1` we can use its alias `a#3` in the [[Sort]] and
+ * avoid adding an extra projection. This is because all of [[Sort]], [[Project]], [[Filter]] and
+ * [[Aggregate]] belong to the same [[NameScope]] since [[Project]] was artificially inserted.
+ *
+ * However, fixed-point can't handle this case properly and produces the following plan:
+ *
+ * Project [a#3]
+ * +- Sort [col1#1 ASC NULLS FIRST], true
+ * +- Project [a#3, col1#1]
+ * +- Filter (col2#2 > 1)
+ * +- Aggregate [col1#1, col2#2], [col1#1 AS a#3, col2#2, col1#1]
+ * +- LocalRelation [col1#1, col2#2]
+ *
+ * Therefore, we need to match this behavior of fixed-point in single-pass in order to avoid
+ * logical plan mismatches.
*/
def retainOriginalOutput(
operator: LogicalPlan,
missingExpressions: Seq[NamedExpression],
- output: Seq[Attribute],
- hiddenOutput: Seq[Attribute]): LogicalPlan = {
+ scopes: NameScopeStack): LogicalPlan = {
if (missingExpressions.isEmpty) {
operator
} else {
@@ -314,17 +351,29 @@ trait ResolvesNameByHiddenOutput {
missingExpressionIds.add(expression.exprId)
}
- val hiddenOutputToPreserve = hiddenOutput.filter { hiddenAttribute =>
+ val hiddenOutputToPreserve = scopes.current.hiddenOutput.filter { hiddenAttribute =>
hiddenAttribute.qualifiedAccessOnly && missingExpressionIds.contains(
hiddenAttribute.exprId
)
}
val project = Project(
- projectList = output ++ hiddenOutputToPreserve,
+ projectList = scopes.current.output ++ hiddenOutputToPreserve,
child = operator
)
+ if (conf.getConf(
+ SQLConf.ANALYZER_SINGLE_PASS_RESOLVER_PREVENT_USING_ALIASES_FROM_NON_DIRECT_CHILDREN
+ )) {
+ scopes.overwriteCurrent(
+ output = Some(scopes.current.output),
+ hiddenOutput = Some(scopes.current.hiddenOutput),
+ availableAliases = Some(scopes.current.availableAliases),
+ aggregateListAliases = Seq.empty,
+ baseAggregate = None
+ )
+ }
+
project
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/RewritesAliasesInTopLcaProject.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/RewritesAliasesInTopLcaProject.scala
new file mode 100644
index 0000000000000..1fc9d579f6324
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/RewritesAliasesInTopLcaProject.scala
@@ -0,0 +1,256 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.analysis.resolver
+
+import java.util.{HashMap, HashSet}
+
+import org.apache.spark.sql.catalyst.expressions.{
+ Alias,
+ AttributeReference,
+ Expression,
+ ExprId,
+ NamedExpression
+}
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Project}
+import org.apache.spark.sql.catalyst.trees.TreePattern.{AGGREGATE_EXPRESSION, ATTRIBUTE_REFERENCE}
+
+/**
+ * During LCA resolution some aliases may be rewritten as new aliases with new [[ExprId]]s. This
+ * trait handles remapping of old aliases to new ones, when these attributes appear in
+ * [[SortOrder]] expressions and Having conditions.
+ */
+trait RewritesAliasesInTopLcaProject {
+
+ /**
+ * When resolving lateral column references in [[Aggregate]] below [[Sort]] or HAVING operators,
+ * fixed-point first resolves [[SortOrder]] expressions and HAVING conditions using
+ * [[TempResolvedColumn]] and only after that resolves lateral column references. For example,
+ * consider the following query:
+ *
+ * {{{ SELECT avg(col1) AS a, a AS b FROM VALUES(1,2,3) GROUP BY col2 ORDER BY max(col3) }}}
+ *
+ * Fixed-point plan before resolving [[SortOrder]]:
+ *
+ * Sort [max(tempresolvedcolumn(col3#5, col3, false)) ASC NULLS FIRST], true
+ * +- Aggregate [col2#4], [avg(col1#3) AS a#6, lateralAliasReference(a) AS b#7]
+ * +- LocalRelation [col1#3, col2#4, col3#5]
+ *
+ * After resolving [[TempResolvedColumn]]:
+ *
+ * Project [a#6, b#7]
+ * +- Sort [max(col3)#10 ASC NULLS FIRST], true
+ * +- Aggregate [col2#4], [avg(col1#3) AS a#6, lca(a) AS b#7, max(col3#5) AS max(col3)#10]
+ * +- LocalRelation [col1#3, col2#4, col3#5]
+ *
+ * In the above case fixed-point first resolves [[SortOrder]] to `max(col3)#10` and only then
+ * resolves LCAs. However, while resolving LCAs in [[Aggregate]], fixed-point first constructs
+ * a base [[Aggregate]] by pushing down all aggregate expressions with new aliases. It then
+ * places a [[Project]] on top reinstating the original alias on top of a newly created one,
+ * in order to still match the attribute reference from [[SortOrder]]:
+ *
+ * Project [a#6, b#7]
+ * +- Sort [max(col3)#10 ASC NULLS FIRST], true
+ * +- Project [avg(col1)#11 AS a#6, lca(a) AS b#7, max(col3)#12 AS max(col3)#10]
+ * +- Aggregate [col2#4], [avg(col1#3) AS avg(col1)#11, max(col3#5) AS max(col3)#12]
+ * +- LocalRelation [col1#3, col2#4, col3#5]
+ *
+ * In the example above, `max(col3#5)` gets pushed down and aliased as `max(col3)#12`, even
+ * though `max(col3)#10` attribute reference already exists. Because of that `max(col3)#12` needs
+ * to be remapped back to `max(col3)#10`.
+ *
+ * However, in single-pass analyzer, we will first resolve all lateral column references before
+ * starting the resolution of [[SortOrder]] resulting in the following plan:
+ *
+ * Project [a#6, b#7]
+ * +- Sort [max(col3)#16 ASC NULLS FIRST], true
+ * +- Project [a#6, a#6 AS b#7, max(col3)#16]
+ * +- Project [avg(col1)#14, avg(col1)#14 AS a#6, max(col3)#16]
+ * +- Aggregate [col2#4], [avg(col1#3) AS avg(col1)#14, max(col3#5) AS max(col3)#16]
+ * +- LocalRelation [col1#3, col2#4, col3#5]
+ *
+ * In the above case, rewriting `max(col3)#16` with an [[Alias]] is not necessary from
+ * correctness perspective, but we need to do it in order to stay compatible with fixed-point
+ * analyzer. Because fixed-point only regenerates aliases from original aggregate list, in
+ * single-pass we need to handle the following:
+ * 1. all aliases from top-level [[Project]] (because they originate from the unresolved
+ * aggregate list);
+ * 2. all references to aliases from the base aggregate (because they are became attribute
+ * references during LCA resolution);
+ *
+ * This same issue also applies to HAVING resolution.
+ */
+ def rewriteNamedExpressionsInTopLcaProject[ExpressionType <: Expression](
+ projectToRewrite: Project,
+ baseAggregate: Aggregate,
+ expressionsToRewrite: Seq[ExpressionType],
+ rewriteCandidates: Seq[NamedExpression],
+ autoGeneratedAliasProvider: AutoGeneratedAliasProvider): (Project, Seq[ExpressionType]) = {
+ val candidateExpressions = getCandidateExpressionsForRewrite(
+ baseAggregate = baseAggregate,
+ oldExpressions = rewriteCandidates,
+ autoGeneratedAliasProvider = autoGeneratedAliasProvider
+ )
+ val newProject = rewriteNamedExpressionsInProject(projectToRewrite, candidateExpressions)
+ val newExpressions = updateAttributeReferencesInExpressions[ExpressionType](
+ expressionsToRewrite,
+ candidateExpressions
+ )
+
+ (newProject, newExpressions)
+ }
+
+ /**
+ * When resolving [[Sort]] or Having on top of an [[Aggregate]] that has lateral column
+ * references, aggregate and grouping expressions might not be correctly replaced in
+ * [[SortOrder]] and HAVING condition, because of [[Project]] nodes created when resolving
+ * lateral column references. Because of that, we need to additionally try and replace
+ * [[SortOrder]] expressions and HAVING conditions that don't appear in the child [[Project]],
+ * but the aliases of semantically equivalent expressions do. In case both the attribute and its
+ * alias exist in the output, don't replace the attribute in [[SortOrder]] / HAVING condition,
+ * because there is no missing input in that case.
+ * For example, consider the following query:
+ *
+ * {{{ SELECT col1 AS a, a FROM VALUES(1) GROUP BY col1 ORDER BY col1 }}}
+ *
+ * After resolving lateral column references and partially resolving [[SortOrder]] expression, we
+ * get the following plan:
+ *
+ * !Sort [col1#3 ASC NULLS FIRST], true
+ * +- Project [a#4, a#4]
+ * +- Project [col1#3, col1#3 AS a#4]
+ * +- Aggregate [col1#3], [col1#3]
+ * +- LocalRelation [col1#3]
+ *
+ * In the above plan, [[Sort]] has a missing input `col1#3`. Because of LCA resolution this
+ * attribute is pushed down into the [[Project]] stack and aliased as `a#4`. Instead of using
+ * `col1#3` we can reference its semantically equivalent alias `a#4` in the [[SortOrder]]. The
+ * resolved plan looks like:
+ *
+ * Sort [a#4 ASC NULLS FIRST], true
+ * +- Project [a#4, a#4]
+ * +- Project [col1#3, col1#3 AS a#4]
+ * +- Aggregate [col1#3], [col1#3]
+ * +- LocalRelation [col1#3]
+ *
+ * Because we used `a#4` alias instead of `col1#3`, we do not need to insert `col1#3` to the
+ * child [[Project]] as a missing expression. Therefore, `missingExpressions` need to be updated
+ * in order not to insert unnecessary attributes in
+ * [[ResolvesNameByHiddenOutput.insertMissingExpressions]]
+ *
+ * However, for a query like:
+ *
+ * {{{ SELECT col1, col1 AS a FROM VALUES(1) GROUP BY col1 ORDER BY col1 }}}
+ *
+ * The resolved plan will be:
+ *
+ * Sort [col1#4 ASC NULLS FIRST], true
+ * +- Aggregate [col1#4], [col1#4, col1#4 AS a#5]
+ * +- LocalRelation [col1#4]
+ *
+ * In the above example, we do not replace `col1#4` with `a#5` because `col1#4` is present in the
+ * output.
+ */
+ def tryReplaceSortOrderOrHavingConditionWithAlias(
+ sortOrderOrCondition: Expression,
+ scopes: NameScopeStack,
+ missingExpressions: Seq[NamedExpression]): (Expression, Seq[NamedExpression]) = {
+ val replacedAttributeReferences = new HashSet[ExprId]
+ val expressionWithReplacedAliases = sortOrderOrCondition.transformDownWithPruning(
+ _.containsAnyPattern(AGGREGATE_EXPRESSION, ATTRIBUTE_REFERENCE)
+ ) {
+ case attributeReference: AttributeReference =>
+ scopes.current.aggregateListAliases
+ .collectFirst {
+ case alias
+ if alias.child.semanticEquals(attributeReference) &&
+ scopes.current.getAttributeById(attributeReference.exprId).isEmpty =>
+ replacedAttributeReferences.add(attributeReference.exprId)
+ alias.toAttribute
+ }
+ .getOrElse(attributeReference)
+ case aggregateExpression: AggregateExpression =>
+ scopes.current.aggregateListAliases
+ .collectFirst {
+ case alias if alias.child.semanticEquals(aggregateExpression) =>
+ alias.toAttribute
+ }
+ .getOrElse(aggregateExpression)
+ }
+ val filteredMissingExpressions = missingExpressions.filter(
+ expression => !replacedAttributeReferences.contains(expression.exprId)
+ )
+
+ (expressionWithReplacedAliases, filteredMissingExpressions)
+ }
+
+ private def getCandidateExpressionsForRewrite(
+ baseAggregate: Aggregate,
+ oldExpressions: Seq[NamedExpression],
+ autoGeneratedAliasProvider: AutoGeneratedAliasProvider): HashMap[ExprId, NamedExpression] = {
+ val expressionsToRewrite = new HashMap[ExprId, NamedExpression](oldExpressions.size)
+ val baseAggregateOutputLookup = new HashSet[ExprId](baseAggregate.aggregateExpressions.size)
+ baseAggregate.aggregateExpressions.foreach {
+ case alias: Alias => baseAggregateOutputLookup.add(alias.exprId)
+ case _ =>
+ }
+ oldExpressions.foreach {
+ case oldAlias: Alias =>
+ expressionsToRewrite.put(
+ oldAlias.exprId,
+ autoGeneratedAliasProvider.newAlias(oldAlias.toAttribute)
+ )
+ case oldAttributeReference: AttributeReference
+ if baseAggregateOutputLookup.contains(oldAttributeReference.exprId) =>
+ expressionsToRewrite.put(
+ oldAttributeReference.exprId,
+ autoGeneratedAliasProvider.newAlias(oldAttributeReference.toAttribute)
+ )
+ case other => expressionsToRewrite.put(other.exprId, other)
+ }
+
+ expressionsToRewrite
+ }
+
+ private def rewriteNamedExpressionsInProject(
+ project: Project,
+ candiidateExpressions: HashMap[ExprId, NamedExpression]): Project = {
+ val newProjectList = project.projectList.map {
+ case namedExpression: NamedExpression =>
+ candiidateExpressions.getOrDefault(namedExpression.exprId, namedExpression)
+ case other => other
+ }
+ project.copy(projectList = newProjectList)
+ }
+
+ private def updateAttributeReferencesInExpressions[ExpressionType <: Expression](
+ expressions: Seq[ExpressionType],
+ candidateAliases: HashMap[ExprId, NamedExpression]
+ ): Seq[ExpressionType] = {
+ expressions.map { expression =>
+ expression
+ .transformDownWithPruning(_.containsPattern(ATTRIBUTE_REFERENCE)) {
+ case attributeReference: AttributeReference =>
+ val newAliasOrOldAttribute =
+ candidateAliases.getOrDefault(attributeReference.exprId, attributeReference)
+ newAliasOrOldAttribute.toAttribute
+ }
+ .asInstanceOf[ExpressionType]
+ }
+ }
+}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/SemiStructuredExtractResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/SemiStructuredExtractResolver.scala
new file mode 100644
index 0000000000000..a5f20ae4abc7a
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/SemiStructuredExtractResolver.scala
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis.resolver
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Expression, Literal, SemiStructuredExtract}
+import org.apache.spark.sql.catalyst.expressions.variant.VariantGet
+import org.apache.spark.sql.types.VariantType
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * Resolver for [[SemiStructuredExtract]]. Resolves [[SemiStructuredExtract]] by resolving its
+ * children, replacing it with the proper semi-structured field extraction method and applying type
+ * coercion to the result.
+ */
+class SemiStructuredExtractResolver(expressionResolver: ExpressionResolver)
+ extends TreeNodeResolver[SemiStructuredExtract, Expression]
+ with ResolvesExpressionChildren
+ with CoercesExpressionTypes {
+
+ private val timezoneAwareExpressionResolver =
+ expressionResolver.getTimezoneAwareExpressionResolver
+
+ /**
+ * Resolves children and replaces [[SemiStructuredExtract]] expressions with the proper
+ * semi-structured field extraction method depending on column type. In case the column is of
+ * [[VariantType]], applies timezone to the result of the previous step.
+ *
+ * Currently only JSON is supported as an extraction method. An important distinction here with
+ * other JSON extraction methods is that the extraction fields provided here should be
+ * case-insensitive, unless explicitly stated through quoting.
+ *
+ * After replacing with proper extraction method, apply type coercion to the result.
+ */
+ override def resolve(semiStructuredExtract: SemiStructuredExtract): Expression = {
+ val semiStructuredExtractWithResolvedChildren =
+ withResolvedChildren(semiStructuredExtract, expressionResolver.resolve _)
+ .asInstanceOf[SemiStructuredExtract]
+
+ val semiStructuredExtractWithProperExtractionMethod =
+ semiStructuredExtractWithResolvedChildren.child.dataType match {
+ case _: VariantType =>
+ val extractResult = VariantGet(
+ child = semiStructuredExtractWithResolvedChildren.child,
+ path = Literal(UTF8String.fromString(semiStructuredExtractWithResolvedChildren.field)),
+ targetType = VariantType,
+ failOnError = true
+ )
+ timezoneAwareExpressionResolver.resolve(extractResult)
+ case _ =>
+ throw new AnalysisException(
+ errorClass = "COLUMN_IS_NOT_VARIANT_TYPE",
+ messageParameters = Map.empty
+ )
+ }
+
+ coerceExpressionTypes(
+ expression = semiStructuredExtractWithProperExtractionMethod,
+ expressionTreeTraversal = expressionResolver.getExpressionTreeTraversals.current
+ )
+ }
+}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/SetOperationLikeResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/SetOperationLikeResolver.scala
index 695413eaf8434..35ec8efaf8301 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/SetOperationLikeResolver.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/SetOperationLikeResolver.scala
@@ -22,17 +22,10 @@ import java.util.HashSet
import org.apache.spark.SparkException
import org.apache.spark.sql.catalyst.analysis.{AnsiTypeCoercion, TypeCoercion, TypeCoercionBase}
import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, ExprId}
-import org.apache.spark.sql.catalyst.plans.logical.{
- Except,
- Intersect,
- LogicalPlan,
- Project,
- SetOperation,
- Union
-}
+import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.types.DataTypeUtils
import org.apache.spark.sql.errors.QueryCompilationErrors
-import org.apache.spark.sql.types.{DataType, MapType, MetadataBuilder, VariantType}
+import org.apache.spark.sql.types.{DataType, MetadataBuilder}
/**
* The [[SetOperationLikeResolver]] performs [[Union]], [[Intersect]] or [[Except]] operator
@@ -53,16 +46,17 @@ class SetOperationLikeResolver(resolver: Resolver, expressionResolver: Expressio
* - Create a new mapping in [[ExpressionIdAssigner]] for the current operator. We only need the
* left child mapping, because that's the only child whose expression IDs get propagated
* upwards for [[Union]], [[Intersect]] or [[Except]]. This is an optimization.
- * - Perform individual output deduplication to handle the distinct union case described in
- * [[performIndividualOutputExpressionIdDeduplication]] scaladoc.
- * - Validate that child outputs have same length or throw "NUM_COLUMNS_MISMATCH" otherwise.
* - Compute widened data types for child output attributes using
* [[getTypeCoercion.findWiderTypeForTwo]] or throw "INCOMPATIBLE_COLUMN_TYPE" if coercion
* fails.
+ * - Perform individual output deduplication to handle the distinct union case described in
+ * [[performIndividualOutputExpressionIdDeduplication]] scaladoc.
+ * - Validate that child outputs have same length or throw "NUM_COLUMNS_MISMATCH" otherwise.
* - Add [[Project]] with [[Cast]] on children needing attribute data type widening.
* - Assert that coerced outputs don't have conflicting expression IDs.
* - Merge transformed outputs using a separate logic for each operator type.
* - Store merged output in current [[NameScope]].
+ * - Validate that the operator doesn't have unsupported data types in the output
* - Create a new mapping in [[ExpressionIdAssigner]] using the coerced and validated outputs.
* - Return the resolved operator with new children optionally wrapped in [[WithCTE]]. See
* [[CteScope]] scaladoc for more info.
@@ -74,30 +68,32 @@ class SetOperationLikeResolver(resolver: Resolver, expressionResolver: Expressio
newOutputIds = childScopes.head.getOutputIds
)
- val (deduplicatedChildren, deduplicatedChildOutputs) =
- performIndividualOutputExpressionIdDeduplication(
- resolvedChildren,
- childScopes.map(_.output),
- unresolvedOperator
- )
+ val childOutputs = childScopes.map(_.output)
- val (newChildren, newChildOutputs) =
- if (needToCoerceChildOutputs(deduplicatedChildOutputs, unresolvedOperator)) {
+ val (coercedChildren, coercedChildOutputs) =
+ if (needToCoerceChildOutputs(childOutputs, unresolvedOperator)) {
coerceChildOutputs(
- deduplicatedChildren,
- deduplicatedChildOutputs,
- validateAndDeduceTypes(unresolvedOperator, deduplicatedChildOutputs)
+ resolvedChildren,
+ childOutputs,
+ validateAndDeduceTypes(unresolvedOperator, childOutputs)
)
} else {
- (deduplicatedChildren, deduplicatedChildOutputs)
+ (resolvedChildren, childOutputs)
}
+ val (newChildren, newChildOutputs) =
+ performIndividualOutputExpressionIdDeduplication(
+ coercedChildren,
+ coercedChildOutputs,
+ unresolvedOperator
+ )
+
ExpressionIdAssigner.assertOutputsHaveNoConflictingExpressionIds(newChildOutputs)
val output = mergeChildOutputs(unresolvedOperator, newChildOutputs)
scopes.overwriteCurrent(output = Some(output), hiddenOutput = Some(output))
- validateOutputs(unresolvedOperator, output)
+ OperatorWithUncomparableTypeValidator.validate(unresolvedOperator, output)
val resolvedOperator = unresolvedOperator.withNewChildren(newChildren)
@@ -362,24 +358,6 @@ class SetOperationLikeResolver(resolver: Resolver, expressionResolver: Expressio
}
}
- /**
- * Validate outputs of [[SetOperation]].
- * - [[MapType]] and [[VariantType]] are currently not supported for [[SetOperations]] and we need
- * to throw a relevant user-facing error.
- */
- private def validateOutputs(unresolvedPlan: LogicalPlan, output: Seq[Attribute]): Unit = {
- unresolvedPlan match {
- case _: SetOperation =>
- output.find(a => hasMapType(a.dataType)).foreach { mapCol =>
- throwUnsupportedSetOperationOnMapType(mapCol, unresolvedPlan)
- }
- output.find(a => hasVariantType(a.dataType)).foreach { variantCol =>
- throwUnsupportedSetOperationOnVariantType(variantCol, unresolvedPlan)
- }
- case _ =>
- }
- }
-
private def getTypeCoercion: TypeCoercionBase = {
if (conf.ansiEnabled) {
AnsiTypeCoercion
@@ -388,24 +366,6 @@ class SetOperationLikeResolver(resolver: Resolver, expressionResolver: Expressio
}
}
- private def throwUnsupportedSetOperationOnMapType(
- mapCol: Attribute,
- unresolvedPlan: LogicalPlan): Unit = {
- throw QueryCompilationErrors.unsupportedSetOperationOnMapType(
- mapCol = mapCol,
- origin = unresolvedPlan.origin
- )
- }
-
- private def throwUnsupportedSetOperationOnVariantType(
- variantCol: Attribute,
- unresolvedPlan: LogicalPlan): Unit = {
- throw QueryCompilationErrors.unsupportedSetOperationOnVariantType(
- variantCol = variantCol,
- origin = unresolvedPlan.origin
- )
- }
-
private def throwNumColumnsMismatch(
expectedNumColumns: Int,
childColumnTypes: Seq[DataType],
@@ -436,12 +396,4 @@ class SetOperationLikeResolver(resolver: Resolver, expressionResolver: Expressio
origin = unresolvedOperator.origin
)
}
-
- private def hasMapType(dt: DataType): Boolean = {
- dt.existsRecursively(_.isInstanceOf[MapType])
- }
-
- private def hasVariantType(dt: DataType): Boolean = {
- dt.existsRecursively(_.isInstanceOf[VariantType])
- }
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/SortResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/SortResolver.scala
index 3e271a324209e..ead425510d57f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/SortResolver.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/SortResolver.scala
@@ -17,15 +17,11 @@
package org.apache.spark.sql.catalyst.analysis.resolver
-import java.util.{HashMap, LinkedHashMap}
+import java.util.HashMap
import scala.collection.mutable
-import scala.jdk.CollectionConverters._
-import org.apache.spark.sql.catalyst.analysis.{
- NondeterministicExpressionCollection,
- UnresolvedAttribute
-}
+import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
import org.apache.spark.sql.catalyst.expressions.{
Alias,
Attribute,
@@ -34,13 +30,14 @@ import org.apache.spark.sql.catalyst.expressions.{
NamedExpression,
SortOrder
}
-import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, LogicalPlan, Project, Sort}
+import org.apache.spark.sql.catalyst.plans.logical._
/**
* Resolves a [[Sort]] by resolving its child and order expressions.
*/
class SortResolver(operatorResolver: Resolver, expressionResolver: ExpressionResolver)
extends TreeNodeResolver[Sort, LogicalPlan]
+ with RewritesAliasesInTopLcaProject
with ResolvesNameByHiddenOutput {
private val scopes: NameScopeStack = operatorResolver.getNameScopes
private val autoGeneratedAliasProvider = new AutoGeneratedAliasProvider(
@@ -102,8 +99,6 @@ class SortResolver(operatorResolver: Resolver, expressionResolver: ExpressionRes
* +- Aggregate [col1, (col2 + 1)],
* [col1, sum(col1) AS sum(col1)#..., (col2 + 1) AS (col2 + 1)#...]
* +- LocalRelation [col1, col2]
- * 5. In case there are non-deterministic expressions in the order expressions, substitute them
- * with derived attribute references to an artificial [[Project]] list.
*/
override def resolve(unresolvedSort: Sort): LogicalPlan = {
val resolvedChild = operatorResolver.resolve(unresolvedSort.child)
@@ -123,50 +118,54 @@ class SortResolver(operatorResolver: Resolver, expressionResolver: ExpressionRes
} else {
val partiallyResolvedSort = unresolvedSort.copy(child = resolvedChild)
- val (resolvedOrderExpressions, missingAttributes) =
+ val (partiallyResolvedOrderExpressions, missingAttributes) =
resolveOrderExpressions(partiallyResolvedSort)
- val (finalOrderExpressions, missingExpressions) = resolvedChild match {
- case _ if scopes.current.hasLcaInAggregate =>
- throw new ExplicitlyUnsupportedResolverFeature(
- "Lateral column alias in Aggregate below a Sort"
- )
- case aggregate: Aggregate =>
- val (cleanedOrderExpressions, extractedExpressions) =
- extractReferencedGroupingAndAggregateExpressions(aggregate, resolvedOrderExpressions)
- (cleanedOrderExpressions, extractedExpressions)
- case filter @ Filter(_, aggregate: Aggregate) =>
- val (cleanedOrderExpressions, extractedExpressions) =
- extractReferencedGroupingAndAggregateExpressions(aggregate, resolvedOrderExpressions)
- (cleanedOrderExpressions, extractedExpressions)
- case project @ Project(_, Filter(_, aggregate: Aggregate)) =>
- throw new ExplicitlyUnsupportedResolverFeature(
- "Project on top of HAVING below a Sort"
+ val (resolvedOrderExpressions, missingExpressions) = resolvedChild match {
+ case _ @(_: Aggregate | _: Filter | _: Project) if scopes.current.baseAggregate.isDefined =>
+ extractReferencedGroupingAndAggregateExpressions(
+ scopes.current.baseAggregate.get,
+ partiallyResolvedOrderExpressions
)
case other =>
- (resolvedOrderExpressions, missingAttributes)
+ (partiallyResolvedOrderExpressions, missingAttributes)
}
+ val (resolvedOrderExpressionsWithAliasesReplaced, filteredMissingExpressions) =
+ tryReplaceSortOrderWithAlias(resolvedOrderExpressions, missingExpressions)
+
+ val deduplicatedMissingExpressions = deduplicateMissingExpressions(filteredMissingExpressions)
+
val resolvedChildWithMissingAttributes =
- insertMissingExpressions(resolvedChild, missingExpressions)
+ insertMissingExpressions(resolvedChild, deduplicatedMissingExpressions)
+
+ val isChildChangedByMissingExpressions = !resolvedChildWithMissingAttributes.eq(resolvedChild)
+
+ val (finalChild, finalOrderExpressions) = resolvedChildWithMissingAttributes match {
+ case project: Project if scopes.current.baseAggregate.isDefined =>
+ rewriteNamedExpressionsInTopLcaProject[SortOrder](
+ projectToRewrite = project,
+ baseAggregate = scopes.current.baseAggregate.get,
+ expressionsToRewrite = resolvedOrderExpressionsWithAliasesReplaced,
+ rewriteCandidates = missingExpressions,
+ autoGeneratedAliasProvider = autoGeneratedAliasProvider
+ )
+ case other => (other, resolvedOrderExpressionsWithAliasesReplaced)
+ }
val resolvedSort = unresolvedSort.copy(
- child = resolvedChildWithMissingAttributes,
+ child = finalChild,
order = finalOrderExpressions
)
- val sortWithOriginalOutput = retainOriginalOutput(
- operator = resolvedSort,
- missingExpressions = missingExpressions,
- output = scopes.current.output,
- hiddenOutput = scopes.current.hiddenOutput
- )
-
- sortWithOriginalOutput match {
- case project @ Project(_, sort: Sort) =>
- project.copy(child = tryPullOutNondeterministic(sort, childOutput = sort.child.output))
- case sort: Sort =>
- tryPullOutNondeterministic(sort, childOutput = scopes.current.output)
+ if (isChildChangedByMissingExpressions) {
+ retainOriginalOutput(
+ operator = resolvedSort,
+ missingExpressions = missingExpressions,
+ scopes = scopes
+ )
+ } else {
+ resolvedSort
}
}
}
@@ -197,10 +196,6 @@ class SortResolver(operatorResolver: Resolver, expressionResolver: ExpressionRes
* SELECT col1 FROM VALUES(1,2) GROUP BY col1 HAVING col1 > 1 ORDER BY col2;
* SELECT col1 FROM VALUES(1) ORDER BY col2;
* }}}
- *
- * If the order expression is not present in the current scope, but an alias of this expression
- * is, replace the order expression with its alias (see
- * [[tryReplaceSortOrderExpressionWithAlias]]).
*/
private def resolveOrderExpressions(
partiallyResolvedSort: Sort): (Seq[SortOrder], Seq[Attribute]) = {
@@ -211,11 +206,9 @@ class SortResolver(operatorResolver: Resolver, expressionResolver: ExpressionRes
.resolveExpressionTreeInOperator(sortOrder, partiallyResolvedSort)
.asInstanceOf[SortOrder]
- tryReplaceSortOrderExpressionWithAlias(resolvedSortOrder).getOrElse {
- referencedAttributes.putAll(expressionResolver.getLastReferencedAttributes)
+ referencedAttributes.putAll(expressionResolver.getLastReferencedAttributes)
- resolvedSortOrder
- }
+ resolvedSortOrder
}
val missingAttributes = scopes.current.resolveMissingAttributesByHiddenOutput(
@@ -225,37 +218,6 @@ class SortResolver(operatorResolver: Resolver, expressionResolver: ExpressionRes
(resolvedSortOrder, missingAttributes)
}
- /**
- * When resolving [[SortOrder]] on top of an [[Aggregate]], if there is an attribute that is
- * present in `hiddenOutput` and there is an [[Alias]] of this attribute in the `output`,
- * [[SortOrder]] should be resolved by the [[Alias]] instead of an attribute. This is done as
- * optimization in order to avoid a [[Project]] node being added when resolving the attribute via
- * missing input (because attribute is not present in direct output, only its alias is).
- *
- * For example, for a query like:
- *
- * {{{
- * SELECT col1 + 1 AS a FROM VALUES(1) GROUP BY a ORDER BY col1 + 1;
- * }}}
- *
- * The resolved plan should be:
- *
- * Sort [a#2 ASC NULLS FIRST], true
- * +- Aggregate [(col1#1 + 1)], [(col1#1 + 1) AS a#2]
- * +- LocalRelation [col1#1]
- *
- * [[SortOrder]] expression is resolved to alias of `col1 + 1` instead of `col1 + 1` itself.
- */
- private def tryReplaceSortOrderExpressionWithAlias(sortOrder: SortOrder): Option[SortOrder] = {
- scopes.current.aggregateListAliases
- .collectFirst {
- case alias if alias.child.semanticEquals(sortOrder.child) => alias.toAttribute
- }
- .map { aliasCandidate =>
- sortOrder.withNewChildren(newChildren = Seq(aliasCandidate)).asInstanceOf[SortOrder]
- }
- }
-
/**
* Extracts the referenced grouping and aggregate expressions from the order expressions. This is
* used to update the output of the child operator and add a [[Project]] as a parent of [[Sort]]
@@ -313,52 +275,33 @@ class SortResolver(operatorResolver: Resolver, expressionResolver: ExpressionRes
}
/**
- * In case there are non-deterministic expressions in `order` expressions replace them with
- * attributes created out of corresponding non-deterministic expression. Example:
- *
- * {{{ SELECT 1 ORDER BY RAND(); }}}
- *
- * This query would have the following analyzed plan:
- *
- * Project [1]
- * +- Sort [_nondeterministic ASC NULLS FIRST], true
- * +- Project [1, rand(...) AS _nondeterministic#...]
- * +- Project [1 AS 1#...]
- * +- OneRowRelation
- *
- * We use `childOutput` instead of directly calling `scopes.current.output`, because
- * [[insertMissingExpressions]] could have changed the output of the child operator.
- * We could just call `sort.child.output`, but this is suboptimal for the simple case when
- * [[Sort]] child is left unchanged, and in that case we actually call `scopes.current.output`.
- * See the call site in [[resolve]].
+ * When resolving [[Sort]] on top of an [[Aggregate]] that has lateral column aliases,
+ * [[extractReferencedGroupingAndAggregateExpressions]] may not correctly replace all
+ * [[SortOrder]] expressions because of newly construct [[Project]] nodes coming from LCA
+ * resolution. This method replaces all [[SortOrder]] with their aliases if those expressions
+ * don't exist in child [[Project]] but the aliases do.
+ * For more details see [[tryReplaceSortOrderOrHavingConditionWithAlias]].
*/
- private def tryPullOutNondeterministic(sort: Sort, childOutput: Seq[Attribute]): LogicalPlan = {
- val nondeterministicToAttributes: LinkedHashMap[Expression, NamedExpression] =
- NondeterministicExpressionCollection.getNondeterministicToAttributes(
- sort.order.map(_.child)
- )
+ private def tryReplaceSortOrderWithAlias(
+ orderExpressions: Seq[SortOrder],
+ missingExpressions: Seq[NamedExpression]
+ ): (Seq[SortOrder], Seq[NamedExpression]) = {
+ val replacedOrderExpressions = new mutable.ArrayBuffer[SortOrder]
+ var currentMissingExpressions = missingExpressions
- if (!nondeterministicToAttributes.isEmpty) {
- val newChild = Project(
- childOutput ++ nondeterministicToAttributes.values.asScala.toSeq,
- sort.child
- )
- val resolvedOrder = sort.order.map { sortOrder =>
- sortOrder.copy(
- child = PullOutNondeterministicExpressionInExpressionTree(
- sortOrder.child,
- nondeterministicToAttributes
- )
+ orderExpressions.map { orderExpression =>
+ val (replacedOrder, updatedMissingExpressions) =
+ tryReplaceSortOrderOrHavingConditionWithAlias(
+ sortOrderOrCondition = orderExpression,
+ scopes = scopes,
+ missingExpressions = currentMissingExpressions
)
- }
- val resolvedSort = sort.copy(
- order = resolvedOrder,
- child = newChild
- )
- Project(projectList = childOutput, child = resolvedSort)
- } else {
- sort
+
+ replacedOrderExpressions += replacedOrder.asInstanceOf[SortOrder]
+ currentMissingExpressions = updatedMissingExpressions
}
+
+ (replacedOrderExpressions.toSeq, currentMissingExpressions)
}
private def canOrderByAll(expressions: Seq[SortOrder]): Boolean = {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TimeAddResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TimeAddResolver.scala
deleted file mode 100644
index 561e921d95dd0..0000000000000
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TimeAddResolver.scala
+++ /dev/null
@@ -1,71 +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.catalyst.analysis.resolver
-
-import org.apache.spark.sql.catalyst.analysis.{
- AnsiStringPromotionTypeCoercion,
- AnsiTypeCoercion,
- StringPromotionTypeCoercion,
- TypeCoercion
-}
-import org.apache.spark.sql.catalyst.expressions.{Expression, TimeAdd}
-
-/**
- * Helper resolver for [[TimeAdd]] which is produced by resolving [[BinaryArithmetic]] nodes.
- */
-class TimeAddResolver(expressionResolver: ExpressionResolver)
- extends TreeNodeResolver[TimeAdd, Expression]
- with ResolvesExpressionChildren
- with CoercesExpressionTypes {
-
- private val traversals = expressionResolver.getExpressionTreeTraversals
-
- protected override val ansiTransformations: CoercesExpressionTypes.Transformations =
- TimeAddResolver.ANSI_TYPE_COERCION_TRANSFORMATIONS
- protected override val nonAnsiTransformations: CoercesExpressionTypes.Transformations =
- TimeAddResolver.TYPE_COERCION_TRANSFORMATIONS
-
- override def resolve(unresolvedTimeAdd: TimeAdd): Expression = {
- val timeAddWithResolvedChildren =
- withResolvedChildren(unresolvedTimeAdd, expressionResolver.resolve _)
- val timeAddWithTypeCoercion: Expression = coerceExpressionTypes(
- expression = timeAddWithResolvedChildren,
- expressionTreeTraversal = traversals.current
- )
- TimezoneAwareExpressionResolver.resolveTimezone(
- timeAddWithTypeCoercion,
- traversals.current.sessionLocalTimeZone
- )
- }
-}
-
-object TimeAddResolver {
- // Ordering in the list of type coercions should be in sync with the list in [[TypeCoercion]].
- private val TYPE_COERCION_TRANSFORMATIONS: Seq[Expression => Expression] = Seq(
- StringPromotionTypeCoercion.apply,
- TypeCoercion.ImplicitTypeCoercion.apply,
- TypeCoercion.DateTimeOperationsTypeCoercion.apply
- )
-
- // Ordering in the list of type coercions should be in sync with the list in [[AnsiTypeCoercion]].
- private val ANSI_TYPE_COERCION_TRANSFORMATIONS: Seq[Expression => Expression] = Seq(
- AnsiStringPromotionTypeCoercion.apply,
- AnsiTypeCoercion.ImplicitTypeCoercion.apply,
- AnsiTypeCoercion.AnsiDateTimeOperationsTypeCoercion.apply
- )
-}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TimezoneAwareExpressionResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TimezoneAwareExpressionResolver.scala
index c084932813c29..712efcef5e7a3 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TimezoneAwareExpressionResolver.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/TimezoneAwareExpressionResolver.scala
@@ -123,6 +123,22 @@ object TimezoneAwareExpressionResolver {
/**
* Applies a timezone to a [[TimeZoneAwareExpression]] while preserving original tags.
*
+ * Method is applied recursively to all the nested [[TimeZoneAwareExpression]]s which lack a
+ * timezone until we find one which has it. This is because sometimes type coercion rules (or
+ * other code) can produce multiple [[Cast]]s on top of an expression. For example:
+ *
+ * {{{ SELECT NANVL(1, null); }}}
+ *
+ * Plan:
+ *
+ * {{{
+ * Project [nanvl(cast(1 as double), cast(cast(null as int) as double)) AS nanvl(1, NULL)#0]
+ * +- OneRowRelation
+ * }}}
+ *
+ * As it can be seen, there are multiple nested [[Cast]] nodes and timezone should be applied to
+ * all of them.
+ *
* This method is particularly useful for cases like resolving [[Cast]] expressions where tags
* such as [[USER_SPECIFIED_CAST]] need to be preserved.
*
@@ -133,7 +149,13 @@ object TimezoneAwareExpressionResolver {
def resolveTimezone(expression: Expression, timeZoneId: String): Expression = {
expression match {
case timezoneExpression: TimeZoneAwareExpression if timezoneExpression.timeZoneId.isEmpty =>
- val withTimezone = timezoneExpression.withTimeZone(timeZoneId)
+ val childrenWithTimeZone = timezoneExpression.children.map { child =>
+ resolveTimezone(child, timeZoneId)
+ }
+ val withNewChildren = timezoneExpression
+ .withNewChildren(childrenWithTimeZone)
+ .asInstanceOf[TimeZoneAwareExpression]
+ val withTimezone = withNewChildren.withTimeZone(timeZoneId)
withTimezone.copyTagsFrom(timezoneExpression)
withTimezone
case other => other
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/UnaryMinusResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/UnaryMinusResolver.scala
deleted file mode 100644
index 48ceb7e10ebd5..0000000000000
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/UnaryMinusResolver.scala
+++ /dev/null
@@ -1,60 +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.catalyst.analysis.resolver
-
-import org.apache.spark.sql.catalyst.analysis.{AnsiTypeCoercion, TypeCoercion}
-import org.apache.spark.sql.catalyst.expressions.{Expression, UnaryMinus}
-
-/**
- * Resolver for [[UnaryMinus]]. Resolves children and applies type coercion to target node.
- */
-class UnaryMinusResolver(expressionResolver: ExpressionResolver)
- extends TreeNodeResolver[UnaryMinus, Expression]
- with ResolvesExpressionChildren
- with CoercesExpressionTypes {
-
- private val traversals = expressionResolver.getExpressionTreeTraversals
-
- protected override val ansiTransformations: CoercesExpressionTypes.Transformations =
- UnaryMinusResolver.ANSI_TYPE_COERCION_TRANSFORMATIONS
- protected override val nonAnsiTransformations: CoercesExpressionTypes.Transformations =
- UnaryMinusResolver.TYPE_COERCION_TRANSFORMATIONS
-
- override def resolve(unresolvedUnaryMinus: UnaryMinus): Expression = {
- val unaryMinusWithResolvedChildren =
- withResolvedChildren(unresolvedUnaryMinus, expressionResolver.resolve _)
- coerceExpressionTypes(
- expression = unaryMinusWithResolvedChildren,
- expressionTreeTraversal = traversals.current
- )
- }
-}
-
-object UnaryMinusResolver {
- // Ordering in the list of type coercions should be in sync with the list in [[TypeCoercion]].
- private val TYPE_COERCION_TRANSFORMATIONS: Seq[Expression => Expression] = Seq(
- TypeCoercion.ImplicitTypeCoercion.apply,
- TypeCoercion.DateTimeOperationsTypeCoercion.apply
- )
-
- // Ordering in the list of type coercions should be in sync with the list in [[AnsiTypeCoercion]].
- private val ANSI_TYPE_COERCION_TRANSFORMATIONS: Seq[Expression => Expression] = Seq(
- AnsiTypeCoercion.ImplicitTypeCoercion.apply,
- AnsiTypeCoercion.AnsiDateTimeOperationsTypeCoercion.apply
- )
-}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/UnsupportedExpressionInOperatorValidation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/UnsupportedExpressionInOperatorValidation.scala
index ae0b5d4a48019..def4e3c30a6c3 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/UnsupportedExpressionInOperatorValidation.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/UnsupportedExpressionInOperatorValidation.scala
@@ -19,16 +19,7 @@ package org.apache.spark.sql.catalyst.analysis.resolver
import org.apache.spark.sql.catalyst.expressions.{Expression, Generator, WindowExpression}
import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
-import org.apache.spark.sql.catalyst.plans.logical.{
- Aggregate,
- BaseEvalPythonUDTF,
- CollectMetrics,
- Generate,
- LateralJoin,
- LogicalPlan,
- Project,
- Window
-}
+import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.errors.QueryCompilationErrors
object UnsupportedExpressionInOperatorValidation {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ViewResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ViewResolver.scala
index 3470bed9cfb2e..ad1926772e7f1 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ViewResolver.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/ViewResolver.scala
@@ -165,6 +165,7 @@ class ViewResolver(resolver: Resolver, catalogManager: CatalogManager)
*
* @param nestedViewDepth Current nested view depth. Cannot exceed the `maxNestedViewDepth`.
* @param maxNestedViewDepth Maximum allowed nested view depth. Configured in the upper context
+ * based on [[SQLConf.MAX_NESTED_VIEW_DEPTH]].
* @param collation View's default collation if explicitly set.
* @param catalogAndNamespace Catalog and camespace under which the [[View]] was created.
*/
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/package.scala
similarity index 100%
rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver.scala
rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/package.scala
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala
index 979613ae11266..d1f37020f2111 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala
@@ -120,9 +120,21 @@ trait ExternalCatalog {
* @param db Database that table to alter schema for exists in
* @param table Name of table to alter schema for
* @param newDataSchema Updated data schema to be used for the table.
+ * @deprecated since 4.1.0 use `alterTableSchema` instead.
*/
def alterTableDataSchema(db: String, table: String, newDataSchema: StructType): Unit
+ /**
+ * Alter the schema of a table identified by the provided database and table name.
+ *
+ * All partition columns must be preserved.
+ *
+ * @param db Database that table to alter schema for exists in
+ * @param table Name of table to alter schema for
+ * @param newSchema Updated data schema to be used for the table.
+ */
+ def alterTableSchema(db: String, table: String, newSchema: StructType): Unit
+
/** Alter the statistics of a table. If `stats` is None, then remove all existing statistics. */
def alterTableStats(db: String, table: String, stats: Option[CatalogStatistics]): Unit
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogWithListener.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogWithListener.scala
index c2613ff74da4a..33f088079caa7 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogWithListener.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogWithListener.scala
@@ -125,6 +125,12 @@ class ExternalCatalogWithListener(delegate: ExternalCatalog)
postToAll(AlterTableEvent(db, table, AlterTableKind.DATASCHEMA))
}
+ override def alterTableSchema(db: String, table: String, newSchema: StructType): Unit = {
+ postToAll(AlterTablePreEvent(db, table, AlterTableKind.SCHEMA))
+ delegate.alterTableSchema(db, table, newSchema)
+ postToAll(AlterTableEvent(db, table, AlterTableKind.SCHEMA))
+ }
+
override def alterTableStats(
db: String,
table: String,
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala
index 284ca63d820fe..5d0184579faac 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala
@@ -331,6 +331,21 @@ class InMemoryCatalog(
catalog(db).tables(table).table = origTable.copy(schema = newSchema)
}
+ override def alterTableSchema(
+ db: String,
+ table: String,
+ newSchema: StructType): Unit = synchronized {
+ requireTableExists(db, table)
+ val origTable = catalog(db).tables(table).table
+
+ val partCols = origTable.partitionColumnNames
+ assert(newSchema.map(_.name).takeRight(partCols.length) == partCols,
+ s"Partition columns ${partCols.mkString("[", ", ", "]")} are only supported at the end of " +
+ s"the new schema ${newSchema.catalogString} for now.")
+
+ catalog(db).tables(table).table = origTable.copy(schema = newSchema)
+ }
+
override def alterTableStats(
db: String,
table: String,
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
index 3eb1b35d24195..e0d1cf011e06a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
@@ -41,7 +41,7 @@ import org.apache.spark.sql.catalyst.catalog.SQLFunction.parseDefault
import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, Cast, Expression, ExpressionInfo, LateralSubquery, NamedArgumentExpression, NamedExpression, OuterReference, ScalarSubquery, UpCast}
import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParserInterface}
import org.apache.spark.sql.catalyst.plans.Inner
-import org.apache.spark.sql.catalyst.plans.logical.{FunctionSignature, InputParameter, LateralJoin, LocalRelation, LogicalPlan, NamedParametersSupport, OneRowRelation, Project, SubqueryAlias, View}
+import org.apache.spark.sql.catalyst.plans.logical.{FunctionSignature, InputParameter, LateralJoin, LogicalPlan, NamedParametersSupport, OneRowRelation, Project, SubqueryAlias, View}
import org.apache.spark.sql.catalyst.trees.CurrentOrigin
import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, StringUtils}
import org.apache.spark.sql.connector.catalog.CatalogManager
@@ -484,6 +484,7 @@ class SessionCatalog(
*
* @param identifier TableIdentifier
* @param newDataSchema Updated data schema to be used for the table
+ * @deprecated since 4.1.0 use `alterTableSchema` instead.
*/
def alterTableDataSchema(
identifier: TableIdentifier,
@@ -507,6 +508,25 @@ class SessionCatalog(
externalCatalog.alterTableDataSchema(db, table, newDataSchema)
}
+ /**
+ * Alter the schema of a table identified by the provided table identifier. All partition columns
+ * must be preserved.
+ *
+ * @param identifier TableIdentifier
+ * @param newSchema Updated schema to be used for the table
+ */
+ def alterTableSchema(
+ identifier: TableIdentifier,
+ newSchema: StructType): Unit = {
+ val qualifiedIdent = qualifyIdentifier(identifier)
+ val db = qualifiedIdent.database.get
+ val table = qualifiedIdent.table
+ requireDbExists(db)
+ requireTableExists(qualifiedIdent)
+
+ externalCatalog.alterTableSchema(db, table, newSchema)
+ }
+
private def columnNameResolved(
resolver: Resolver,
schema: StructType,
@@ -982,7 +1002,13 @@ class SessionCatalog(
objectType = Some("VIEW"),
objectName = Some(metadata.qualifiedName)
)
- val parsedPlan = SQLConf.withExistingConf(View.effectiveSQLConf(viewConfigs, isTempView)) {
+ val parsedPlan = SQLConf.withExistingConf(
+ View.effectiveSQLConf(
+ configs = viewConfigs,
+ isTempView = isTempView,
+ createSparkVersion = metadata.createVersion
+ )
+ ) {
CurrentOrigin.withOrigin(origin) {
parser.parseQuery(viewText)
}
@@ -1010,7 +1036,11 @@ class SessionCatalog(
// Note that, the column names may have duplication, e.g. `CREATE VIEW v(x, y) AS
// SELECT 1 col, 2 col`. We need to make sure that the matching attributes have the same
// number of duplications, and pick the corresponding attribute by ordinal.
- val viewConf = View.effectiveSQLConf(metadata.viewSQLConfigs, isTempView)
+ val viewConf = View.effectiveSQLConf(
+ configs = metadata.viewSQLConfigs,
+ isTempView = isTempView,
+ createSparkVersion = metadata.createVersion
+ )
val normalizeColName: String => String = if (viewConf.caseSensitiveAnalysis) {
identity
} else {
@@ -1459,7 +1489,12 @@ class SessionCatalog(
requireDbExists(db)
val newFuncDefinition = funcDefinition.copy(identifier = qualifiedIdent)
if (!functionExists(qualifiedIdent)) {
- externalCatalog.createFunction(db, newFuncDefinition)
+ try {
+ externalCatalog.createFunction(db, newFuncDefinition)
+ } catch {
+ case e: FunctionAlreadyExistsException if ignoreIfExists =>
+ // Ignore the exception as ignoreIfNotExists is set to true
+ }
} else if (!ignoreIfExists) {
throw new FunctionAlreadyExistsException(Seq(db, qualifiedIdent.funcName))
}
@@ -1481,6 +1516,8 @@ class SessionCatalog(
// For a permanent function, because we loaded it to the FunctionRegistry
// when it's first used, we also need to drop it from the FunctionRegistry.
functionRegistry.dropFunction(qualifiedIdent)
+ } else if (tableFunctionRegistry.functionExists(qualifiedIdent)) {
+ tableFunctionRegistry.dropFunction(qualifiedIdent)
}
externalCatalog.dropFunction(db, funcName)
} else if (!ignoreIfNotExists) {
@@ -1615,6 +1652,7 @@ class SessionCatalog(
// Use captured SQL configs when parsing a SQL function.
val conf = new SQLConf()
function.getSQLConfigs.foreach { case (k, v) => conf.settings.put(k, v) }
+ Analyzer.trySetAnsiValue(conf)
SQLConf.withExistingConf(conf) {
val inputParam = function.inputParam
val returnType = function.getScalarFuncReturnType
@@ -1646,7 +1684,14 @@ class SessionCatalog(
paddedInput.zip(param.fields).map {
case (expr, param) =>
- Alias(Cast(expr, param.dataType), param.name)(
+ // Add outer references to all resolved attributes and outer references in the function
+ // input. Outer references also need to be wrapped because the function input may
+ // already contain outer references.
+ val outer = expr.transform {
+ case a: Attribute if a.resolved => OuterReference(a)
+ case o: OuterReference => OuterReference(o)
+ }
+ Alias(Cast(outer, param.dataType), param.name)(
qualifier = qualifier,
// mark the alias as function input
explicitMetadata = Some(metaForFuncInputAlias))
@@ -1654,8 +1699,7 @@ class SessionCatalog(
}.getOrElse(Nil)
val body = if (query.isDefined) ScalarSubquery(query.get) else expression.get
- Project(Alias(Cast(body, returnType), funcName)() :: Nil,
- Project(inputs, LocalRelation(inputs.flatMap(_.references))))
+ Project(Alias(Cast(body, returnType), funcName)() :: Nil, Project(inputs, OneRowRelation()))
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunctionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunctionErrors.scala
index 904a17bc8ce44..ad142704f1047 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunctionErrors.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunctionErrors.scala
@@ -98,6 +98,12 @@ object UserDefinedFunctionErrors extends QueryErrorsBase {
"tempObjName" -> toSQLId(varName)))
}
+ def cyclicFunctionReference(path: String): Throwable = {
+ new AnalysisException(
+ errorClass = "CYCLIC_FUNCTION_REFERENCE",
+ messageParameters = Map("path" -> path))
+ }
+
def routinePropertyTooLarge(routineName: String): Throwable = {
new AnalysisException(
errorClass = "USER_DEFINED_FUNCTIONS.ROUTINE_PROPERTY_TOO_LARGE",
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/VariableManager.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/VariableManager.scala
index ae313f66c9f3d..4c7d8db6604b6 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/VariableManager.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/VariableManager.scala
@@ -81,6 +81,14 @@ trait VariableManager {
* @return true if at least one variable exists, false otherwise.
*/
def isEmpty: Boolean
+
+ /**
+ *
+ * @param variableName Name of the variable
+ * @return variable name formatting for the error
+ */
+ def getVariableNameForError(variableName: String): String
+
}
/**
@@ -105,6 +113,9 @@ class TempVariableManager extends VariableManager with DataTypeErrorsBase {
@GuardedBy("this")
private val variables = new mutable.HashMap[String, VariableDefinition]
+ override def getVariableNameForError(variableName: String): String =
+ toSQLId(Seq(SYSTEM_CATALOG_NAME, SESSION_NAMESPACE, variableName))
+
override def create(
nameParts: Seq[String],
varDef: VariableDefinition,
@@ -114,7 +125,7 @@ class TempVariableManager extends VariableManager with DataTypeErrorsBase {
throw new AnalysisException(
errorClass = "VARIABLE_ALREADY_EXISTS",
messageParameters = Map(
- "variableName" -> toSQLId(Seq(SYSTEM_CATALOG_NAME, SESSION_NAMESPACE, name))))
+ "variableName" -> getVariableNameForError(name)))
}
variables.put(name, varDef)
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/events.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/events.scala
index e7d41644392d5..974c225afbae3 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/events.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/events.scala
@@ -126,6 +126,7 @@ case class RenameTableEvent(
object AlterTableKind extends Enumeration {
val TABLE = "table"
val DATASCHEMA = "dataSchema"
+ val SCHEMA = "schema"
val STATS = "stats"
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
index d92d2881445ce..badfd0dfafb63 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
@@ -22,17 +22,17 @@ import java.time.{ZoneId, ZoneOffset}
import java.util.Date
import scala.collection.mutable
+import scala.util.Try
import scala.util.control.NonFatal
import com.fasterxml.jackson.annotation.JsonInclude.Include
import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper}
import com.fasterxml.jackson.module.scala.{ClassTagExtensions, DefaultScalaModule}
-import org.apache.commons.lang3.StringUtils
import org.json4s.JsonAST.{JArray, JBool, JDecimal, JDouble, JInt, JLong, JNull, JObject, JString, JValue}
import org.json4s.jackson.JsonMethods._
import org.apache.spark.SparkException
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys._
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.{CurrentUserContext, FunctionIdentifier, InternalRow, SQLConfHelper, TableIdentifier}
@@ -50,6 +50,7 @@ import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
import org.apache.spark.sql.util.{CaseInsensitiveStringMap, SchemaUtils}
import org.apache.spark.util.ArrayImplicits._
+import org.apache.spark.util.Utils
/**
* Interface providing util to convert JValue to String representation of catalog entities.
@@ -629,10 +630,6 @@ case class CatalogTable(
if (lastAccessTime <= 0) JString("UNKNOWN")
else JLong(lastAccessTime)
- val viewQueryOutputColumns: JValue =
- if (viewQueryColumnNames.nonEmpty) JArray(viewQueryColumnNames.map(JString).toList)
- else JNull
-
val map = mutable.LinkedHashMap[String, JValue]()
if (identifier.catalog.isDefined) map += "Catalog" -> JString(identifier.catalog.get)
@@ -649,21 +646,35 @@ case class CatalogTable(
}
if (comment.isDefined) map += "Comment" -> JString(comment.get)
if (collation.isDefined) map += "Collation" -> JString(collation.get)
- if (tableType == CatalogTableType.VIEW && viewText.isDefined) {
- map += "View Text" -> JString(viewText.get)
- }
- if (tableType == CatalogTableType.VIEW && viewOriginalText.isDefined) {
- map += "View Original Text" -> JString(viewOriginalText.get)
- }
- if (SQLConf.get.viewSchemaBindingEnabled && tableType == CatalogTableType.VIEW) {
- map += "View Schema Mode" -> JString(viewSchemaMode.toString)
- }
- if (viewCatalogAndNamespace.nonEmpty && tableType == CatalogTableType.VIEW) {
- import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
- map += "View Catalog and Namespace" -> JString(viewCatalogAndNamespace.quoted)
- }
- if (viewQueryOutputColumns != JNull) {
- map += "View Query Output Columns" -> viewQueryOutputColumns
+
+ if (tableType == CatalogTableType.VIEW) {
+ if (viewText.isDefined) {
+ map += "View Text" -> JString(viewText.get)
+ }
+ if (viewOriginalText.isDefined) {
+ map += "View Original Text" -> JString(viewOriginalText.get)
+ }
+ if (SQLConf.get.viewSchemaBindingEnabled) {
+ val viewSchemaModeInfo = Try(viewSchemaMode.toString).getOrElse("UNKNOWN")
+ map += "View Schema Mode" -> JString(viewSchemaModeInfo)
+ }
+ val viewCatalogAndNamespaceInfos = Try(viewCatalogAndNamespace).getOrElse(Seq.empty)
+ if (viewCatalogAndNamespaceInfos.nonEmpty) {
+ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+ map += "View Catalog and Namespace" -> JString(viewCatalogAndNamespaceInfos.quoted)
+ }
+ val viewQueryOutputColumns: JValue = Try {
+ if (viewSchemaMode == SchemaEvolution) {
+ JArray(schema.map(_.name).map(JString).toList)
+ } else if (viewQueryColumnNames.nonEmpty) {
+ JArray(viewQueryColumnNames.map(JString).toList)
+ } else {
+ JNull
+ }
+ }.getOrElse(JNull)
+ if (viewQueryOutputColumns != JNull) {
+ map += "View Query Output Columns" -> viewQueryOutputColumns
+ }
}
if (tableProperties != JNull) map += "Table Properties" -> tableProperties
stats.foreach { s =>
@@ -1144,7 +1155,7 @@ case class HiveTableRelation(
val metadataEntries = metadata.toSeq.map {
case (key, value) if key == "CatalogTable" => value
case (key, value) =>
- key + ": " + StringUtils.abbreviate(value, SQLConf.get.maxMetadataStringLength)
+ key + ": " + Utils.abbreviate(value, SQLConf.get.maxMetadataStringLength)
}
val metadataStr = truncatedString(metadataEntries, "[", ", ", "]", maxFields)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtils.scala
index 7b6664a4117a2..5f1f0bff6df40 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtils.scala
@@ -17,8 +17,6 @@
package org.apache.spark.sql.catalyst.csv
-import org.apache.commons.lang3.StringUtils
-
import org.apache.spark.SparkIllegalArgumentException
object CSVExprUtils {
@@ -134,7 +132,7 @@ object CSVExprUtils {
// in order to use existing escape logic
val readAhead = if (str(idx) == '\\') 2 else 1
// get the chunk of 1 or 2 input characters to convert to a single delimiter char
- val chunk = StringUtils.substring(str, idx, idx + readAhead)
+ val chunk = str.substring(idx, idx + readAhead)
delimiter += toChar(chunk)
// advance the counter by the length of input chunk processed
idx += chunk.length()
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVHeaderChecker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVHeaderChecker.scala
index 1c6eecad170f1..bec52747dea7c 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVHeaderChecker.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVHeaderChecker.scala
@@ -21,7 +21,7 @@ import com.univocity.parsers.common.AbstractParser
import com.univocity.parsers.csv.{CsvParser, CsvParserSettings}
import org.apache.spark.SparkIllegalArgumentException
-import org.apache.spark.internal.{Logging, MDC, MessageWithContext}
+import org.apache.spark.internal.{Logging, MessageWithContext}
import org.apache.spark.internal.LogKeys.{CSV_HEADER_COLUMN_NAME, CSV_HEADER_COLUMN_NAMES, CSV_HEADER_LENGTH, CSV_SCHEMA_FIELD_NAME, CSV_SCHEMA_FIELD_NAMES, CSV_SOURCE, NUM_COLUMNS}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.StructType
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AliasHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AliasHelper.scala
index 2cd313c873c63..6ea79f7363227 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AliasHelper.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AliasHelper.scala
@@ -99,16 +99,12 @@ trait AliasHelper {
val res = CurrentOrigin.withOrigin(e.origin) {
e match {
case a: Alias =>
- val metadata = if (a.metadata == Metadata.empty) {
- None
- } else {
- Some(a.metadata)
- }
+ // Preserve the _effective_ metadata.
a.copy(child = trimAliases(a.child))(
exprId = a.exprId,
qualifier = a.qualifier,
- explicitMetadata = metadata,
- nonInheritableMetadataKeys = a.nonInheritableMetadataKeys)
+ explicitMetadata = Some(a.metadata),
+ nonInheritableMetadataKeys = Nil)
case a: MultiAlias =>
a.copy(child = trimAliases(a.child))
case other => trimAliases(other)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ApproxTopKExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ApproxTopKExpressions.scala
new file mode 100644
index 0000000000000..3c9440764a9a1
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ApproxTopKExpressions.scala
@@ -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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions
+
+import org.apache.datasketches.frequencies.ItemsSketch
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.FunctionRegistry
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess}
+import org.apache.spark.sql.catalyst.expressions.aggregate.ApproxTopK
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
+import org.apache.spark.sql.types._
+
+/**
+ * An expression that estimates the top K items from a sketch.
+ *
+ * The input is a sketch state that is generated by the ApproxTopKAccumulation function.
+ * The output is an array of structs, each containing a frequent item and its estimated frequency.
+ * The items are sorted by their estimated frequency in descending order.
+ *
+ * @param state The sketch state, which is a struct containing the serialized sketch data,
+ * the original data type and the max items tracked of the sketch.
+ * @param k The number of top items to estimate.
+ */
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+ usage = """
+ _FUNC_(state, k) - Returns top k items with their frequency.
+ `k` An optional INTEGER literal greater than 0. If k is not specified, it defaults to 5.
+ """,
+ examples = """
+ Examples:
+ > SELECT _FUNC_(approx_top_k_accumulate(expr)) FROM VALUES (0), (0), (1), (1), (2), (3), (4), (4) AS tab(expr);
+ [{"item":0,"count":2},{"item":4,"count":2},{"item":1,"count":2},{"item":2,"count":1},{"item":3,"count":1}]
+
+ > SELECT _FUNC_(approx_top_k_accumulate(expr), 2) FROM VALUES 'a', 'b', 'c', 'c', 'c', 'c', 'd', 'd' tab(expr);
+ [{"item":"c","count":4},{"item":"d","count":2}]
+ """,
+ group = "misc_funcs",
+ since = "4.1.0")
+// scalastyle:on line.size.limit
+case class ApproxTopKEstimate(state: Expression, k: Expression)
+ extends BinaryExpression
+ with CodegenFallback
+ with ImplicitCastInputTypes {
+
+ def this(child: Expression, topK: Int) = this(child, Literal(topK))
+
+ def this(child: Expression) = this(child, Literal(ApproxTopK.DEFAULT_K))
+
+ private lazy val itemDataType: DataType = {
+ // itemDataType is the type of the second field of the output of ACCUMULATE or COMBINE
+ state.dataType.asInstanceOf[StructType](1).dataType
+ }
+
+ override def left: Expression = state
+
+ override def right: Expression = k
+
+ override def inputTypes: Seq[AbstractDataType] = Seq(StructType, IntegerType)
+
+ private def checkStateFieldAndType(state: Expression): TypeCheckResult = {
+ val stateStructType = state.dataType.asInstanceOf[StructType]
+ if (stateStructType.length != 3) {
+ return TypeCheckFailure("State must be a struct with 3 fields. " +
+ "Expected struct: struct. " +
+ "Got: " + state.dataType.simpleString)
+ }
+
+ if (stateStructType.head.dataType != BinaryType) {
+ TypeCheckFailure("State struct must have the first field to be binary. " +
+ "Got: " + stateStructType.head.dataType.simpleString)
+ } else if (!ApproxTopK.isDataTypeSupported(itemDataType)) {
+ TypeCheckFailure("State struct must have the second field to be a supported data type. " +
+ "Got: " + itemDataType.simpleString)
+ } else if (stateStructType(2).dataType != IntegerType) {
+ TypeCheckFailure("State struct must have the third field to be int. " +
+ "Got: " + stateStructType(2).dataType.simpleString)
+ } else {
+ TypeCheckSuccess
+ }
+ }
+
+
+ override def checkInputDataTypes(): TypeCheckResult = {
+ val defaultCheck = super.checkInputDataTypes()
+ if (defaultCheck.isFailure) {
+ defaultCheck
+ } else {
+ val stateCheck = checkStateFieldAndType(state)
+ if (stateCheck.isFailure) {
+ stateCheck
+ } else if (!k.foldable) {
+ TypeCheckFailure("K must be a constant literal")
+ } else {
+ TypeCheckSuccess
+ }
+ }
+ }
+
+ override def dataType: DataType = ApproxTopK.getResultDataType(itemDataType)
+
+ override def eval(input: InternalRow): Any = {
+ // null check
+ ApproxTopK.checkExpressionNotNull(k, "k")
+ // eval
+ val stateEval = left.eval(input)
+ val kEval = right.eval(input)
+ val dataSketchBytes = stateEval.asInstanceOf[InternalRow].getBinary(0)
+ val maxItemsTrackedVal = stateEval.asInstanceOf[InternalRow].getInt(2)
+ val kVal = kEval.asInstanceOf[Int]
+ ApproxTopK.checkK(kVal)
+ ApproxTopK.checkMaxItemsTracked(maxItemsTrackedVal, kVal)
+ val itemsSketch = ItemsSketch.getInstance(
+ Memory.wrap(dataSketchBytes), ApproxTopK.genSketchSerDe(itemDataType))
+ ApproxTopK.genEvalResult(itemsSketch, kVal, itemDataType)
+ }
+
+ override protected def withNewChildrenInternal(newState: Expression, newK: Expression)
+ : Expression = copy(state = newState, k = newK)
+
+ override def nullable: Boolean = false
+
+ override def prettyName: String =
+ getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("approx_top_k_estimate")
+}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
index 7a4145933fc7f..23610f97dbbd4 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
@@ -115,6 +115,8 @@ object Cast extends QueryErrorsBase {
case (_: AnsiIntervalType, _: IntegralType | _: DecimalType) => true
case (_: IntegralType | _: DecimalType, _: AnsiIntervalType) => true
+ case (_: TimeType, _: DecimalType) => true
+
case (_: DayTimeIntervalType, _: DayTimeIntervalType) => true
case (_: YearMonthIntervalType, _: YearMonthIntervalType) => true
@@ -134,6 +136,9 @@ object Cast extends QueryErrorsBase {
// to convert data of these types to Variant Objects.
case (_, VariantType) => variant.VariantGet.checkDataType(from, allowStructsAndMaps = false)
+ case (_: TimeType, _: TimeType) => true
+ case (_: TimeType, _: IntegralType) => true
+
// non-null variants can generate nulls even in ANSI mode
case (ArrayType(fromType, fn), ArrayType(toType, tn)) =>
canAnsiCast(fromType, toType) && resolvableNullability(fn || (fromType == VariantType), tn)
@@ -228,6 +233,8 @@ object Cast extends QueryErrorsBase {
case (TimestampType, DateType) => true
case (TimestampNTZType, DateType) => true
+ case (_: TimeType, _: DecimalType) => true
+
case (_: StringType, CalendarIntervalType) => true
case (_: StringType, _: DayTimeIntervalType) => true
case (_: StringType, _: YearMonthIntervalType) => true
@@ -251,6 +258,9 @@ object Cast extends QueryErrorsBase {
// to convert data of these types to Variant Objects.
case (_, VariantType) => variant.VariantGet.checkDataType(from, allowStructsAndMaps = false)
+ case (_: TimeType, _: TimeType) => true
+ case (_: TimeType, _: IntegralType) => true
+
case (ArrayType(fromType, fn), ArrayType(toType, tn)) =>
canCast(fromType, toType) &&
resolvableNullability(fn || forceNullable(fromType, toType), tn)
@@ -366,6 +376,7 @@ object Cast extends QueryErrorsBase {
case (_, _: StringType) => false
case (TimestampType, ByteType | ShortType | IntegerType) => true
+ case (_: TimeType, ByteType | ShortType) => true
case (FloatType | DoubleType, TimestampType) => true
case (TimestampType, DateType) => false
case (_, DateType) => true
@@ -493,6 +504,10 @@ case class Cast(
final override def nodePatternsInternal(): Seq[TreePattern] = Seq(CAST)
+ override def contextIndependentFoldable: Boolean = {
+ child.contextIndependentFoldable && !Cast.needsTimeZone(child.dataType, dataType)
+ }
+
def ansiEnabled: Boolean = {
evalMode == EvalMode.ANSI || (evalMode == EvalMode.TRY && !canUseLegacyCastForTryCast)
}
@@ -716,6 +731,9 @@ case class Cast(
private[this] def timestampToDouble(ts: Long): Double = {
ts / MICROS_PER_SECOND.toDouble
}
+ private[this] def timeToLong(timeNanos: Long): Long = {
+ Math.floorDiv(timeNanos, NANOS_PER_SECOND)
+ }
// DateConverter
private[this] def castToDate(from: DataType): Any => Any = from match {
@@ -733,13 +751,15 @@ case class Cast(
buildCast[Long](_, t => microsToDays(t, ZoneOffset.UTC))
}
- private[this] def castToTime(from: DataType): Any => Any = from match {
+ private[this] def castToTime(from: DataType, to: TimeType): Any => Any = from match {
case _: StringType =>
if (ansiEnabled) {
buildCast[UTF8String](_, s => DateTimeUtils.stringToTimeAnsi(s, getContextOrNull()))
} else {
buildCast[UTF8String](_, s => DateTimeUtils.stringToTime(s).orNull)
}
+ case _: TimeType =>
+ buildCast[Long](_, nanos => DateTimeUtils.truncateTimeToPrecision(nanos, to.precision))
}
// IntervalConverter
@@ -801,6 +821,8 @@ case class Cast(
buildCast[Int](_, d => null)
case TimestampType =>
buildCast[Long](_, t => timestampToLong(t))
+ case _: TimeType =>
+ buildCast[Long](_, t => timeToLong(t))
case x: NumericType if ansiEnabled =>
val exactNumeric = PhysicalNumericType.exactNumeric(x)
b => exactNumeric.toLong(b)
@@ -841,6 +863,8 @@ case class Cast(
errorOrNull(t, from, IntegerType)
}
})
+ case _: TimeType =>
+ buildCast[Long](_, t => timeToLong(t).toInt)
case x: NumericType if ansiEnabled =>
val exactNumeric = PhysicalNumericType.exactNumeric(x)
b => exactNumeric.toInt(b)
@@ -877,6 +901,15 @@ case class Cast(
errorOrNull(t, from, ShortType)
}
})
+ case _: TimeType =>
+ buildCast[Long](_, t => {
+ val longValue = timeToLong(t)
+ if (longValue == longValue.toShort) {
+ longValue.toShort
+ } else {
+ errorOrNull(t, from, ShortType)
+ }
+ })
case x: NumericType if ansiEnabled =>
val exactNumeric = PhysicalNumericType.exactNumeric(x)
b =>
@@ -924,6 +957,15 @@ case class Cast(
errorOrNull(t, from, ByteType)
}
})
+ case _: TimeType =>
+ buildCast[Long](_, t => {
+ val longValue = timeToLong(t)
+ if (longValue == longValue.toByte) {
+ longValue.toByte
+ } else {
+ errorOrNull(t, from, ByteType)
+ }
+ })
case x: NumericType if ansiEnabled =>
val exactNumeric = PhysicalNumericType.exactNumeric(x)
b =>
@@ -1002,9 +1044,15 @@ case class Cast(
b => toPrecision(if (b) Decimal.ONE else Decimal.ZERO, target, getContextOrNull()))
case DateType =>
buildCast[Int](_, d => null) // date can't cast to decimal in Hive
- case TimestampType =>
- // Note that we lose precision here.
- buildCast[Long](_, t => changePrecision(Decimal(timestampToDouble(t)), target))
+ case TimestampType => buildCast[Long](_, t => changePrecision(
+ // 19 digits is enough to represent any TIMESTAMP value in Long.
+ // 6 digits of scale is for microseconds precision of TIMESTAMP values.
+ Decimal.apply(t, 19, 6), target))
+ case _: TimeType => buildCast[Long](_, t => changePrecision(
+ // 14 digits is enough to cover the full range of TIME value [0, 24:00) which is
+ // [0, 24 * 60 * 60 * 1000 * 1000 * 1000) = [0, 86400000000000).
+ // 9 digits of scale is for nanoseconds precision of TIME values.
+ Decimal.apply(t, precision = 14, scale = 9), target))
case dt: DecimalType =>
b => toPrecision(b.asInstanceOf[Decimal], target, getContextOrNull())
case t: IntegralType =>
@@ -1149,7 +1197,7 @@ case class Cast(
case s: StringType => castToString(from, s.constraint)
case BinaryType => castToBinary(from)
case DateType => castToDate(from)
- case _: TimeType => castToTime(from)
+ case it: TimeType => castToTime(from, it)
case decimal: DecimalType => castToDecimal(from, decimal)
case TimestampType => castToTimestamp(from)
case TimestampNTZType => castToTimestampNTZ(from)
@@ -1257,7 +1305,7 @@ case class Cast(
(c, evPrim, _) => castToStringCode(from, ctx, s.constraint).apply(c, evPrim)
case BinaryType => castToBinaryCode(from)
case DateType => castToDateCode(from, ctx)
- case _: TimeType => castToTimeCode(from, ctx)
+ case it: TimeType => castToTimeCode(from, it, ctx)
case decimal: DecimalType => castToDecimalCode(from, decimal, ctx)
case TimestampType => castToTimestampCode(from, ctx)
case TimestampNTZType => castToTimestampNTZCode(from, ctx)
@@ -1354,6 +1402,7 @@ case class Cast(
private[this] def castToTimeCode(
from: DataType,
+ to: TimeType,
ctx: CodegenContext): CastFunction = {
from match {
case _: StringType =>
@@ -1374,7 +1423,11 @@ case class Cast(
}
"""
}
-
+ case _: TimeType =>
+ (nanos, evPrim, _) =>
+ code"""
+ $evPrim = $dateTimeUtilsCls.truncateTimeToPrecision($nanos, ${to.precision});
+ """
case _ =>
(_, _, evNull) => code"$evNull = true;"
}
@@ -1458,11 +1511,15 @@ case class Cast(
// date can't cast to decimal in Hive
(c, evPrim, evNull) => code"$evNull = true;"
case TimestampType =>
- // Note that we lose precision here.
(c, evPrim, evNull) =>
code"""
- Decimal $tmp = Decimal.apply(
- scala.math.BigDecimal.valueOf(${timestampToDoubleCode(c)}));
+ Decimal $tmp = Decimal.apply($c, 19, 6);
+ ${changePrecision(tmp, target, evPrim, evNull, canNullSafeCast, ctx)}
+ """
+ case _: TimeType =>
+ (c, evPrim, evNull) =>
+ code"""
+ Decimal $tmp = Decimal.apply($c, 14, 9);
${changePrecision(tmp, target, evPrim, evNull, canNullSafeCast, ctx)}
"""
case DecimalType() =>
@@ -1712,6 +1769,9 @@ case class Cast(
private[this] def timestampToDoubleCode(ts: ExprValue): Block =
code"$ts / (double)$MICROS_PER_SECOND"
+ private[this] def timeToLongCode(timeValue: ExprValue): Block =
+ code"Math.floorDiv($timeValue, ${NANOS_PER_SECOND}L)"
+
private[this] def castToBooleanCode(
from: DataType,
ctx: CodegenContext): CastFunction = from match {
@@ -1771,6 +1831,33 @@ case class Cast(
"""
}
+ private[this] def castTimeToIntegralTypeCode(
+ ctx: CodegenContext,
+ integralType: String,
+ from: DataType,
+ to: DataType): CastFunction = {
+
+ val longValue = ctx.freshName("longValue")
+ val fromDt = ctx.addReferenceObj("from", from, from.getClass.getName)
+ val toDt = ctx.addReferenceObj("to", to, to.getClass.getName)
+
+ (c, evPrim, evNull) =>
+ val overflow = if (ansiEnabled) {
+ code"""throw QueryExecutionErrors.castingCauseOverflowError($c, $fromDt, $toDt);"""
+ } else {
+ code"$evNull = true;"
+ }
+
+ code"""
+ long $longValue = ${timeToLongCode(c)};
+ if ($longValue == ($integralType) $longValue) {
+ $evPrim = ($integralType) $longValue;
+ } else {
+ $overflow
+ }
+ """
+ }
+
private[this] def castDayTimeIntervalToIntegralTypeCode(
startField: Byte,
endField: Byte,
@@ -1877,6 +1964,7 @@ case class Cast(
case DateType =>
(c, evPrim, evNull) => code"$evNull = true;"
case TimestampType => castTimestampToIntegralTypeCode(ctx, "byte", from, ByteType)
+ case _: TimeType => castTimeToIntegralTypeCode(ctx, "byte", from, ByteType)
case DecimalType() => castDecimalToIntegralTypeCode("byte")
case ShortType | IntegerType | LongType if ansiEnabled =>
castIntegralTypeToIntegralTypeExactCode(ctx, "byte", from, ByteType)
@@ -1914,6 +2002,7 @@ case class Cast(
case DateType =>
(c, evPrim, evNull) => code"$evNull = true;"
case TimestampType => castTimestampToIntegralTypeCode(ctx, "short", from, ShortType)
+ case _: TimeType => castTimeToIntegralTypeCode(ctx, "short", from, ShortType)
case DecimalType() => castDecimalToIntegralTypeCode("short")
case IntegerType | LongType if ansiEnabled =>
castIntegralTypeToIntegralTypeExactCode(ctx, "short", from, ShortType)
@@ -1949,6 +2038,8 @@ case class Cast(
case DateType =>
(c, evPrim, evNull) => code"$evNull = true;"
case TimestampType => castTimestampToIntegralTypeCode(ctx, "int", from, IntegerType)
+ case _: TimeType =>
+ (c, evPrim, _) => code"$evPrim = (int) ${timeToLongCode(c)};"
case DecimalType() => castDecimalToIntegralTypeCode("int")
case LongType if ansiEnabled =>
castIntegralTypeToIntegralTypeExactCode(ctx, "int", from, IntegerType)
@@ -1985,6 +2076,8 @@ case class Cast(
(c, evPrim, evNull) => code"$evNull = true;"
case TimestampType =>
(c, evPrim, evNull) => code"$evPrim = (long) ${timestampToLongCode(c)};"
+ case _: TimeType =>
+ (c, evPrim, evNull) => code"$evPrim = (long) ${timeToLongCode(c)};"
case DecimalType() => castDecimalToIntegralTypeCode("long")
case FloatType | DoubleType if ansiEnabled =>
castFractionToIntegralTypeCode(ctx, "long", from, LongType)
@@ -2221,6 +2314,10 @@ case class UpCast(child: Expression, target: AbstractDataType, walkedTypePath: S
case _ => target.asInstanceOf[DataType]
}
+ override def contextIndependentFoldable: Boolean = {
+ child.contextIndependentFoldable && !Cast.needsTimeZone(child.dataType, dataType)
+ }
+
override protected def withNewChildInternal(newChild: Expression): UpCast = copy(child = newChild)
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala
index 8b7d641828ba1..783de160f83b6 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala
@@ -20,7 +20,6 @@ package org.apache.spark.sql.catalyst.expressions
import java.text.{DecimalFormat, DecimalFormatSymbols, ParsePosition}
import java.util.Locale
-import org.apache.spark.SparkException
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{DataTypeMismatch, TypeCheckSuccess}
@@ -209,17 +208,6 @@ object ExprUtils extends EvalHelper with QueryErrorsBase {
"sqlExpr" -> toSQLExpr(expr),
"dataType" -> toSQLType(expr.dataType)))
}
-
- if (!expr.deterministic) {
- // This is just a sanity check, our analysis rule PullOutNondeterministic should
- // already pull out those nondeterministic expressions and evaluate them in
- // a Project node.
- throw SparkException.internalError(
- msg = s"Non-deterministic expression '${toSQLExpr(expr)}' should not appear in " +
- "grouping expression.",
- context = expr.origin.getQueryContext,
- summary = expr.origin.context.summary)
- }
}
a.groupingExpressions.foreach(checkValidGroupingExprs)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
index cfdad6e9a51fe..f706741fc98ce 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
@@ -103,6 +103,18 @@ abstract class Expression extends TreeNode[Expression] {
*/
def foldable: Boolean = false
+ /**
+ * Returns true if the expression can be folded without relying on external context,
+ * such as current time zone, session configurations, or catalogs.
+ *
+ * When an expression is context-independent foldable, it can be safely evaluated during DDL
+ * operations like creating tables, views, or constraints. This allows systems to store the
+ * computed value rather than the expression itself, improving both simplicity and performance.
+ *
+ * Default is false to ensure explicit marking of context independence.
+ */
+ def contextIndependentFoldable: Boolean = false
+
/**
* Returns true when the current expression always return the same result for fixed inputs from
* children. The non-deterministic expressions should not change in number and order. They should
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala
index cc6fea2f1b7f1..0c4395d98475f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions
import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer
-import org.apache.spark.sql.catalyst.util.SparkStringUtils
+import org.apache.spark.util.SparkStringUtils
object ExpressionSet {
/**
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedUnsafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedUnsafeProjection.scala
index 46e92e0f38e23..004cd576ace03 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedUnsafeProjection.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedUnsafeProjection.scala
@@ -268,7 +268,7 @@ object InterpretedUnsafeProjection {
writer.setNull2Bytes(i)
}
}
- case IntegerType | DateType | FloatType =>
+ case IntegerType | DateType | FloatType | _: YearMonthIntervalType =>
(v, i) => {
if (!v.isNullAt(i)) {
unsafeWriter(v, i)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/PythonUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/PythonUDF.scala
index 287f044a81610..e4d0f9642773e 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/PythonUDF.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/PythonUDF.scala
@@ -46,7 +46,8 @@ object PythonUDF {
}
def isWindowPandasUDF(e: PythonFuncExpression): Boolean = {
- // This is currently only `PythonUDAF` (which means SQL_GROUPED_AGG_PANDAS_UDF), but we might
+ // This is currently only `PythonUDAF` (which means SQL_GROUPED_AGG_PANDAS_UDF or
+ // SQL_GROUPED_AGG_ARROW_UDF), but we might
// support new types in the future, e.g, N -> N transform.
e.isInstanceOf[PythonUDAF]
}
@@ -120,11 +121,10 @@ case class PythonUDAF(
dataType: DataType,
children: Seq[Expression],
udfDeterministic: Boolean,
+ evalType: Int = PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF,
resultId: ExprId = NamedExpression.newExprId)
extends UnevaluableAggregateFunc with PythonFuncExpression {
- override def evalType: Int = PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF
-
override def sql(isDistinct: Boolean): String = {
val distinct = if (isDistinct) "DISTINCT " else ""
s"$name($distinct${children.mkString(", ")})"
@@ -176,6 +176,7 @@ abstract class UnevaluableGenerator extends Generator {
* @param pythonUDTFPartitionColumnIndexes holds the zero-based indexes of the projected results of
* all PARTITION BY expressions within the TABLE argument of
* the Python UDTF call, if applicable
+ * @param tableArguments holds whether an input argument is a table argument
*/
case class PythonUDTF(
name: String,
@@ -186,7 +187,8 @@ case class PythonUDTF(
evalType: Int,
udfDeterministic: Boolean,
resultId: ExprId = NamedExpression.newExprId,
- pythonUDTFPartitionColumnIndexes: Option[PythonUDTFPartitionColumnIndexes] = None)
+ pythonUDTFPartitionColumnIndexes: Option[PythonUDTFPartitionColumnIndexes] = None,
+ tableArguments: Option[Seq[Boolean]] = None)
extends UnevaluableGenerator with PythonFuncExpression {
override lazy val canonicalized: Expression = {
@@ -215,7 +217,8 @@ case class UnresolvedPolymorphicPythonUDTF(
evalType: Int,
udfDeterministic: Boolean,
resolveElementMetadata: (PythonFunction, Seq[Expression]) => PythonUDTFAnalyzeResult,
- resultId: ExprId = NamedExpression.newExprId)
+ resultId: ExprId = NamedExpression.newExprId,
+ tableArguments: Option[Seq[Boolean]] = None)
extends UnevaluableGenerator with PythonFuncExpression {
override lazy val resolved = false
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SemiStructuredExtract.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SemiStructuredExtract.scala
new file mode 100644
index 0000000000000..af5d2ec39d00e
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SemiStructuredExtract.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.catalyst.expressions
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.variant.VariantGet
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SEMI_STRUCTURED_EXTRACT, TreePattern}
+import org.apache.spark.sql.types.{DataType, StringType, VariantType}
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * Represents the extraction of data from a field that contains semi-structured data. The
+ * semi-structured column can only be a Variant type for now.
+ * @param child The semi-structured column
+ * @param field The field to extract
+ */
+case class SemiStructuredExtract(
+ child: Expression, field: String) extends UnaryExpression with Unevaluable {
+ override lazy val resolved = false
+ override def dataType: DataType = StringType
+
+ final override val nodePatterns: Seq[TreePattern] = Seq(SEMI_STRUCTURED_EXTRACT)
+
+ override protected def withNewChildInternal(newChild: Expression): SemiStructuredExtract =
+ copy(child = newChild)
+}
+
+/**
+ * Replaces SemiStructuredExtract expressions by extracting the specified field from the
+ * semi-structured column (only VariantType is supported for now).
+ */
+case object ExtractSemiStructuredFields extends Rule[LogicalPlan] {
+ override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveExpressionsWithPruning(
+ _.containsPattern(SEMI_STRUCTURED_EXTRACT), ruleId) {
+ case SemiStructuredExtract(column, field) if column.resolved =>
+ if (column.dataType.isInstanceOf[VariantType]) {
+ VariantGet(column, Literal(UTF8String.fromString(field)), VariantType, failOnError = true)
+ } else {
+ throw new AnalysisException(
+ errorClass = "COLUMN_IS_NOT_VARIANT_TYPE", messageParameters = Map.empty)
+ }
+ }
+}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificInternalRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificInternalRow.scala
index 971cfcae8e478..1f755df0516ff 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificInternalRow.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificInternalRow.scala
@@ -17,6 +17,8 @@
package org.apache.spark.sql.catalyst.expressions
+import scala.annotation.tailrec
+
import org.apache.spark.sql.types._
/**
@@ -192,6 +194,7 @@ final class MutableAny extends MutableValue {
*/
final class SpecificInternalRow(val values: Array[MutableValue]) extends BaseGenericInternalRow {
+ @tailrec
private[this] def dataTypeToMutableValue(dataType: DataType): MutableValue = dataType match {
// We use INT for DATE and YearMonthIntervalType internally
case IntegerType | DateType | _: YearMonthIntervalType => new MutableInt
@@ -203,6 +206,7 @@ final class SpecificInternalRow(val values: Array[MutableValue]) extends BaseGen
case BooleanType => new MutableBoolean
case ByteType => new MutableByte
case ShortType => new MutableShort
+ case udt: UserDefinedType[_] => dataTypeToMutableValue(udt.sqlType)
case _ => new MutableAny
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ToStringBase.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ToStringBase.scala
index 2e649763a9ac9..bc294fd722b3e 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ToStringBase.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ToStringBase.scala
@@ -22,7 +22,7 @@ import java.time.ZoneOffset
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.codegen._
import org.apache.spark.sql.catalyst.expressions.codegen.Block._
-import org.apache.spark.sql.catalyst.util.{ArrayData, CharVarcharCodegenUtils, DateFormatter, FractionTimeFormatter, IntervalStringStyles, IntervalUtils, MapData, SparkStringUtils, TimestampFormatter}
+import org.apache.spark.sql.catalyst.util.{ArrayData, CharVarcharCodegenUtils, DateFormatter, FractionTimeFormatter, IntervalStringStyles, IntervalUtils, MapData, TimestampFormatter}
import org.apache.spark.sql.catalyst.util.IntervalStringStyles.ANSI_STYLE
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.SQLConf.BinaryOutputStyle
@@ -30,6 +30,7 @@ import org.apache.spark.sql.types._
import org.apache.spark.unsafe.UTF8StringBuilder
import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
import org.apache.spark.util.ArrayImplicits._
+import org.apache.spark.util.SparkStringUtils
trait ToStringBase { self: UnaryExpression with TimeZoneAwareExpression =>
@@ -165,7 +166,7 @@ trait ToStringBase { self: UnaryExpression with TimeZoneAwareExpression =>
})
case pudt: PythonUserDefinedType => castToString(pudt.sqlType)
case udt: UserDefinedType[_] =>
- o => UTF8String.fromString(udt.deserialize(o).toString)
+ o => UTF8String.fromString(udt.stringifyValue(udt.deserialize(o)))
case YearMonthIntervalType(startField, endField) =>
acceptAny[Int](i => UTF8String.fromString(
IntervalUtils.toYearMonthIntervalString(i, ANSI_STYLE, startField, endField)))
@@ -274,7 +275,7 @@ trait ToStringBase { self: UnaryExpression with TimeZoneAwareExpression =>
case udt: UserDefinedType[_] =>
val udtRef = JavaCode.global(ctx.addReferenceObj("udt", udt), udt.sqlType)
(c, evPrim) =>
- code"$evPrim = UTF8String.fromString($udtRef.deserialize($c).toString());"
+ code"$evPrim = UTF8String.fromString($udtRef.stringifyValue($udtRef.deserialize($c)));"
case i: YearMonthIntervalType =>
val iu = IntervalUtils.getClass.getName.stripSuffix("$")
val iss = IntervalStringStyles.getClass.getName.stripSuffix("$")
@@ -471,7 +472,24 @@ object ToStringBase {
(array: Array[Byte]) => UTF8String.fromString(SparkStringUtils.getHexString(array))
}
}
+
+ def getBinaryParser: BinaryParser = {
+ val style = SQLConf.get.getConf(SQLConf.BINARY_OUTPUT_STYLE)
+ style match {
+ case Some(BinaryOutputStyle.UTF8) =>
+ (utf8: UTF8String) => utf8.getBytes
+ case Some(BinaryOutputStyle.BASIC) =>
+ (utf8: UTF8String) =>
+ utf8.toString.stripPrefix("[").stripSuffix("]").split(",").map(_.trim.toByte)
+ case Some(BinaryOutputStyle.BASE64) =>
+ (utf8: UTF8String) => java.util.Base64.getDecoder.decode(utf8.getBytes)
+ case Some(BinaryOutputStyle.HEX) =>
+ (utf8: UTF8String) => Hex.unhex(utf8.getBytes)
+ case _ =>
+ (utf8: UTF8String) => SparkStringUtils.fromHexString(utf8.toString)
+ }
+ }
}
trait BinaryFormatter extends (Array[Byte] => UTF8String) with Serializable
-
+trait BinaryParser extends (UTF8String => Array[Byte]) with Serializable
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala
index 7cc03f3ac3fa6..fd3d1daae6742 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions
import java.lang.reflect.{Method, Modifier}
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{FUNCTION_NAME, FUNCTION_PARAM}
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.{InternalRow, SQLConfHelper}
@@ -243,6 +243,7 @@ object V2ExpressionUtils extends SQLConfHelper with Logging {
case "ENDS_WITH" => convertBinaryExpr(expr, EndsWith)
case "CONTAINS" => convertBinaryExpr(expr, Contains)
case "IN" => convertExpr(expr, children => In(children.head, children.tail))
+ case "BOOLEAN_EXPRESSION" => toCatalyst(expr.children().head)
case _ => None
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproxTopKAggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproxTopKAggregates.scala
new file mode 100644
index 0000000000000..cefe0a14dee56
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproxTopKAggregates.scala
@@ -0,0 +1,437 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.datasketches.common._
+import org.apache.datasketches.frequencies.{ErrorType, ItemsSketch}
+import org.apache.datasketches.memory.Memory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, TypeCheckResult}
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess}
+import org.apache.spark.sql.catalyst.expressions.{ArrayOfDecimalsSerDe, Expression, ExpressionDescription, ImplicitCastInputTypes, Literal}
+import org.apache.spark.sql.catalyst.trees.{BinaryLike, TernaryLike}
+import org.apache.spark.sql.catalyst.util.{CollationFactory, GenericArrayData}
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.types._
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * The ApproxTopK function (i.e., "approx_top_k") is an aggregate function that estimates
+ * the approximate top K (aka. k-most-frequent) items in a column.
+ *
+ * The result is an array of structs, each containing a frequent item and its estimated frequency.
+ * The items are sorted by their estimated frequency in descending order.
+ *
+ * The function uses the ItemsSketch from the DataSketches library to do the estimation.
+ *
+ * See [[https://datasketches.apache.org/docs/Frequency/FrequencySketches.html]]
+ * for more information.
+ *
+ * @param expr the child expression to estimate the top K items from
+ * @param k the number of top items to return (K)
+ * @param maxItemsTracked the maximum number of items to track in the sketch
+ * @param mutableAggBufferOffset the offset for mutable aggregation buffer
+ * @param inputAggBufferOffset the offset for input aggregation buffer
+ */
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+ usage = """
+ _FUNC_(expr, k, maxItemsTracked) - Returns top k items with their frequency.
+ `k` An optional INTEGER literal greater than 0. If k is not specified, it defaults to 5.
+ `maxItemsTracked` An optional INTEGER literal greater than or equal to k and has upper limit of 1000000. If maxItemsTracked is not specified, it defaults to 10000.
+ """,
+ examples = """
+ Examples:
+ > SELECT _FUNC_(expr) FROM VALUES (0), (0), (1), (1), (2), (3), (4), (4) AS tab(expr);
+ [{"item":0,"count":2},{"item":4,"count":2},{"item":1,"count":2},{"item":2,"count":1},{"item":3,"count":1}]
+
+ > SELECT _FUNC_(expr, 2) FROM VALUES 'a', 'b', 'c', 'c', 'c', 'c', 'd', 'd' AS tab(expr);
+ [{"item":"c","count":4},{"item":"d","count":2}]
+
+ > SELECT _FUNC_(expr, 10, 100) FROM VALUES (0), (1), (1), (2), (2), (2) AS tab(expr);
+ [{"item":2,"count":3},{"item":1,"count":2},{"item":0,"count":1}]
+ """,
+ group = "agg_funcs",
+ since = "4.1.0")
+// scalastyle:on line.size.limit
+case class ApproxTopK(
+ expr: Expression,
+ k: Expression,
+ maxItemsTracked: Expression,
+ mutableAggBufferOffset: Int = 0,
+ inputAggBufferOffset: Int = 0)
+ extends TypedImperativeAggregate[ItemsSketch[Any]]
+ with ImplicitCastInputTypes
+ with TernaryLike[Expression] {
+
+ def this(child: Expression, topK: Expression, maxItemsTracked: Expression) =
+ this(child, topK, maxItemsTracked, 0, 0)
+
+ def this(child: Expression, topK: Int, maxItemsTracked: Int) =
+ this(child, Literal(topK), Literal(maxItemsTracked), 0, 0)
+
+ def this(child: Expression, topK: Expression) =
+ this(child, topK, Literal(ApproxTopK.DEFAULT_MAX_ITEMS_TRACKED), 0, 0)
+
+ def this(child: Expression, topK: Int) =
+ this(child, Literal(topK), Literal(ApproxTopK.DEFAULT_MAX_ITEMS_TRACKED), 0, 0)
+
+ def this(child: Expression) =
+ this(child, Literal(ApproxTopK.DEFAULT_K), Literal(ApproxTopK.DEFAULT_MAX_ITEMS_TRACKED), 0, 0)
+
+ private lazy val itemDataType: DataType = expr.dataType
+ private lazy val kVal: Int = {
+ ApproxTopK.checkExpressionNotNull(k, "k")
+ val kVal = k.eval().asInstanceOf[Int]
+ ApproxTopK.checkK(kVal)
+ kVal
+ }
+ private lazy val maxItemsTrackedVal: Int = {
+ ApproxTopK.checkExpressionNotNull(maxItemsTracked, "maxItemsTracked")
+ val maxItemsTrackedVal = maxItemsTracked.eval().asInstanceOf[Int]
+ ApproxTopK.checkMaxItemsTracked(maxItemsTrackedVal, kVal)
+ maxItemsTrackedVal
+ }
+
+ override def first: Expression = expr
+
+ override def second: Expression = k
+
+ override def third: Expression = maxItemsTracked
+
+ override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType, IntegerType, IntegerType)
+
+ override def checkInputDataTypes(): TypeCheckResult = {
+ val defaultCheck = super.checkInputDataTypes()
+ if (defaultCheck.isFailure) {
+ defaultCheck
+ } else if (!ApproxTopK.isDataTypeSupported(itemDataType)) {
+ TypeCheckFailure(f"${itemDataType.typeName} columns are not supported")
+ } else if (!k.foldable) {
+ TypeCheckFailure("K must be a constant literal")
+ } else if (!maxItemsTracked.foldable) {
+ TypeCheckFailure("Number of items tracked must be a constant literal")
+ } else {
+ TypeCheckSuccess
+ }
+ }
+
+ override def dataType: DataType = ApproxTopK.getResultDataType(itemDataType)
+
+ override def createAggregationBuffer(): ItemsSketch[Any] = {
+ val maxMapSize = ApproxTopK.calMaxMapSize(maxItemsTrackedVal)
+ ApproxTopK.createAggregationBuffer(expr, maxMapSize)
+ }
+
+ override def update(buffer: ItemsSketch[Any], input: InternalRow): ItemsSketch[Any] =
+ ApproxTopK.updateSketchBuffer(expr, buffer, input)
+
+ override def merge(buffer: ItemsSketch[Any], input: ItemsSketch[Any]): ItemsSketch[Any] =
+ buffer.merge(input)
+
+ override def eval(buffer: ItemsSketch[Any]): GenericArrayData =
+ ApproxTopK.genEvalResult(buffer, kVal, itemDataType)
+
+ override def serialize(buffer: ItemsSketch[Any]): Array[Byte] =
+ buffer.toByteArray(ApproxTopK.genSketchSerDe(itemDataType))
+
+ override def deserialize(storageFormat: Array[Byte]): ItemsSketch[Any] =
+ ItemsSketch.getInstance(Memory.wrap(storageFormat), ApproxTopK.genSketchSerDe(itemDataType))
+
+ override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate =
+ copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+ override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate =
+ copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+ override protected def withNewChildrenInternal(
+ newExpr: Expression,
+ newK: Expression,
+ newMaxItemsTracked: Expression): Expression =
+ copy(expr = newExpr, k = newK, maxItemsTracked = newMaxItemsTracked)
+
+ override def nullable: Boolean = false
+
+ override def prettyName: String =
+ getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("approx_top_k")
+}
+
+object ApproxTopK {
+
+ val DEFAULT_K: Int = 5
+ val DEFAULT_MAX_ITEMS_TRACKED: Int = 10000
+ private val MAX_ITEMS_TRACKED_LIMIT: Int = 1000000
+
+ def checkExpressionNotNull(expr: Expression, exprName: String): Unit = {
+ if (expr == null || expr.eval() == null) {
+ throw QueryExecutionErrors.approxTopKNullArg(exprName)
+ }
+ }
+
+ def checkK(k: Int): Unit = {
+ if (k <= 0) {
+ throw QueryExecutionErrors.approxTopKNonPositiveValue("k", k)
+ }
+ }
+
+ def checkMaxItemsTracked(maxItemsTracked: Int): Unit = {
+ if (maxItemsTracked > MAX_ITEMS_TRACKED_LIMIT) {
+ throw QueryExecutionErrors.approxTopKMaxItemsTrackedExceedsLimit(
+ maxItemsTracked, MAX_ITEMS_TRACKED_LIMIT)
+ }
+ if (maxItemsTracked <= 0) {
+ throw QueryExecutionErrors.approxTopKNonPositiveValue("maxItemsTracked", maxItemsTracked)
+ }
+ }
+
+ def checkMaxItemsTracked(maxItemsTracked: Int, k: Int): Unit = {
+ checkMaxItemsTracked(maxItemsTracked)
+ if (maxItemsTracked < k) {
+ throw QueryExecutionErrors.approxTopKMaxItemsTrackedLessThanK(maxItemsTracked, k)
+ }
+ }
+
+ def getResultDataType(itemDataType: DataType): DataType = {
+ val resultEntryType = StructType(
+ StructField("item", itemDataType, nullable = false) ::
+ StructField("count", LongType, nullable = false) :: Nil)
+ ArrayType(resultEntryType, containsNull = false)
+ }
+
+ def isDataTypeSupported(itemType: DataType): Boolean = {
+ itemType match {
+ case _: BooleanType | _: ByteType | _: ShortType | _: IntegerType |
+ _: LongType | _: FloatType | _: DoubleType | _: DateType |
+ _: TimestampType | _: TimestampNTZType | _: StringType | _: DecimalType => true
+ // BinaryType is not supported now, as ItemsSketch seems cannot count the frequency correctly
+ case _ => false
+ }
+ }
+
+ def calMaxMapSize(maxItemsTracked: Int): Int = {
+ // The maximum capacity of this internal hash map has maxMapCap = 0.75 * maxMapSize
+ // Therefore, the maxMapSize must be at least ceil(maxItemsTracked / 0.75)
+ // https://datasketches.apache.org/docs/Frequency/FrequentItemsOverview.html
+ val ceilMaxMapSize = math.ceil(maxItemsTracked / 0.75).toInt
+ // The maxMapSize must be a power of 2 and greater than ceilMaxMapSize
+ math.pow(2, math.ceil(math.log(ceilMaxMapSize) / math.log(2))).toInt
+ }
+
+ def createAggregationBuffer(itemExpression: Expression, maxMapSize: Int): ItemsSketch[Any] = {
+ itemExpression.dataType match {
+ case _: BooleanType =>
+ new ItemsSketch[Boolean](maxMapSize).asInstanceOf[ItemsSketch[Any]]
+ case _: ByteType | _: ShortType | _: IntegerType | _: FloatType | _: DateType =>
+ new ItemsSketch[Number](maxMapSize).asInstanceOf[ItemsSketch[Any]]
+ case _: LongType | _: TimestampType | _: TimestampNTZType =>
+ new ItemsSketch[Long](maxMapSize).asInstanceOf[ItemsSketch[Any]]
+ case _: DoubleType =>
+ new ItemsSketch[Double](maxMapSize).asInstanceOf[ItemsSketch[Any]]
+ case _: StringType =>
+ new ItemsSketch[String](maxMapSize).asInstanceOf[ItemsSketch[Any]]
+ case _: DecimalType =>
+ new ItemsSketch[Decimal](maxMapSize).asInstanceOf[ItemsSketch[Any]]
+ }
+ }
+
+ def updateSketchBuffer(
+ itemExpression: Expression,
+ buffer: ItemsSketch[Any],
+ input: InternalRow): ItemsSketch[Any] = {
+ val v = itemExpression.eval(input)
+ if (v != null) {
+ itemExpression.dataType match {
+ case _: BooleanType => buffer.update(v.asInstanceOf[Boolean])
+ case _: ByteType => buffer.update(v.asInstanceOf[Byte])
+ case _: ShortType => buffer.update(v.asInstanceOf[Short])
+ case _: IntegerType => buffer.update(v.asInstanceOf[Int])
+ case _: LongType => buffer.update(v.asInstanceOf[Long])
+ case _: FloatType => buffer.update(v.asInstanceOf[Float])
+ case _: DoubleType => buffer.update(v.asInstanceOf[Double])
+ case _: DateType => buffer.update(v.asInstanceOf[Int])
+ case _: TimestampType => buffer.update(v.asInstanceOf[Long])
+ case _: TimestampNTZType => buffer.update(v.asInstanceOf[Long])
+ case st: StringType =>
+ val cKey = CollationFactory.getCollationKey(v.asInstanceOf[UTF8String], st.collationId)
+ buffer.update(cKey.toString)
+ case _: DecimalType => buffer.update(v.asInstanceOf[Decimal])
+ }
+ }
+ buffer
+ }
+
+ def genEvalResult(
+ itemsSketch: ItemsSketch[Any],
+ k: Int,
+ itemDataType: DataType): GenericArrayData = {
+ val items = itemsSketch.getFrequentItems(ErrorType.NO_FALSE_POSITIVES)
+ val resultLength = math.min(items.length, k)
+ val result = new Array[AnyRef](resultLength)
+ for (i <- 0 until resultLength) {
+ val row = items(i)
+ itemDataType match {
+ case _: BooleanType | _: ByteType | _: ShortType | _: IntegerType |
+ _: LongType | _: FloatType | _: DoubleType | _: DecimalType |
+ _: DateType | _: TimestampType | _: TimestampNTZType =>
+ result(i) = InternalRow.apply(row.getItem, row.getEstimate)
+ case _: StringType =>
+ val item = UTF8String.fromString(row.getItem.asInstanceOf[String])
+ result(i) = InternalRow.apply(item, row.getEstimate)
+ }
+ }
+ new GenericArrayData(result)
+ }
+
+ def genSketchSerDe(dataType: DataType): ArrayOfItemsSerDe[Any] = {
+ dataType match {
+ case _: BooleanType => new ArrayOfBooleansSerDe().asInstanceOf[ArrayOfItemsSerDe[Any]]
+ case _: ByteType | _: ShortType | _: IntegerType | _: FloatType | _: DateType =>
+ new ArrayOfNumbersSerDe().asInstanceOf[ArrayOfItemsSerDe[Any]]
+ case _: LongType | _: TimestampType | _: TimestampNTZType =>
+ new ArrayOfLongsSerDe().asInstanceOf[ArrayOfItemsSerDe[Any]]
+ case _: DoubleType =>
+ new ArrayOfDoublesSerDe().asInstanceOf[ArrayOfItemsSerDe[Any]]
+ case _: StringType =>
+ new ArrayOfStringsSerDe().asInstanceOf[ArrayOfItemsSerDe[Any]]
+ case dt: DecimalType =>
+ new ArrayOfDecimalsSerDe(dt).asInstanceOf[ArrayOfItemsSerDe[Any]]
+ }
+ }
+
+ def getSketchStateDataType(itemDataType: DataType): StructType =
+ StructType(
+ StructField("sketch", BinaryType, nullable = false) ::
+ StructField("itemDataType", itemDataType) ::
+ StructField("maxItemsTracked", IntegerType, nullable = false) :: Nil)
+}
+
+/**
+ * An aggregate function that accumulates items into a sketch, which can then be used
+ * to combine with other sketches, via ApproxTopKCombine,
+ * or to estimate the top K items, via ApproxTopKEstimate.
+ *
+ * The output of this function is a struct containing the sketch in binary format,
+ * a null object indicating the type of items in the sketch,
+ * and the maximum number of items tracked by the sketch.
+ *
+ * @param expr the child expression to accumulate items from
+ * @param maxItemsTracked the maximum number of items to track in the sketch
+ * @param mutableAggBufferOffset the offset for mutable aggregation buffer
+ * @param inputAggBufferOffset the offset for input aggregation buffer
+ */
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+ usage = """
+ _FUNC_(expr, maxItemsTracked) - Accumulates items into a sketch.
+ `maxItemsTracked` An optional positive INTEGER literal with upper limit of 1000000. If maxItemsTracked is not specified, it defaults to 10000.
+ """,
+ examples = """
+ Examples:
+ > SELECT approx_top_k_estimate(_FUNC_(expr)) FROM VALUES (0), (0), (1), (1), (2), (3), (4), (4) AS tab(expr);
+ [{"item":0,"count":2},{"item":4,"count":2},{"item":1,"count":2},{"item":2,"count":1},{"item":3,"count":1}]
+
+ > SELECT approx_top_k_estimate(_FUNC_(expr, 100), 2) FROM VALUES 'a', 'b', 'c', 'c', 'c', 'c', 'd', 'd' AS tab(expr);
+ [{"item":"c","count":4},{"item":"d","count":2}]
+ """,
+ group = "agg_funcs",
+ since = "4.1.0")
+// scalastyle:on line.size.limit
+case class ApproxTopKAccumulate(
+ expr: Expression,
+ maxItemsTracked: Expression,
+ mutableAggBufferOffset: Int = 0,
+ inputAggBufferOffset: Int = 0)
+ extends TypedImperativeAggregate[ItemsSketch[Any]]
+ with ImplicitCastInputTypes
+ with BinaryLike[Expression] {
+
+ def this(child: Expression, maxItemsTracked: Expression) = this(child, maxItemsTracked, 0, 0)
+
+ def this(child: Expression, maxItemsTracked: Int) = this(child, Literal(maxItemsTracked), 0, 0)
+
+ def this(child: Expression) = this(child, Literal(ApproxTopK.DEFAULT_MAX_ITEMS_TRACKED), 0, 0)
+
+ private lazy val itemDataType: DataType = expr.dataType
+
+ private lazy val maxItemsTrackedVal: Int = {
+ ApproxTopK.checkExpressionNotNull(maxItemsTracked, "maxItemsTracked")
+ val maxItemsTrackedVal = maxItemsTracked.eval().asInstanceOf[Int]
+ ApproxTopK.checkMaxItemsTracked(maxItemsTrackedVal)
+ maxItemsTrackedVal
+ }
+
+ override def left: Expression = expr
+
+ override def right: Expression = maxItemsTracked
+
+ override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType, IntegerType)
+
+ override def checkInputDataTypes(): TypeCheckResult = {
+ val defaultCheck = super.checkInputDataTypes()
+ if (defaultCheck.isFailure) {
+ defaultCheck
+ } else if (!ApproxTopK.isDataTypeSupported(itemDataType)) {
+ TypeCheckFailure(f"${itemDataType.typeName} columns are not supported")
+ } else if (!maxItemsTracked.foldable) {
+ TypeCheckFailure("Number of items tracked must be a constant literal")
+ } else {
+ TypeCheckSuccess
+ }
+ }
+
+ override def dataType: DataType = ApproxTopK.getSketchStateDataType(itemDataType)
+
+ override def createAggregationBuffer(): ItemsSketch[Any] = {
+ val maxMapSize = ApproxTopK.calMaxMapSize(maxItemsTrackedVal)
+ ApproxTopK.createAggregationBuffer(expr, maxMapSize)
+ }
+
+ override def update(buffer: ItemsSketch[Any], input: InternalRow): ItemsSketch[Any] =
+ ApproxTopK.updateSketchBuffer(expr, buffer, input)
+
+ override def merge(buffer: ItemsSketch[Any], input: ItemsSketch[Any]): ItemsSketch[Any] =
+ buffer.merge(input)
+
+ override def eval(buffer: ItemsSketch[Any]): Any = {
+ val sketchBytes = serialize(buffer)
+ InternalRow.apply(sketchBytes, null, maxItemsTrackedVal)
+ }
+
+ override def serialize(buffer: ItemsSketch[Any]): Array[Byte] =
+ buffer.toByteArray(ApproxTopK.genSketchSerDe(itemDataType))
+
+ override def deserialize(storageFormat: Array[Byte]): ItemsSketch[Any] =
+ ItemsSketch.getInstance(Memory.wrap(storageFormat), ApproxTopK.genSketchSerDe(itemDataType))
+
+ override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate =
+ copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+ override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate =
+ copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+ override protected def withNewChildrenInternal(
+ newLeft: Expression,
+ newRight: Expression): Expression =
+ copy(expr = newLeft, maxItemsTracked = newRight)
+
+ override def nullable: Boolean = false
+
+ override def prettyName: String =
+ getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("approx_top_k_accumulate")
+}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
index f9e8b6a17896f..c98e988ad3079 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
@@ -55,6 +55,8 @@ case class UnaryMinus(
override def dataType: DataType = child.dataType
+ override def contextIndependentFoldable: Boolean = child.contextIndependentFoldable
+
override def toString: String = s"-$child"
private lazy val numeric = TypeUtils.getNumeric(dataType, failOnError)
@@ -123,6 +125,8 @@ case class UnaryPositive(child: Expression)
override def dataType: DataType = child.dataType
+ override def contextIndependentFoldable: Boolean = child.contextIndependentFoldable
+
override def sql: String = s"(+ ${child.sql})"
override lazy val replacement: Expression = child
@@ -155,6 +159,8 @@ case class Abs(child: Expression, failOnError: Boolean = SQLConf.get.ansiEnabled
override def dataType: DataType = child.dataType
+ override def contextIndependentFoldable: Boolean = child.contextIndependentFoldable
+
private lazy val numeric = (dataType match {
case _: DayTimeIntervalType => LongExactNumeric
case _: YearMonthIntervalType => IntegerExactNumeric
@@ -187,6 +193,9 @@ case class Abs(child: Expression, failOnError: Boolean = SQLConf.get.ansiEnabled
abstract class BinaryArithmetic extends BinaryOperator with SupportQueryContext {
override def nullIntolerant: Boolean = true
+ override def contextIndependentFoldable: Boolean =
+ left.contextIndependentFoldable && right.contextIndependentFoldable
+
protected val evalMode: EvalMode.Value
private lazy val internalDataType: DataType = (left.dataType, right.dataType) match {
@@ -1193,6 +1202,7 @@ case class Least(children: Seq[Expression]) extends ComplexTypeMergingExpression
override def nullable: Boolean = children.forall(_.nullable)
override def foldable: Boolean = children.forall(_.foldable)
+ override def contextIndependentFoldable: Boolean = children.forall(_.contextIndependentFoldable)
private lazy val ordering = TypeUtils.getInterpretedOrdering(dataType)
@@ -1281,6 +1291,7 @@ case class Greatest(children: Seq[Expression]) extends ComplexTypeMergingExpress
override def nullable: Boolean = children.forall(_.nullable)
override def foldable: Boolean = children.forall(_.foldable)
+ override def contextIndependentFoldable: Boolean = children.forall(_.contextIndependentFoldable)
private lazy val ordering = TypeUtils.getInterpretedOrdering(dataType)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala
index 2ec31458270f3..4fb180a720129 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala
@@ -144,7 +144,7 @@ private class CodeFormatter {
// Lines starting with '}' should be de-indented even if they contain '{' after;
// in addition, lines ending with ':' are typically labels
val thisLineIndent = if (line.startsWith("}") || line.startsWith(")") || line.endsWith(":")) {
- " " * (indentSize * (indentLevel - 1))
+ " ".repeat(indentSize * (indentLevel - 1))
} else {
indentString
}
@@ -157,7 +157,7 @@ private class CodeFormatter {
}
code.append("\n")
indentLevel = newIndentLevel
- indentString = " " * (indentSize * newIndentLevel)
+ indentString = " ".repeat(indentSize * newIndentLevel)
currentLine += 1
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
index 2564d4eab9bd6..8c702815e9b9f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
@@ -33,7 +33,7 @@ import org.codehaus.janino.util.ClassFile.CodeAttribute
import org.apache.spark.{SparkException, SparkIllegalArgumentException, TaskContext, TaskKilledException}
import org.apache.spark.executor.InputMetrics
-import org.apache.spark.internal.{Logging, LogKeys, MDC}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.internal.LogKeys._
import org.apache.spark.metrics.source.CodegenMetrics
import org.apache.spark.sql.catalyst.InternalRow
@@ -1989,7 +1989,7 @@ object CodeGenerator extends Logging {
case ByteType => java.lang.Byte.TYPE
case ShortType => java.lang.Short.TYPE
case IntegerType | DateType | _: YearMonthIntervalType => java.lang.Integer.TYPE
- case LongType | TimestampType | TimestampNTZType | _: DayTimeIntervalType =>
+ case LongType | TimestampType | TimestampNTZType | _: DayTimeIntervalType | _: TimeType =>
java.lang.Long.TYPE
case FloatType => java.lang.Float.TYPE
case DoubleType => java.lang.Double.TYPE
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
index b4978fbe1f70a..59196a959f758 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
@@ -116,6 +116,7 @@ case class Size(child: Expression, legacySizeOfNull: Boolean)
def this(child: Expression) = this(child, SQLConf.get.legacySizeOfNull)
override def dataType: DataType = IntegerType
+ override def contextIndependentFoldable: Boolean = child.contextIndependentFoldable
override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(ArrayType, MapType))
override def nullable: Boolean = if (legacySizeOfNull) false else super.nullable
@@ -1003,7 +1004,7 @@ case class MapSort(base: Expression)
| ${CodeGenerator.getValue(values, valueType, i)});
|}
|
- |java.util.Arrays.sort($sortArray, new java.util.Comparator() {
+ |java.util.Arrays.parallelSort($sortArray, new java.util.Comparator() {
| @Override public int compare(Object $o1entry, Object $o2entry) {
| Object $o1 = (($simpleEntryType) $o1entry).getKey();
| Object $o2 = (($simpleEntryType) $o2entry).getKey();
@@ -1148,7 +1149,7 @@ case class SortArray(base: Expression, ascendingOrder: Expression)
private def sortEval(array: Any, ascending: Boolean): Any = {
val data = array.asInstanceOf[ArrayData].toArray[AnyRef](elementType)
if (elementType != NullType) {
- java.util.Arrays.sort(data, if (ascending) lt else gt)
+ java.util.Arrays.parallelSort(data, if (ascending) lt else gt)
}
new GenericArrayData(data.asInstanceOf[Array[Any]])
}
@@ -1190,7 +1191,7 @@ case class SortArray(base: Expression, ascendingOrder: Expression)
s"""
|if ($order) {
| $javaType[] $array = $base.to${primitiveTypeName}Array();
- | java.util.Arrays.sort($array);
+ | java.util.Arrays.parallelSort($array);
| ${ev.value} = $unsafeArrayData.fromPrimitiveArray($array);
|} else
""".stripMargin
@@ -1202,7 +1203,7 @@ case class SortArray(base: Expression, ascendingOrder: Expression)
|{
| Object[] $array = $base.toObjectArray($elementTypeTerm);
| final int $sortOrder = $order ? 1 : -1;
- | java.util.Arrays.sort($array, new java.util.Comparator() {
+ | java.util.Arrays.parallelSort($array, new java.util.Comparator() {
| @Override public int compare(Object $o1, Object $o2) {
| if ($o1 == null && $o2 == null) {
| return 0;
@@ -2158,6 +2159,8 @@ case class ArrayJoin(
override def foldable: Boolean = children.forall(_.foldable)
+ override def contextIndependentFoldable: Boolean = children.forall(_.contextIndependentFoldable)
+
override def eval(input: InternalRow): Any = {
val arrayEval = array.eval(input)
if (arrayEval == null) return null
@@ -2897,6 +2900,8 @@ case class Concat(children: Seq[Expression]) extends ComplexTypeMergingExpressio
override def foldable: Boolean = children.forall(_.foldable)
+ override def contextIndependentFoldable: Boolean = children.forall(_.contextIndependentFoldable)
+
override def eval(input: InternalRow): Any = doConcat(input)
@transient private lazy val doConcat: InternalRow => Any = dataType match {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
index 81484f8dd7da5..f4232cae1066d 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
@@ -71,6 +71,8 @@ case class CreateArray(children: Seq[Expression], useStringTypeWhenEmpty: Boolea
override def foldable: Boolean = children.forall(_.foldable)
+ override def contextIndependentFoldable: Boolean = children.forall(_.contextIndependentFoldable)
+
override def stringArgs: Iterator[Any] = super.stringArgs.take(1)
override def checkInputDataTypes(): TypeCheckResult = {
@@ -204,6 +206,8 @@ case class CreateMap(children: Seq[Expression], useStringTypeWhenEmpty: Boolean)
override def foldable: Boolean = children.forall(_.foldable)
+ override def contextIndependentFoldable: Boolean = children.forall(_.contextIndependentFoldable)
+
override def stringArgs: Iterator[Any] = super.stringArgs.take(1)
override def checkInputDataTypes(): TypeCheckResult = {
@@ -450,6 +454,8 @@ case class CreateNamedStruct(children: Seq[Expression]) extends Expression with
override def foldable: Boolean = valExprs.forall(_.foldable)
+ override def contextIndependentFoldable: Boolean = children.forall(_.contextIndependentFoldable)
+
final override val nodePatterns: Seq[TreePattern] = Seq(CREATE_NAMED_STRUCT)
override lazy val dataType: StructType = {
@@ -591,18 +597,21 @@ case class StringToMap(text: Expression, pairDelim: Expression, keyValueDelim: E
private final lazy val collationId: Int = text.dataType.asInstanceOf[StringType].collationId
+ private lazy val legacySplitTruncate =
+ SQLConf.get.getConf(SQLConf.LEGACY_TRUNCATE_FOR_EMPTY_REGEX_SPLIT)
+
override def nullSafeEval(
inputString: Any,
stringDelimiter: Any,
keyValueDelimiter: Any): Any = {
val keyValues = CollationAwareUTF8String.splitSQL(inputString.asInstanceOf[UTF8String],
- stringDelimiter.asInstanceOf[UTF8String], -1, collationId)
+ stringDelimiter.asInstanceOf[UTF8String], -1, collationId, legacySplitTruncate)
val keyValueDelimiterUTF8String = keyValueDelimiter.asInstanceOf[UTF8String]
var i = 0
while (i < keyValues.length) {
val keyValueArray = CollationAwareUTF8String.splitSQL(
- keyValues(i), keyValueDelimiterUTF8String, 2, collationId)
+ keyValues(i), keyValueDelimiterUTF8String, 2, collationId, legacySplitTruncate)
val key = keyValueArray(0)
val value = if (keyValueArray.length < 2) null else keyValueArray(1)
mapBuilder.put(key, value)
@@ -617,9 +626,11 @@ case class StringToMap(text: Expression, pairDelim: Expression, keyValueDelim: E
nullSafeCodeGen(ctx, ev, (text, pd, kvd) =>
s"""
- |UTF8String[] $keyValues = CollationAwareUTF8String.splitSQL($text, $pd, -1, $collationId);
+ |UTF8String[] $keyValues =
+ | CollationAwareUTF8String.splitSQL($text, $pd, -1, $collationId, $legacySplitTruncate);
|for(UTF8String kvEntry: $keyValues) {
- | UTF8String[] kv = CollationAwareUTF8String.splitSQL(kvEntry, $kvd, 2, $collationId);
+ | UTF8String[] kv = CollationAwareUTF8String.splitSQL(
+ | kvEntry, $kvd, 2, $collationId, $legacySplitTruncate);
| $builderTerm.put(kv[0], kv.length == 2 ? kv[1] : null);
|}
|${ev.value} = $builderTerm.build();
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala
index 804c80bd68b49..5cdbdf3f0e7c7 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala
@@ -35,6 +35,27 @@ import org.apache.spark.sql.types._
object ExtractValue {
+ /**
+ * Returns the resolved `ExtractValue` using the `extractValue` method call. In case the method
+ * returns `None`, it throws.
+ *
+ * See `extractValue` doc for more info.
+ */
+ def apply(
+ child: Expression,
+ extraction: Expression,
+ resolver: Resolver): Expression = {
+ extractValue(child, extraction, resolver) match {
+ case Some(expression) => expression
+ case None =>
+ throw QueryCompilationErrors.dataTypeUnsupportedByExtractValueError(
+ child.dataType,
+ extraction,
+ child
+ )
+ }
+ }
+
/**
* Returns the resolved `ExtractValue`. It will return one kind of concrete `ExtractValue`,
* depend on the type of `child` and `extraction`.
@@ -46,33 +67,52 @@ object ExtractValue {
* Array | Integral type | GetArrayItem
* Map | map key type | GetMapValue
*/
- def apply(
+ def extractValue(
child: Expression,
extraction: Expression,
- resolver: Resolver): Expression = {
-
+ resolver: Resolver): Option[Expression] = {
(child.dataType, extraction) match {
case (StructType(fields), NonNullLiteral(v, _: StringType)) =>
val fieldName = v.toString
val ordinal = findField(fields, fieldName, resolver)
- GetStructField(child, ordinal, Some(fieldName))
+ Some(GetStructField(child, ordinal, Some(fieldName)))
case (ArrayType(StructType(fields), containsNull), NonNullLiteral(v, _: StringType)) =>
val fieldName = v.toString
val ordinal = findField(fields, fieldName, resolver)
- GetArrayStructFields(child, fields(ordinal).copy(name = fieldName),
- ordinal, fields.length, containsNull || fields(ordinal).nullable)
+ Some(
+ GetArrayStructFields(
+ child,
+ fields(ordinal).copy(name = fieldName),
+ ordinal,
+ fields.length,
+ containsNull || fields(ordinal).nullable
+ )
+ )
- case (_: ArrayType, _) => GetArrayItem(child, extraction)
+ case (_: ArrayType, _) => Some(GetArrayItem(child, extraction))
- case (MapType(_, _, _), _) => GetMapValue(child, extraction)
+ case (MapType(_, _, _), _) => Some(GetMapValue(child, extraction))
- case (otherType, _) =>
- throw QueryCompilationErrors.dataTypeUnsupportedByExtractValueError(
- otherType, extraction, child)
+ case (otherType, _) => None
}
}
+ /**
+ * Check that [[attribute]] can be fully extracted using the given [[nestedFields]].
+ */
+ def isExtractable(
+ attribute: Attribute, nestedFields: Seq[String], resolver: Resolver): Boolean = {
+ nestedFields
+ .foldLeft(Some(attribute): Option[Expression]) {
+ case (Some(expression), field) =>
+ ExtractValue.extractValue(expression, Literal(field), resolver)
+ case _ =>
+ None
+ }
+ .isDefined
+ }
+
/**
* Find the ordinal of StructField, report error if no desired field or over one
* desired fields are found.
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala
index 609d457fbd067..621f02ca18b86 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala
@@ -200,6 +200,8 @@ case class CaseWhen(
branches.exists(_._2.nullable) || elseValue.map(_.nullable).getOrElse(true)
}
+ override def contextIndependentFoldable: Boolean = children.forall(_.contextIndependentFoldable)
+
override def checkInputDataTypes(): TypeCheckResult = {
if (TypeCoercion.haveSameType(inputTypesForMerging)) {
// Make sure all branch conditions are boolean types.
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
index 4ae8883dae043..375b58d8bb7e2 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
@@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.{ExpressionBuilder, FunctionRegistry}
import org.apache.spark.sql.catalyst.expressions.codegen._
import org.apache.spark.sql.catalyst.expressions.codegen.Block._
+import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke
import org.apache.spark.sql.catalyst.trees.CurrentOrigin.withOrigin
import org.apache.spark.sql.catalyst.trees.TreePattern._
import org.apache.spark.sql.catalyst.util.{DateTimeUtils, LegacyDateFormats, TimestampFormatter}
@@ -323,6 +324,7 @@ case class DateAdd(startDate: Expression, days: Expression)
override def nullIntolerant: Boolean = true
override def left: Expression = startDate
override def right: Expression = days
+ override def contextIndependentFoldable: Boolean = children.forall(_.contextIndependentFoldable)
override def inputTypes: Seq[AbstractDataType] =
Seq(DateType, TypeCollection(IntegerType, ShortType, ByteType))
@@ -362,6 +364,7 @@ case class DateSub(startDate: Expression, days: Expression)
override def nullIntolerant: Boolean = true
override def left: Expression = startDate
override def right: Expression = days
+ override def contextIndependentFoldable: Boolean = children.forall(_.contextIndependentFoldable)
override def inputTypes: Seq[AbstractDataType] =
Seq(DateType, TypeCollection(IntegerType, ShortType, ByteType))
@@ -1646,7 +1649,10 @@ case class NextDay(
/**
* Adds an interval to timestamp.
*/
-case class TimeAdd(start: Expression, interval: Expression, timeZoneId: Option[String] = None)
+case class TimestampAddInterval(
+ start: Expression,
+ interval: Expression,
+ timeZoneId: Option[String] = None)
extends BinaryExpression with TimeZoneAwareExpression with ExpectsInputTypes {
override def nullIntolerant: Boolean = true
@@ -1689,7 +1695,7 @@ case class TimeAdd(start: Expression, interval: Expression, timeZoneId: Option[S
}
override protected def withNewChildrenInternal(
- newLeft: Expression, newRight: Expression): TimeAdd =
+ newLeft: Expression, newRight: Expression): TimestampAddInterval =
copy(start = newLeft, interval = newRight)
}
@@ -2119,7 +2125,7 @@ case class ParseToDate(
this(left, None)
}
- override def prettyName: String = "to_date"
+ override def nodeName: String = "to_date"
override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression =
copy(timeZoneId = Some(timeZoneId))
@@ -2149,6 +2155,45 @@ case class ParseToDate(
}
}
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+ usage = """
+ _FUNC_(date_str[, fmt]) - Parses the `date_str` expression with the `fmt` expression to a date.
+ The function always returns null on an invalid input with/without ANSI SQL mode enabled. By
+ default, it follows casting rules to a date if the `fmt` is omitted.
+ """,
+ arguments = """
+ Arguments:
+ * date_str - A string to be parsed to date.
+ * fmt - Date format pattern to follow. See Datetime Patterns for valid
+ date and time format patterns.
+ """,
+ examples = """
+ Examples:
+ > SELECT _FUNC_('2016-12-31');
+ 2016-12-31
+ > SELECT _FUNC_('2016-12-31', 'yyyy-MM-dd');
+ 2016-12-31
+ > SELECT _FUNC_('foo', 'yyyy-MM-dd');
+ NULL
+ """,
+ group = "datetime_funcs",
+ since = "4.0.0")
+// scalastyle:on line.size.limit
+object TryToDateExpressionBuilder extends ExpressionBuilder {
+ override def build(funcName: String, expressions: Seq[Expression]): Expression = {
+ val numArgs = expressions.length
+ if (numArgs == 1 || numArgs == 2) {
+ ParseToDate(
+ expressions.head,
+ expressions.drop(1).lastOption,
+ ansiEnabled = false)
+ } else {
+ throw QueryCompilationErrors.wrongNumArgsError(funcName, Seq(1, 2), numArgs)
+ }
+ }
+}
+
/**
* Parses a column to a timestamp based on the supplied format.
*/
@@ -2554,9 +2599,36 @@ case class MakeDate(
copy(year = newFirst, month = newSecond, day = newThird)
}
+case class MakeTimestampNTZ(left: Expression, right: Expression)
+ extends BinaryExpression
+ with RuntimeReplaceable
+ with ExpectsInputTypes {
+
+ override def replacement: Expression = StaticInvoke(
+ classOf[DateTimeUtils.type],
+ TimestampNTZType,
+ "makeTimestampNTZ",
+ Seq(left, right),
+ Seq(left.dataType, right.dataType)
+ )
+
+ override def inputTypes: Seq[AbstractDataType] = Seq(DateType, AnyTimeType)
+
+ override def prettyName: String = "make_timestamp_ntz"
+
+ override protected def withNewChildrenInternal(
+ newLeft: Expression, newRight: Expression): Expression = {
+ copy(left = newLeft, right = newRight)
+ }
+}
+
// scalastyle:off line.size.limit
@ExpressionDescription(
- usage = "_FUNC_(year, month, day, hour, min, sec) - Create local date-time from year, month, day, hour, min, sec fields. If the configuration `spark.sql.ansi.enabled` is false, the function returns NULL on invalid inputs. Otherwise, it will throw an error instead.",
+ usage = """
+ _FUNC_(year, month, day, hour, min, sec) - Create local date-time from year, month, day, hour, min, sec fields. If the configuration `spark.sql.ansi.enabled` is false, the function returns NULL on invalid inputs. Otherwise, it will throw an error instead.
+
+ _FUNC_(date, time) - Create a local date-time from date and time fields.
+ """,
arguments = """
Arguments:
* year - the year to represent, from 1 to 9999
@@ -2567,6 +2639,8 @@ case class MakeDate(
* sec - the second-of-minute and its micro-fraction to represent, from
0 to 60. If the sec argument equals to 60, the seconds field is set
to 0 and 1 minute is added to the final timestamp.
+ * date - a date to represent, from 0001-01-01 to 9999-12-31
+ * time - a local time to represent, from 00:00:00 to 23:59:59.999999
""",
examples = """
Examples:
@@ -2576,6 +2650,8 @@ case class MakeDate(
2019-07-01 00:00:00
> SELECT _FUNC_(null, 7, 22, 15, 30, 0);
NULL
+ > SELECT _FUNC_(DATE'2014-12-28', TIME'6:30:45.887');
+ 2014-12-28 06:30:45.887
""",
group = "datetime_funcs",
since = "3.4.0")
@@ -2583,7 +2659,9 @@ case class MakeDate(
object MakeTimestampNTZExpressionBuilder extends ExpressionBuilder {
override def build(funcName: String, expressions: Seq[Expression]): Expression = {
val numArgs = expressions.length
- if (numArgs == 6) {
+ if (numArgs == 2) {
+ MakeTimestampNTZ(expressions(0), expressions(1))
+ } else if (numArgs == 6) {
MakeTimestamp(
expressions(0),
expressions(1),
@@ -2593,14 +2671,18 @@ object MakeTimestampNTZExpressionBuilder extends ExpressionBuilder {
expressions(5),
dataType = TimestampNTZType)
} else {
- throw QueryCompilationErrors.wrongNumArgsError(funcName, Seq(6), numArgs)
+ throw QueryCompilationErrors.wrongNumArgsError(funcName, Seq(2, 6), numArgs)
}
}
}
// scalastyle:off line.size.limit
@ExpressionDescription(
- usage = "_FUNC_(year, month, day, hour, min, sec) - Try to create local date-time from year, month, day, hour, min, sec fields. The function returns NULL on invalid inputs.",
+ usage = """
+ _FUNC_(year, month, day, hour, min, sec) - Try to create local date-time from year, month, day, hour, min, sec fields. The function returns NULL on invalid inputs.
+
+ _FUNC_(date, time) - Create a local date-time from date and time fields.
+ """,
arguments = """
Arguments:
* year - the year to represent, from 1 to 9999
@@ -2611,6 +2693,8 @@ object MakeTimestampNTZExpressionBuilder extends ExpressionBuilder {
* sec - the second-of-minute and its micro-fraction to represent, from
0 to 60. If the sec argument equals to 60, the seconds field is set
to 0 and 1 minute is added to the final timestamp.
+ * date - a date to represent, from 0001-01-01 to 9999-12-31
+ * time - a local time to represent, from 00:00:00 to 23:59:59.999999
""",
examples = """
Examples:
@@ -2622,6 +2706,8 @@ object MakeTimestampNTZExpressionBuilder extends ExpressionBuilder {
NULL
> SELECT _FUNC_(2024, 13, 22, 15, 30, 0);
NULL
+ > SELECT _FUNC_(DATE'2014-12-28', TIME'6:30:45.887');
+ 2014-12-28 06:30:45.887
""",
group = "datetime_funcs",
since = "4.0.0")
@@ -2629,7 +2715,9 @@ object MakeTimestampNTZExpressionBuilder extends ExpressionBuilder {
object TryMakeTimestampNTZExpressionBuilder extends ExpressionBuilder {
override def build(funcName: String, expressions: Seq[Expression]): Expression = {
val numArgs = expressions.length
- if (numArgs == 6) {
+ if (numArgs == 2) {
+ TryEval(MakeTimestampNTZ(expressions(0), expressions(1)))
+ } else if (numArgs == 6) {
MakeTimestamp(
expressions(0),
expressions(1),
@@ -2647,7 +2735,11 @@ object TryMakeTimestampNTZExpressionBuilder extends ExpressionBuilder {
// scalastyle:off line.size.limit
@ExpressionDescription(
- usage = "_FUNC_(year, month, day, hour, min, sec[, timezone]) - Create the current timestamp with local time zone from year, month, day, hour, min, sec and timezone fields. If the configuration `spark.sql.ansi.enabled` is false, the function returns NULL on invalid inputs. Otherwise, it will throw an error instead.",
+ usage = """
+ _FUNC_(year, month, day, hour, min, sec[, timezone]) - Create the current timestamp with local time zone from year, month, day, hour, min, sec and (optional) timezone fields. If the configuration `spark.sql.ansi.enabled` is false, the function returns NULL on invalid inputs. Otherwise, it will throw an error instead.
+
+ _FUNC_(date, time[, timezone]) - Create a local date-time from date, time and (optional) timezone fields.
+ """,
arguments = """
Arguments:
* year - the year to represent, from 1 to 9999
@@ -2659,6 +2751,8 @@ object TryMakeTimestampNTZExpressionBuilder extends ExpressionBuilder {
0 to 60. If the sec argument equals to 60, the seconds field is set
to 0 and 1 minute is added to the final timestamp.
* timezone - the time zone identifier. For example, CET, UTC and etc.
+ * date - a date to represent, from 0001-01-01 to 9999-12-31
+ * time - a local time to represent, from 00:00:00 to 23:59:59.999999
""",
examples = """
Examples:
@@ -2670,6 +2764,10 @@ object TryMakeTimestampNTZExpressionBuilder extends ExpressionBuilder {
2019-07-01 00:00:00
> SELECT _FUNC_(null, 7, 22, 15, 30, 0);
NULL
+ > SELECT _FUNC_(DATE'2014-12-28', TIME'6:30:45.887');
+ 2014-12-28 06:30:45.887
+ > SELECT _FUNC_(DATE'2014-12-28', TIME'6:30:45.887', 'CET');
+ 2014-12-27 21:30:45.887
""",
group = "datetime_funcs",
since = "3.4.0")
@@ -2677,7 +2775,16 @@ object TryMakeTimestampNTZExpressionBuilder extends ExpressionBuilder {
object MakeTimestampLTZExpressionBuilder extends ExpressionBuilder {
override def build(funcName: String, expressions: Seq[Expression]): Expression = {
val numArgs = expressions.length
- if (numArgs == 6 || numArgs == 7) {
+ if (numArgs == 2 || numArgs == 3) {
+ // Overload for: date, time[, timezone].
+ MakeTimestampFromDateTime(
+ expressions(0),
+ Some(expressions(1)),
+ expressions.drop(2).lastOption
+ )
+ }
+ else if (numArgs == 6 || numArgs == 7) {
+ // Overload for: year, month, day, hour, min, sec[, timezone].
MakeTimestamp(
expressions(0),
expressions(1),
@@ -2688,7 +2795,7 @@ object MakeTimestampLTZExpressionBuilder extends ExpressionBuilder {
expressions.drop(6).lastOption,
dataType = TimestampType)
} else {
- throw QueryCompilationErrors.wrongNumArgsError(funcName, Seq(6), numArgs)
+ throw QueryCompilationErrors.wrongNumArgsError(funcName, Seq(2, 6), numArgs)
}
}
}
@@ -2744,38 +2851,6 @@ object TryMakeTimestampLTZExpressionBuilder extends ExpressionBuilder {
}
}
-// scalastyle:off line.size.limit
-@ExpressionDescription(
- usage = "_FUNC_(year, month, day, hour, min, sec[, timezone]) - Create timestamp from year, month, day, hour, min, sec and timezone fields. The result data type is consistent with the value of configuration `spark.sql.timestampType`. If the configuration `spark.sql.ansi.enabled` is false, the function returns NULL on invalid inputs. Otherwise, it will throw an error instead.",
- arguments = """
- Arguments:
- * year - the year to represent, from 1 to 9999
- * month - the month-of-year to represent, from 1 (January) to 12 (December)
- * day - the day-of-month to represent, from 1 to 31
- * hour - the hour-of-day to represent, from 0 to 23
- * min - the minute-of-hour to represent, from 0 to 59
- * sec - the second-of-minute and its micro-fraction to represent, from 0 to 60.
- The value can be either an integer like 13 , or a fraction like 13.123.
- If the sec argument equals to 60, the seconds field is set
- to 0 and 1 minute is added to the final timestamp.
- * timezone - the time zone identifier. For example, CET, UTC and etc.
- """,
- examples = """
- Examples:
- > SELECT _FUNC_(2014, 12, 28, 6, 30, 45.887);
- 2014-12-28 06:30:45.887
- > SELECT _FUNC_(2014, 12, 28, 6, 30, 45.887, 'CET');
- 2014-12-27 21:30:45.887
- > SELECT _FUNC_(2019, 6, 30, 23, 59, 60);
- 2019-07-01 00:00:00
- > SELECT _FUNC_(2019, 6, 30, 23, 59, 1);
- 2019-06-30 23:59:01
- > SELECT _FUNC_(null, 7, 22, 15, 30, 0);
- NULL
- """,
- group = "datetime_funcs",
- since = "3.0.0")
-// scalastyle:on line.size.limit
case class MakeTimestamp(
year: Expression,
month: Expression,
@@ -3030,6 +3105,147 @@ case class TryMakeTimestamp(
}
}
+case class MakeTimestampFromDateTime(
+ date: Expression,
+ time: Option[Expression] = None,
+ timezone: Option[Expression] = None,
+ timeZoneId: Option[String] = None)
+ extends Expression with RuntimeReplaceable with ExpectsInputTypes with TimeZoneAwareExpression {
+
+ def this(date: Expression) =
+ this(date, None, None, None)
+
+ def this(date: Expression, time: Expression) =
+ this(date, Some(time), None, None)
+
+ def this(date: Expression, time: Expression, timezone: Expression) =
+ this(date, Some(time), Some(timezone), None)
+
+ override def children: Seq[Expression] = Seq(date) ++ time ++ timezone
+
+ override def inputTypes: Seq[AbstractDataType] = Seq(DateType, AnyTimeType) ++
+ timezone.map(_ => StringTypeWithCollation(supportsTrimCollation = true))
+
+ override def replacement: Expression = {
+ // If time is not provided, we use midnight, i.e. 00:00:00.
+ val timeExpr = time.getOrElse(Literal(0L, TimeType(0)))
+ // If timezone is not provided, we use the session timezone.
+ val zoneIdExpr = timezone.getOrElse(Literal(timeZoneId.get))
+ StaticInvoke(
+ classOf[DateTimeUtils.type],
+ TimestampType,
+ "makeTimestamp",
+ Seq(date, timeExpr, zoneIdExpr),
+ Seq(date.dataType, timeExpr.dataType, zoneIdExpr.dataType),
+ returnNullable = children.exists(_.nullable)
+ )
+ }
+
+ override def prettyName: String = "make_timestamp"
+
+ override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression =
+ copy(timeZoneId = Option(timeZoneId))
+
+ override def nodePatternsInternal(): Seq[TreePattern] = Seq(RUNTIME_REPLACEABLE)
+
+ override protected def withNewChildrenInternal(
+ newChildren: IndexedSeq[Expression]): Expression = {
+ val timeOpt = if (time.isDefined) Some(newChildren(1)) else None
+ val timezoneOpt = if (timezone.isDefined) Some(newChildren(2)) else None
+ copy(date = newChildren(0), time = timeOpt, timezone = timezoneOpt)
+ }
+}
+
+/**
+ * This make_timestamp expression builder creates a timestamp from:
+ * - a year, month, day, hour, min, sec and optional timezone, or
+ * - a date, optional time, and optional timezone.
+ */
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+ usage = """
+ _FUNC_(year, month, day, hour, min, sec[, timezone]) - Create the current timestamp with local time zone from year, month, day, hour, min, sec and timezone fields. If the configuration `spark.sql.ansi.enabled` is false, the function returns NULL on invalid inputs. Otherwise, it will throw an error instead.
+
+ _FUNC_(date[, time[, timezone]]) - Create timestamp from date and time fields.
+ """,
+ arguments = """
+ Arguments:
+ * year - the year to represent, from 1 to 9999
+ * month - the month-of-year to represent, from 1 (January) to 12 (December)
+ * day - the day-of-month to represent, from 1 to 31
+ * hour - the hour-of-day to represent, from 0 to 23
+ * min - the minute-of-hour to represent, from 0 to 59
+ * sec - the second-of-minute and its micro-fraction to represent, from
+ 0 to 60. If the sec argument equals to 60, the seconds field is set
+ to 0 and 1 minute is added to the final timestamp.
+ * date - a date expression
+ * time - a time expression (optional). Default is 00:00:00.
+ * timezone - the time zone identifier (optional). For example, CET, UTC and etc.
+ """,
+ examples = """
+ Examples:
+ > SELECT _FUNC_(2014, 12, 28, 6, 30, 45.887);
+ 2014-12-28 06:30:45.887
+ > SELECT _FUNC_(2014, 12, 28, 6, 30, 45.887, 'CET');
+ 2014-12-27 21:30:45.887
+ > SELECT _FUNC_(DATE'2014-12-28');
+ 2014-12-28 00:00:00
+ > SELECT _FUNC_(DATE'2014-12-28', TIME'6:30:45.887');
+ 2014-12-28 06:30:45.887
+ > SELECT _FUNC_(DATE'2014-12-28', TIME'6:30:45.887', 'CET');
+ 2014-12-27 21:30:45.887
+ """,
+ group = "datetime_funcs",
+ since = "3.0.0")
+// scalastyle:on line.size.limit
+object MakeTimestampExpressionBuilder extends ExpressionBuilder {
+ override def build(funcName: String, expressions: Seq[Expression]): Expression = {
+ val numArgs = expressions.length
+ if (numArgs == 1) {
+ // date
+ MakeTimestampFromDateTime(
+ expressions(0)
+ )
+ } else if (numArgs == 2) {
+ // date, time
+ MakeTimestampFromDateTime(
+ expressions(0),
+ Some(expressions(1))
+ )
+ } else if (numArgs == 3) {
+ // date, time, timezone
+ MakeTimestampFromDateTime(
+ expressions(0),
+ Some(expressions(1)),
+ Some(expressions(2))
+ )
+ } else if (numArgs == 6) {
+ // year, month, day, hour, min, sec
+ MakeTimestamp(
+ expressions(0),
+ expressions(1),
+ expressions(2),
+ expressions(3),
+ expressions(4),
+ expressions(5)
+ )
+ } else if (numArgs == 7) {
+ // year, month, day, hour, min, sec, timezone
+ MakeTimestamp(
+ expressions(0),
+ expressions(1),
+ expressions(2),
+ expressions(3),
+ expressions(4),
+ expressions(5),
+ Some(expressions(6))
+ )
+ } else {
+ throw QueryCompilationErrors.wrongNumArgsError(funcName, Seq(1, 2, 3, 6, 7), numArgs)
+ }
+ }
+}
+
object DatePart {
def parseExtractField(
@@ -3294,6 +3510,8 @@ case class SubtractDates(
if (legacyInterval) CalendarIntervalType else DayTimeIntervalType(DAY)
}
+ override def contextIndependentFoldable: Boolean = children.forall(_.contextIndependentFoldable)
+
@transient
private lazy val evalFunc: (Int, Int) => Any = if (legacyInterval) {
(leftDays: Int, rightDays: Int) => subtractDates(leftDays, rightDays)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala
index 7cb645e601d36..88e22a91a64a7 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala
@@ -38,6 +38,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeConstants._
import org.apache.spark.sql.errors.QueryCompilationErrors
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
+import org.apache.spark.sql.util.SchemaUtils
import org.apache.spark.unsafe.Platform
import org.apache.spark.unsafe.hash.Murmur3_x86_32
import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
@@ -68,6 +69,8 @@ case class Md5(child: Expression)
override def inputTypes: Seq[DataType] = Seq(BinaryType)
+ override def contextIndependentFoldable: Boolean = child.contextIndependentFoldable
+
protected override def nullSafeEval(input: Any): Any =
UTF8String.fromString(DigestUtils.md5Hex(input.asInstanceOf[Array[Byte]]))
@@ -112,6 +115,8 @@ case class Sha2(left: Expression, right: Expression)
override def inputTypes: Seq[DataType] = Seq(BinaryType, IntegerType)
+ override def contextIndependentFoldable: Boolean = children.forall(_.contextIndependentFoldable)
+
protected override def nullSafeEval(input1: Any, input2: Any): Any = {
val bitLength = input2.asInstanceOf[Int]
val input = input1.asInstanceOf[Array[Byte]]
@@ -178,6 +183,8 @@ case class Sha1(child: Expression)
override def inputTypes: Seq[DataType] = Seq(BinaryType)
+ override def contextIndependentFoldable: Boolean = child.contextIndependentFoldable
+
protected override def nullSafeEval(input: Any): Any =
UTF8String.fromString(DigestUtils.sha1Hex(input.asInstanceOf[Array[Byte]]))
@@ -211,6 +218,8 @@ case class Crc32(child: Expression)
override def inputTypes: Seq[DataType] = Seq(BinaryType)
+ override def contextIndependentFoldable: Boolean = child.contextIndependentFoldable
+
protected override def nullSafeEval(input: Any): Any = {
val checksum = new CRC32
checksum.update(input.asInstanceOf[Array[Byte]], 0, input.asInstanceOf[Array[Byte]].length)
@@ -273,8 +282,15 @@ abstract class HashExpression[E] extends Expression {
override def foldable: Boolean = children.forall(_.foldable)
+ override def contextIndependentFoldable: Boolean = children.forall(_.contextIndependentFoldable)
+
override def nullable: Boolean = false
+ protected def isCollationAware: Boolean
+
+ protected lazy val legacyCollationAwareHashing: Boolean =
+ SQLConf.get.getConf(SQLConf.COLLATION_AWARE_HASHING_ENABLED)
+
private def hasMapType(dt: DataType): Boolean = {
dt.existsRecursively(_.isInstanceOf[MapType])
}
@@ -429,14 +445,43 @@ abstract class HashExpression[E] extends Expression {
val numBytes = s"$input.numBytes()"
s"$result = $hasherClassName.hashUnsafeBytes($baseObject, $baseOffset, $numBytes, $result);"
} else {
- val stringHash = ctx.freshName("stringHash")
- s"""
- long $stringHash = CollationFactory.fetchCollation(${stringType.collationId})
- .hashFunction.applyAsLong($input);
- $result = $hasherClassName.hashLong($stringHash, $result);
- """
+ if (isCollationAware) {
+ val key = ctx.freshName("key")
+ val offset = "Platform.BYTE_ARRAY_OFFSET"
+ s"""
+ byte[] $key = (byte[]) CollationFactory.fetchCollation(${stringType.collationId})
+ .sortKeyFunction.apply($input);
+ $result = $hasherClassName.hashUnsafeBytes($key, $offset, $key.length, $result);
+ """
+ } else if (legacyCollationAwareHashing) {
+ val collation = CollationFactory.fetchCollation(stringType.collationId)
+ val stringHash = ctx.freshName("stringHash")
+ if (collation.isUtf8BinaryType || collation.isUtf8LcaseType) {
+ s"""
+ long $stringHash = UTF8String.fromBytes((byte[]) CollationFactory
+ .fetchCollation(${stringType.collationId}).sortKeyFunction.apply($input)).hashCode();
+ $result = $hasherClassName.hashLong($stringHash, $result);
+ """
+ } else if (collation.supportsSpaceTrimming) {
+ s"""
+ long $stringHash = CollationFactory.fetchCollation(${stringType.collationId})
+ .getCollator().getCollationKey($input.trimRight().toValidString()).hashCode();
+ $result = $hasherClassName.hashLong($stringHash, $result);
+ """
+ } else {
+ s"""
+ long $stringHash = CollationFactory.fetchCollation(${stringType.collationId})
+ .getCollator().getCollationKey($input.toValidString()).hashCode();
+ $result = $hasherClassName.hashLong($stringHash, $result);
+ """
+ }
+ } else {
+ val baseObject = s"$input.getBaseObject()"
+ val baseOffset = s"$input.getBaseOffset()"
+ val numBytes = s"$input.numBytes()"
+ s"$result = $hasherClassName.hashUnsafeBytes($baseObject, $baseOffset, $numBytes, $result);"
+ }
}
-
}
protected def genHashForMap(
@@ -546,10 +591,38 @@ abstract class InterpretedHashFunction {
protected def hashUnsafeBytes(base: AnyRef, offset: Long, length: Int, seed: Long): Long
/**
- * Computes hash of a given `value` of type `dataType`. The caller needs to check the validity
- * of input `value`.
+ * This method is intended for callers using the old hash API and preserves compatibility for
+ * supported data types. It must only be used for data types that do not include collated strings
+ * or complex types (e.g., structs, arrays, maps) that may contain collated strings.
+ *
+ * The caller is responsible for ensuring that `dataType` does not involve collation-aware fields.
+ * This is validated via an internal assertion.
+ *
+ * @throws IllegalArgumentException if `dataType` contains non-UTF8 binary collation.
*/
def hash(value: Any, dataType: DataType, seed: Long): Long = {
+ require(!SchemaUtils.hasNonUTF8BinaryCollation(dataType))
+ // For UTF8_BINARY, hashing behavior is the same regardless of the isCollationAware flag.
+ hash(
+ value = value,
+ dataType = dataType,
+ seed = seed,
+ isCollationAware = false,
+ legacyCollationAwareHashing = false)
+ }
+
+ /**
+ * Computes hash of a given `value` of type `dataType`. The caller needs to check the validity
+ * of input `value`. The `isCollationAware` boolean flag indicates whether hashing should take
+ * a string's collation into account. If not, the bytes of the string are hashed, otherwise the
+ * collation key of the string is hashed.
+ */
+ def hash(
+ value: Any,
+ dataType: DataType,
+ seed: Long,
+ isCollationAware: Boolean,
+ legacyCollationAwareHashing: Boolean): Long = {
value match {
case null => seed
case b: Boolean => hashInt(if (b) 1 else 0, seed)
@@ -575,12 +648,25 @@ abstract class InterpretedHashFunction {
case s: UTF8String =>
val st = dataType.asInstanceOf[StringType]
if (st.supportsBinaryEquality) {
- hashUnsafeBytes(s.getBaseObject, s.getBaseOffset, s.numBytes(), seed)
+ hashUnsafeBytes(s.getBaseObject, s.getBaseOffset, s.numBytes, seed)
} else {
- val stringHash = CollationFactory
- .fetchCollation(st.collationId)
- .hashFunction.applyAsLong(s)
- hashLong(stringHash, seed)
+ if (isCollationAware) {
+ val key = CollationFactory.fetchCollation(st.collationId).sortKeyFunction.apply(s)
+ .asInstanceOf[Array[Byte]]
+ hashUnsafeBytes(key, Platform.BYTE_ARRAY_OFFSET, key.length, seed)
+ } else if (legacyCollationAwareHashing) {
+ val collation = CollationFactory.fetchCollation(st.collationId)
+ val stringHash = if (collation.isUtf8BinaryType || collation.isUtf8LcaseType) {
+ UTF8String.fromBytes(collation.sortKeyFunction.apply(s)).hashCode
+ } else if (collation.supportsSpaceTrimming) {
+ collation.getCollator.getCollationKey(s.trimRight.toValidString).hashCode
+ } else {
+ collation.getCollator.getCollationKey(s.toValidString).hashCode
+ }
+ hashLong(stringHash, seed)
+ } else {
+ hashUnsafeBytes(s.getBaseObject, s.getBaseOffset, s.numBytes, seed)
+ }
}
case array: ArrayData =>
@@ -591,7 +677,12 @@ abstract class InterpretedHashFunction {
var result = seed
var i = 0
while (i < array.numElements()) {
- result = hash(array.get(i, elementType), elementType, result)
+ result = hash(
+ array.get(i, elementType),
+ elementType,
+ result,
+ isCollationAware,
+ legacyCollationAwareHashing)
i += 1
}
result
@@ -608,8 +699,18 @@ abstract class InterpretedHashFunction {
var result = seed
var i = 0
while (i < map.numElements()) {
- result = hash(keys.get(i, kt), kt, result)
- result = hash(values.get(i, vt), vt, result)
+ result = hash(
+ keys.get(i, kt),
+ kt,
+ result,
+ isCollationAware,
+ legacyCollationAwareHashing)
+ result = hash(
+ values.get(i, vt),
+ vt,
+ result,
+ isCollationAware,
+ legacyCollationAwareHashing)
i += 1
}
result
@@ -624,7 +725,12 @@ abstract class InterpretedHashFunction {
var i = 0
val len = struct.numFields
while (i < len) {
- result = hash(struct.get(i, types(i)), types(i), result)
+ result = hash(
+ struct.get(i, types(i)),
+ types(i),
+ result,
+ isCollationAware,
+ legacyCollationAwareHashing)
i += 1
}
result
@@ -656,8 +762,12 @@ case class Murmur3Hash(children: Seq[Expression], seed: Int) extends HashExpress
override protected def hasherClassName: String = classOf[Murmur3_x86_32].getName
+ override protected def isCollationAware: Boolean = false
+
override protected def computeHash(value: Any, dataType: DataType, seed: Int): Int = {
- Murmur3HashFunction.hash(value, dataType, seed).toInt
+ Murmur3HashFunction.hash(
+ value, dataType, seed, isCollationAware, legacyCollationAwareHashing
+ ).toInt
}
override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): Murmur3Hash =
@@ -678,6 +788,29 @@ object Murmur3HashFunction extends InterpretedHashFunction {
}
}
+case class CollationAwareMurmur3Hash(children: Seq[Expression], seed: Int)
+ extends HashExpression[Int]
+{
+ def this(arguments: Seq[Expression]) = this(arguments, 42)
+
+ override def dataType: DataType = IntegerType
+
+ override def prettyName: String = "collation_aware_hash"
+
+ override protected def hasherClassName: String = classOf[Murmur3_x86_32].getName
+
+ override protected def isCollationAware: Boolean = true
+
+ override protected def computeHash(value: Any, dataType: DataType, seed: Int): Int = {
+ Murmur3HashFunction.hash(
+ value, dataType, seed, isCollationAware, legacyCollationAwareHashing
+ ).toInt
+ }
+
+ override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]):
+ CollationAwareMurmur3Hash = copy(children = newChildren)
+}
+
/**
* A xxHash64 64-bit hash expression.
*/
@@ -700,8 +833,10 @@ case class XxHash64(children: Seq[Expression], seed: Long) extends HashExpressio
override protected def hasherClassName: String = classOf[XXH64].getName
+ override protected def isCollationAware: Boolean = false
+
override protected def computeHash(value: Any, dataType: DataType, seed: Long): Long = {
- XxHash64Function.hash(value, dataType, seed)
+ XxHash64Function.hash(value, dataType, seed, isCollationAware, legacyCollationAwareHashing)
}
override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): XxHash64 =
@@ -718,6 +853,28 @@ object XxHash64Function extends InterpretedHashFunction {
}
}
+case class CollationAwareXxHash64(children: Seq[Expression], seed: Long)
+ extends HashExpression[Long]
+{
+ def this(arguments: Seq[Expression]) = this(arguments, 42L)
+
+ override def dataType: DataType = LongType
+
+ override def prettyName: String = "collation_aware_xxhash64"
+
+ override protected def hasherClassName: String = classOf[XXH64].getName
+
+ override protected def isCollationAware: Boolean = true
+
+ override protected def computeHash(value: Any, dataType: DataType, seed: Long): Long = {
+ XxHash64Function.hash(
+ value, dataType, seed, isCollationAware, legacyCollationAwareHashing)
+ }
+
+ override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]):
+ CollationAwareXxHash64 = copy(children = newChildren)
+}
+
/**
* Simulates Hive's hashing function from Hive v1.2.1 at
* org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils#hashcode()
@@ -738,8 +895,12 @@ case class HiveHash(children: Seq[Expression]) extends HashExpression[Int] {
override protected def hasherClassName: String = classOf[HiveHasher].getName
+ override protected def isCollationAware: Boolean = true
+
override protected def computeHash(value: Any, dataType: DataType, seed: Int): Int = {
- HiveHashFunction.hash(value, dataType, this.seed).toInt
+ HiveHashFunction.hash(
+ value, dataType, this.seed, isCollationAware, legacyCollationAwareHashing
+ ).toInt
}
override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
@@ -825,17 +986,18 @@ case class HiveHash(children: Seq[Expression]) extends HashExpression[Int] {
override protected def genHashString(
ctx: CodegenContext, stringType: StringType, input: String, result: String): String = {
- if (stringType.supportsBinaryEquality) {
+ if (stringType.supportsBinaryEquality || !isCollationAware) {
val baseObject = s"$input.getBaseObject()"
val baseOffset = s"$input.getBaseOffset()"
val numBytes = s"$input.numBytes()"
s"$result = $hasherClassName.hashUnsafeBytes($baseObject, $baseOffset, $numBytes);"
} else {
- val stringHash = ctx.freshName("stringHash")
+ val key = ctx.freshName("key")
+ val offset = Platform.BYTE_ARRAY_OFFSET
s"""
- long $stringHash = CollationFactory.fetchCollation(${stringType.collationId})
- .hashFunction.applyAsLong($input);
- $result = $hasherClassName.hashLong($stringHash);
+ byte[] $key = (byte[]) CollationFactory.fetchCollation(${stringType.collationId})
+ .sortKeyFunction.apply($input);
+ $result = $hasherClassName.hashUnsafeBytes($key, $offset, $key.length, $result);
"""
}
}
@@ -1018,7 +1180,12 @@ object HiveHashFunction extends InterpretedHashFunction {
(result * 37) + nanoSeconds
}
- override def hash(value: Any, dataType: DataType, seed: Long): Long = {
+ override def hash(
+ value: Any,
+ dataType: DataType,
+ seed: Long,
+ isCollationAware: Boolean,
+ legacyCollationAwareHashing: Boolean): Long = {
value match {
case null => 0
case array: ArrayData =>
@@ -1031,7 +1198,9 @@ object HiveHashFunction extends InterpretedHashFunction {
var i = 0
val length = array.numElements()
while (i < length) {
- result = (31 * result) + hash(array.get(i, elementType), elementType, 0).toInt
+ result = (31 * result) + hash(
+ array.get(i, elementType), elementType, 0, isCollationAware, legacyCollationAwareHashing
+ ).toInt
i += 1
}
result
@@ -1050,7 +1219,11 @@ object HiveHashFunction extends InterpretedHashFunction {
var i = 0
val length = map.numElements()
while (i < length) {
- result += hash(keys.get(i, kt), kt, 0).toInt ^ hash(values.get(i, vt), vt, 0).toInt
+ result += hash(
+ keys.get(i, kt), kt, 0, isCollationAware, legacyCollationAwareHashing
+ ).toInt ^ hash(
+ values.get(i, vt), vt, 0, isCollationAware, legacyCollationAwareHashing
+ ).toInt
i += 1
}
result
@@ -1066,7 +1239,10 @@ object HiveHashFunction extends InterpretedHashFunction {
var i = 0
val length = struct.numFields
while (i < length) {
- result = (31 * result) + hash(struct.get(i, types(i)), types(i), 0).toInt
+ result = (31 * result) +
+ hash(
+ struct.get(i, types(i)), types(i), 0, isCollationAware, legacyCollationAwareHashing
+ ).toInt
i += 1
}
result
@@ -1074,7 +1250,7 @@ object HiveHashFunction extends InterpretedHashFunction {
case d: Decimal => normalizeDecimal(d.toJavaBigDecimal).hashCode()
case timestamp: Long if dataType.isInstanceOf[TimestampType] => hashTimestamp(timestamp)
case calendarInterval: CalendarInterval => hashCalendarInterval(calendarInterval)
- case _ => super.hash(value, dataType, 0)
+ case _ => super.hash(value, dataType, 0, isCollationAware, legacyCollationAwareHashing)
}
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala
index b942006e87e9d..f0473f5a414f2 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala
@@ -33,13 +33,13 @@ import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StringType, Str
import org.apache.spark.unsafe.types.{UTF8String, VariantVal}
import org.apache.spark.util.Utils
-private[this] sealed trait PathInstruction
-private[this] object PathInstruction {
+sealed trait PathInstruction
+object PathInstruction {
private[expressions] case object Subscript extends PathInstruction
private[expressions] case object Wildcard extends PathInstruction
private[expressions] case object Key extends PathInstruction
private[expressions] case class Index(index: Long) extends PathInstruction
- private[expressions] case class Named(name: String) extends PathInstruction
+ case class Named(name: String) extends PathInstruction
}
private[this] sealed trait WriteStyle
@@ -49,7 +49,7 @@ private[this] object WriteStyle {
private[expressions] case object FlattenStyle extends WriteStyle
}
-private[this] object JsonPathParser extends RegexParsers {
+object JsonPathParser extends RegexParsers {
import PathInstruction._
def root: Parser[Char] = '$'
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
index e3ed2c4a0b0b8..c799415dfc706 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
@@ -30,14 +30,13 @@ import java.nio.charset.StandardCharsets
import java.sql.{Date, Timestamp}
import java.time.{Duration, Instant, LocalDate, LocalDateTime, LocalTime, Period, ZoneOffset}
import java.util
-import java.util.Objects
+import java.util.{HexFormat, Objects}
import scala.collection.{immutable, mutable}
import scala.math.{BigDecimal, BigInt}
import scala.reflect.runtime.universe.TypeTag
import scala.util.Try
-import org.apache.commons.codec.binary.{Hex => ApacheHex}
import org.json4s.JsonAST._
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, FunctionIdentifier, InternalRow, ScalaReflection}
@@ -49,7 +48,7 @@ import org.apache.spark.sql.catalyst.trees.TreePattern
import org.apache.spark.sql.catalyst.trees.TreePattern.{LITERAL, NULL_LITERAL, TRUE_OR_FALSE_LITERAL}
import org.apache.spark.sql.catalyst.types._
import org.apache.spark.sql.catalyst.util._
-import org.apache.spark.sql.catalyst.util.DateTimeUtils.{instantToMicros, localTimeToMicros}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils.{instantToMicros, localTimeToNanos}
import org.apache.spark.sql.catalyst.util.IntervalStringStyles.ANSI_STYLE
import org.apache.spark.sql.catalyst.util.IntervalUtils.{durationToMicros, periodToMonths, toDayTimeIntervalString, toYearMonthIntervalString}
import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
@@ -89,7 +88,7 @@ object Literal {
case l: LocalDateTime => Literal(DateTimeUtils.localDateTimeToMicros(l), TimestampNTZType)
case ld: LocalDate => Literal(ld.toEpochDay.toInt, DateType)
case d: Date => Literal(DateTimeUtils.fromJavaDate(d), DateType)
- case lt: LocalTime => Literal(localTimeToMicros(lt), TimeType())
+ case lt: LocalTime => Literal(localTimeToNanos(lt), TimeType())
case d: Duration => Literal(durationToMicros(d), DayTimeIntervalType())
case p: Period => Literal(periodToMonths(p), YearMonthIntervalType())
case a: Array[Byte] => Literal(a, BinaryType)
@@ -415,6 +414,9 @@ case class Literal (value: Any, dataType: DataType) extends LeafExpression {
Literal.validateLiteralValue(value, dataType)
override def foldable: Boolean = true
+
+ override def contextIndependentFoldable: Boolean = true
+
override def nullable: Boolean = value == null
private def timeZoneId = DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone)
@@ -429,7 +431,7 @@ case class Literal (value: Any, dataType: DataType) extends LeafExpression {
override def toString: String = value match {
case null => "null"
- case binary: Array[Byte] => "0x" + ApacheHex.encodeHexString(binary, false)
+ case binary: Array[Byte] => "0x" + HexFormat.of().withUpperCase().formatHex(binary)
case d: ArrayBasedMapData => s"map(${d.toString})"
case other =>
dataType match {
@@ -575,7 +577,7 @@ case class Literal (value: Any, dataType: DataType) extends LeafExpression {
s"TIMESTAMP_NTZ '$toString'"
case (i: CalendarInterval, CalendarIntervalType) =>
s"INTERVAL '${i.toString}'"
- case (v: Array[Byte], BinaryType) => s"X'${ApacheHex.encodeHexString(v, false)}'"
+ case (v: Array[Byte], BinaryType) => s"X'${HexFormat.of().withUpperCase().formatHex(v)}'"
case (i: Long, DayTimeIntervalType(startField, endField)) =>
toDayTimeIntervalString(i, ANSI_STYLE, startField, endField)
case (i: Int, YearMonthIntervalType(startField, endField)) =>
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala
index 9db2ac7f9b041..ee3e3e0272767 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala
@@ -48,6 +48,7 @@ abstract class LeafMathExpression(c: Double, name: String)
override def dataType: DataType = DoubleType
override def foldable: Boolean = true
+ override def contextIndependentFoldable: Boolean = true
override def nullable: Boolean = false
override def toString: String = s"$name()"
override def prettyName: String = name
@@ -68,6 +69,7 @@ abstract class UnaryMathExpression(val f: Double => Double, name: String)
override def inputTypes: Seq[AbstractDataType] = Seq(DoubleType)
override def dataType: DataType = DoubleType
+ override def contextIndependentFoldable: Boolean = child.contextIndependentFoldable
override def nullable: Boolean = true
override def toString: String = s"$prettyName($child)"
override def prettyName: String = getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse(name)
@@ -87,6 +89,8 @@ abstract class UnaryMathExpression(val f: Double => Double, name: String)
abstract class UnaryLogExpression(f: Double => Double, name: String)
extends UnaryMathExpression(f, name) {
+ override def contextIndependentFoldable: Boolean = child.contextIndependentFoldable
+
override def nullable: Boolean = true
// values less than or equal to yAsymptote eval to null in Hive, instead of NaN or -Infinity
@@ -123,6 +127,8 @@ abstract class BinaryMathExpression(f: (Double, Double) => Double, name: String)
override def inputTypes: Seq[DataType] = Seq(DoubleType, DoubleType)
+ override def contextIndependentFoldable: Boolean = children.forall(_.contextIndependentFoldable)
+
override def toString: String = s"$prettyName($left, $right)"
override def prettyName: String = getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse(name)
@@ -1011,6 +1017,7 @@ case class Bin(child: Expression)
with DefaultStringProducingExpression {
override def nullIntolerant: Boolean = true
override def inputTypes: Seq[DataType] = Seq(LongType)
+ override def contextIndependentFoldable: Boolean = child.contextIndependentFoldable
protected override def nullSafeEval(input: Any): Any =
UTF8String.toBinaryString(input.asInstanceOf[Long])
@@ -1129,6 +1136,8 @@ case class Hex(child: Expression)
case _ => super.dataType
}
+ override def contextIndependentFoldable: Boolean = child.contextIndependentFoldable
+
protected override def nullSafeEval(num: Any): Any = child.dataType match {
case LongType => Hex.hex(num.asInstanceOf[Long])
case BinaryType => Hex.hex(num.asInstanceOf[Array[Byte]])
@@ -1164,6 +1173,7 @@ case class Hex(child: Expression)
case class Unhex(child: Expression, failOnError: Boolean = false)
extends UnaryExpression with ImplicitCastInputTypes {
override def nullIntolerant: Boolean = true
+ override def contextIndependentFoldable: Boolean = child.contextIndependentFoldable
def this(expr: Expression) = this(expr, false)
@@ -1263,7 +1273,7 @@ case class Pow(left: Expression, right: Expression)
sealed trait BitShiftOperation
extends BinaryExpression with ImplicitCastInputTypes {
override def nullIntolerant: Boolean = true
-
+ override def contextIndependentFoldable: Boolean = children.forall(_.contextIndependentFoldable)
def symbol: String
def shiftInt: (Int, Int) => Int
def shiftLong: (Long, Int) => Long
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala
index dcbca34b240b5..6f806760b3736 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala
@@ -327,6 +327,7 @@ case class SparkVersion()
case class TypeOf(child: Expression) extends UnaryExpression with DefaultStringProducingExpression {
override def nullable: Boolean = false
override def foldable: Boolean = true
+ override def contextIndependentFoldable: Boolean = true
override def eval(input: InternalRow): Any = UTF8String.fromString(child.dataType.catalogString)
override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala
index 290f523cc02c1..1aa1d0b25e44c 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala
@@ -285,6 +285,8 @@ case class IsNaN(child: Expression) extends UnaryExpression
override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(DoubleType, FloatType))
+ override def contextIndependentFoldable: Boolean = child.contextIndependentFoldable
+
override def nullable: Boolean = false
override def eval(input: InternalRow): Any = {
@@ -334,6 +336,8 @@ case class NaNvl(left: Expression, right: Expression)
override def inputTypes: Seq[AbstractDataType] =
Seq(TypeCollection(DoubleType, FloatType), TypeCollection(DoubleType, FloatType))
+ override def contextIndependentFoldable: Boolean = children.forall(_.contextIndependentFoldable)
+
/**
* We can only guarantee the left child can be always accessed. If we hit the left child,
* the right child will not be accessed.
@@ -406,6 +410,8 @@ case class NaNvl(left: Expression, right: Expression)
case class IsNull(child: Expression) extends UnaryExpression with Predicate {
override def nullable: Boolean = false
+ override def contextIndependentFoldable: Boolean = child.contextIndependentFoldable
+
final override val nodePatterns: Seq[TreePattern] = Seq(NULL_CHECK)
override def eval(input: InternalRow): Any = {
@@ -440,6 +446,8 @@ case class IsNotNull(child: Expression) extends UnaryExpression with Predicate {
final override val nodePatterns: Seq[TreePattern] = Seq(NULL_CHECK)
+ override def contextIndependentFoldable: Boolean = child.contextIndependentFoldable
+
override def eval(input: InternalRow): Any = {
child.eval(input) != null
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
index 21dcbba818d9f..0a6d23977d2f6 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
@@ -54,6 +54,9 @@ abstract class ToNumberBase(left: Expression, right: Expression, errorOnFail: Bo
StringTypeWithCollation(supportsTrimCollation = true),
StringTypeWithCollation(supportsTrimCollation = true))
+
+ override def contextIndependentFoldable: Boolean = children.forall(_.contextIndependentFoldable)
+
override def checkInputDataTypes(): TypeCheckResult = {
val inputTypeCheck = super.checkInputDataTypes()
if (inputTypeCheck.isSuccess) {
@@ -310,6 +313,7 @@ case class ToCharacter(left: Expression, right: Expression)
inputTypeCheck
}
}
+ override def contextIndependentFoldable: Boolean = children.forall(_.contextIndependentFoldable)
override def prettyName: String = "to_char"
override def nullSafeEval(decimal: Any, format: Any): Any = {
val input = decimal.asInstanceOf[Decimal]
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala
index 37a3b3a34e49c..c735f5b334bd5 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala
@@ -38,6 +38,8 @@ class BaseOrdering extends Ordering[InternalRow] {
* An interpreted row ordering comparator.
*/
class InterpretedOrdering(ordering: Seq[SortOrder]) extends BaseOrdering {
+ private val leftEvaluators = ordering.map(_.child)
+ private val rightEvaluators = leftEvaluators.map(_.freshCopyIfContainsStatefulExpression())
private lazy val physicalDataTypes = ordering.map { order =>
val dt = order.dataType match {
case udt: UserDefinedType[_] => udt.sqlType
@@ -54,8 +56,8 @@ class InterpretedOrdering(ordering: Seq[SortOrder]) extends BaseOrdering {
val size = ordering.size
while (i < size) {
val order = ordering(i)
- val left = order.child.eval(a)
- val right = order.child.eval(b)
+ val left = leftEvaluators(i).eval(a)
+ val right = rightEvaluators(i).eval(b)
if (left == null && right == null) {
// Both null, continue looking.
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
index c31c72bc11488..7767e97d34ac5 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
@@ -32,11 +32,11 @@ import org.apache.spark.sql.catalyst.expressions.codegen.Block._
import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LeafNode, LogicalPlan, Project, Union}
import org.apache.spark.sql.catalyst.trees.TreePattern._
import org.apache.spark.sql.catalyst.util.{CollationFactory, TypeUtils}
-import org.apache.spark.sql.catalyst.util.SparkStringUtils.truncatedString
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
import org.apache.spark.util.ArrayImplicits._
+import org.apache.spark.util.SparkStringUtils.truncatedString
/**
* A base class for generated/interpreted predicate
@@ -67,6 +67,8 @@ case class InterpretedPredicate(expression: Expression) extends BasePredicate {
*/
trait Predicate extends Expression {
override def dataType: DataType = BooleanType
+
+ override def contextIndependentFoldable: Boolean = children.forall(_.contextIndependentFoldable)
}
/**
@@ -319,6 +321,8 @@ case class Not(child: Expression)
extends UnaryExpression with Predicate with ImplicitCastInputTypes {
override def nullIntolerant: Boolean = true
+ override def contextIndependentFoldable: Boolean = child.contextIndependentFoldable
+
override def toString: String = s"NOT $child"
override def inputTypes: Seq[DataType] = Seq(BooleanType)
@@ -481,6 +485,7 @@ case class In(value: Expression, list: Seq[Expression]) extends Predicate {
override def nullable: Boolean = children.exists(_.nullable)
override def foldable: Boolean = children.forall(_.foldable)
+ override def contextIndependentFoldable: Boolean = children.forall(_.contextIndependentFoldable)
final override val nodePatterns: Seq[TreePattern] = Seq(IN)
private val legacyNullInEmptyBehavior =
@@ -619,6 +624,8 @@ case class InSet(child: Expression, hset: Set[Any]) extends UnaryExpression with
require(hset != null, "hset could not be null")
+ override def contextIndependentFoldable: Boolean = child.contextIndependentFoldable
+
override def simpleString(maxFields: Int): String = {
if (!child.resolved) {
return s"$child INSET (values with unresolved data types)"
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
index be7d227d80026..c6e5c480f3c2c 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
@@ -37,8 +37,8 @@ import org.apache.spark.sql.catalyst.trees.BinaryLike
import org.apache.spark.sql.catalyst.trees.TreePattern.{LIKE_FAMLIY, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, TreePattern}
import org.apache.spark.sql.catalyst.util.{CollationSupport, GenericArrayData, StringUtils}
import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
-import org.apache.spark.sql.internal.types.{
- StringTypeBinaryLcase, StringTypeWithCollation}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.types.{StringTypeBinaryLcase, StringTypeWithCollation}
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
@@ -293,6 +293,8 @@ sealed abstract class MultiLikeBase
extends UnaryExpression with ImplicitCastInputTypes with Predicate {
override def nullIntolerant: Boolean = true
+ override def contextIndependentFoldable: Boolean = child.contextIndependentFoldable
+
protected def patterns: Seq[UTF8String]
protected def isNotSpecified: Boolean
@@ -578,20 +580,33 @@ case class StringSplit(str: Expression, regex: Expression, limit: Expression)
final lazy val collationId: Int = str.dataType.asInstanceOf[StringType].collationId
+ private lazy val legacySplitTruncate =
+ SQLConf.get.getConf(SQLConf.LEGACY_TRUNCATE_FOR_EMPTY_REGEX_SPLIT)
+
def this(exp: Expression, regex: Expression) = this(exp, regex, Literal(-1))
override def nullSafeEval(string: Any, regex: Any, limit: Any): Any = {
- val pattern = CollationSupport.collationAwareRegex(regex.asInstanceOf[UTF8String], collationId)
- val strings = string.asInstanceOf[UTF8String].split(pattern, limit.asInstanceOf[Int])
+ val pattern = CollationSupport.collationAwareRegex(
+ regex.asInstanceOf[UTF8String], collationId, legacySplitTruncate)
+ val strings = if (legacySplitTruncate) {
+ string.asInstanceOf[UTF8String].splitLegacyTruncate(pattern, limit.asInstanceOf[Int])
+ } else {
+ string.asInstanceOf[UTF8String].split(pattern, limit.asInstanceOf[Int])
+ }
new GenericArrayData(strings.asInstanceOf[Array[Any]])
}
override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
val arrayClass = classOf[GenericArrayData].getName
+ val pattern = ctx.freshName("pattern")
nullSafeCodeGen(ctx, ev, (str, regex, limit) => {
// Array in java is covariant, so we don't need to cast UTF8String[] to Object[].
- s"""${ev.value} = new $arrayClass($str.split(
- |CollationSupport.collationAwareRegex($regex, $collationId),$limit));""".stripMargin
+ s"""
+ |UTF8String $pattern =
+ | CollationSupport.collationAwareRegex($regex, $collationId, $legacySplitTruncate);
+ |${ev.value} = new $arrayClass($legacySplitTruncate ?
+ | $str.splitLegacyTruncate($pattern, $limit) : $str.split($pattern, $limit));
+ |""".stripMargin
})
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
index 55cc030a99537..3d32a2ef606c7 100755
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
@@ -94,6 +94,7 @@ case class ConcatWs(children: Seq[Expression])
override def nullable: Boolean = children.head.nullable
override def foldable: Boolean = children.forall(_.foldable)
+ override def contextIndependentFoldable: Boolean = children.forall(_.contextIndependentFoldable)
override def checkInputDataTypes(): TypeCheckResult = {
if (children.isEmpty) {
@@ -440,6 +441,7 @@ trait String2StringExpression extends ImplicitCastInputTypes {
override def dataType: DataType = child.dataType
override def inputTypes: Seq[AbstractDataType] =
Seq(StringTypeWithCollation(supportsTrimCollation = true))
+ override def contextIndependentFoldable: Boolean = child.contextIndependentFoldable
protected override def nullSafeEval(input: Any): Any =
convert(input.asInstanceOf[UTF8String])
@@ -1264,6 +1266,8 @@ case class FindInSet(left: Expression, right: Expression) extends BinaryExpressi
StringTypeWithCollation(supportsTrimCollation = true)
)
+ override def contextIndependentFoldable: Boolean = super.contextIndependentFoldable
+
override protected def nullSafeEval(word: Any, set: Any): Any = {
CollationSupport.FindInSet.
exec(word.asInstanceOf[UTF8String], set.asInstanceOf[UTF8String], collationId)
@@ -1297,6 +1301,7 @@ trait String2TrimExpression extends Expression with ImplicitCastInputTypes {
override def nullable: Boolean = children.exists(_.nullable)
override def foldable: Boolean = children.forall(_.foldable)
+ override def contextIndependentFoldable: Boolean = children.forall(_.contextIndependentFoldable)
protected def doEval(srcString: UTF8String): UTF8String
protected def doEval(srcString: UTF8String, trimString: UTF8String): UTF8String
@@ -1678,6 +1683,8 @@ case class StringInstr(str: Expression, substr: Expression)
StringTypeNonCSAICollation(supportsTrimCollation = true)
)
+ override def contextIndependentFoldable: Boolean = super.contextIndependentFoldable
+
override def nullSafeEval(string: Any, sub: Any): Any = {
CollationSupport.StringInstr.
exec(string.asInstanceOf[UTF8String], sub.asInstanceOf[UTF8String], collationId) + 1
@@ -1729,6 +1736,7 @@ case class SubstringIndex(strExpr: Expression, delimExpr: Expression, countExpr:
StringTypeNonCSAICollation(supportsTrimCollation = true),
IntegerType
)
+ override def contextIndependentFoldable: Boolean = children.forall(_.contextIndependentFoldable)
override def first: Expression = strExpr
override def second: Expression = delimExpr
override def third: Expression = countExpr
@@ -2050,6 +2058,7 @@ case class FormatString(children: Expression*) extends Expression with ImplicitC
override def foldable: Boolean = children.forall(_.foldable)
+ override def contextIndependentFoldable: Boolean = children.forall(_.contextIndependentFoldable)
override def nullable: Boolean = children(0).nullable
override def dataType: DataType = children(0).dataType
@@ -2205,6 +2214,8 @@ case class StringRepeat(str: Expression, times: Expression)
IntegerType
)
+ override def contextIndependentFoldable: Boolean = super.contextIndependentFoldable
+
override def nullSafeEval(string: Any, n: Any): Any = {
string.asInstanceOf[UTF8String].repeat(n.asInstanceOf[Integer])
}
@@ -2235,6 +2246,7 @@ case class StringSpace(child: Expression)
extends UnaryExpression with ImplicitCastInputTypes with DefaultStringProducingExpression {
override def nullIntolerant: Boolean = true
override def inputTypes: Seq[DataType] = Seq(IntegerType)
+ override def contextIndependentFoldable: Boolean = child.contextIndependentFoldable
override def nullSafeEval(s: Any): Any = {
val length = s.asInstanceOf[Int]
@@ -2302,6 +2314,8 @@ case class Substring(str: Expression, pos: Expression, len: Expression)
IntegerType
)
+ override def contextIndependentFoldable: Boolean = children.forall(_.contextIndependentFoldable)
+
override def first: Expression = str
override def second: Expression = pos
override def third: Expression = len
@@ -2354,7 +2368,7 @@ case class Right(str: Expression, len: Expression) extends RuntimeReplaceable
If(
LessThanOrEqual(len, Literal(0)),
Literal(UTF8String.EMPTY_UTF8, str.dataType),
- new Substring(str, UnaryMinus(len))
+ new Substring(str, UnaryMinus(len, failOnError = false))
)
)
@@ -2441,6 +2455,7 @@ case class Length(child: Expression)
BinaryType
)
)
+ override def contextIndependentFoldable: Boolean = child.contextIndependentFoldable
protected override def nullSafeEval(value: Any): Any = child.dataType match {
case _: StringType => value.asInstanceOf[UTF8String].numChars
@@ -2482,6 +2497,7 @@ case class BitLength(child: Expression)
BinaryType
)
)
+ override def contextIndependentFoldable: Boolean = child.contextIndependentFoldable
protected override def nullSafeEval(value: Any): Any = child.dataType match {
case _: StringType => value.asInstanceOf[UTF8String].numBytes * 8
case BinaryType => value.asInstanceOf[Array[Byte]].length * 8
@@ -2526,6 +2542,7 @@ case class OctetLength(child: Expression)
BinaryType
)
)
+ override def contextIndependentFoldable: Boolean = child.contextIndependentFoldable
protected override def nullSafeEval(value: Any): Any = child.dataType match {
case _: StringType => value.asInstanceOf[UTF8String].numBytes
@@ -2617,6 +2634,7 @@ case class Levenshtein(
override def nullIntolerant: Boolean = true
override def foldable: Boolean = children.forall(_.foldable)
+ override def contextIndependentFoldable: Boolean = children.forall(_.contextIndependentFoldable)
override def eval(input: InternalRow): Any = {
val leftEval = left.eval(input)
@@ -2722,6 +2740,8 @@ case class SoundEx(child: Expression)
override def inputTypes: Seq[AbstractDataType] =
Seq(StringTypeWithCollation(supportsTrimCollation = true))
+ override def contextIndependentFoldable: Boolean = child.contextIndependentFoldable
+
override def nullSafeEval(input: Any): Any = input.asInstanceOf[UTF8String].soundex()
override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
@@ -2811,6 +2831,8 @@ case class Chr(child: Expression)
}
}
+ override def contextIndependentFoldable: Boolean = child.contextIndependentFoldable
+
override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
nullSafeCodeGen(ctx, ev, lon => {
s"""
@@ -2853,6 +2875,8 @@ case class Base64(child: Expression, chunkBase64: Boolean)
override def inputTypes: Seq[DataType] = Seq(BinaryType)
+ override def contextIndependentFoldable: Boolean = child.contextIndependentFoldable
+
override lazy val replacement: Expression = StaticInvoke(
classOf[Base64],
dataType,
@@ -2901,6 +2925,7 @@ case class UnBase64(child: Expression, failOnError: Boolean = false)
override def dataType: DataType = BinaryType
override def inputTypes: Seq[AbstractDataType] =
Seq(StringTypeWithCollation(supportsTrimCollation = true))
+ override def contextIndependentFoldable: Boolean = child.contextIndependentFoldable
def this(expr: Expression) = this(expr, false)
@@ -3340,6 +3365,7 @@ case class FormatNumber(x: Expression, d: Expression)
override def right: Expression = d
override def nullable: Boolean = true
override def nullIntolerant: Boolean = true
+ override def contextIndependentFoldable: Boolean = super.contextIndependentFoldable
override def inputTypes: Seq[AbstractDataType] =
Seq(
@@ -3590,6 +3616,7 @@ case class StringSplitSQL(
override def left: Expression = str
override def right: Expression = delimiter
override def nullIntolerant: Boolean = true
+ override def contextIndependentFoldable: Boolean = super.contextIndependentFoldable
override def nullSafeEval(string: Any, delimiter: Any): Any = {
val strings = CollationSupport.StringSplitSQL.exec(string.asInstanceOf[UTF8String],
@@ -3752,6 +3779,8 @@ case class Quote(input: Expression)
Seq(StringTypeWithCollation(supportsTrimCollation = true))
}
+ override def contextIndependentFoldable: Boolean = child.contextIndependentFoldable
+
override def nodeName: String = "quote"
override def nullable: Boolean = true
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/timeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/timeExpressions.scala
index 47f2d5d73e212..ff088876969bd 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/timeExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/timeExpressions.scala
@@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.expressions
import java.time.DateTimeException
import java.util.Locale
+import org.apache.spark.SparkException
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.{ExpressionBuilder, TypeCheckResult}
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{DataTypeMismatch, TypeCheckSuccess}
@@ -32,7 +33,8 @@ import org.apache.spark.sql.catalyst.util.TimeFormatter
import org.apache.spark.sql.catalyst.util.TypeUtils.ordinalNumber
import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
import org.apache.spark.sql.internal.types.StringTypeWithCollation
-import org.apache.spark.sql.types.{AbstractDataType, DataType, DecimalType, IntegerType, ObjectType, TimeType, TypeCollection}
+import org.apache.spark.sql.types.{AbstractDataType, AnyTimeType, ByteType, DataType, DayTimeIntervalType, DecimalType, IntegerType, LongType, ObjectType, TimeType}
+import org.apache.spark.sql.types.DayTimeIntervalType.{HOUR, SECOND}
import org.apache.spark.unsafe.types.UTF8String
/**
@@ -208,8 +210,7 @@ case class MinutesOfTime(child: Expression)
Seq(child.dataType)
)
- override def inputTypes: Seq[AbstractDataType] =
- Seq(TypeCollection(TimeType.MIN_PRECISION to TimeType.MAX_PRECISION map TimeType.apply: _*))
+ override def inputTypes: Seq[AbstractDataType] = Seq(AnyTimeType)
override def children: Seq[Expression] = Seq(child)
@@ -268,8 +269,7 @@ case class HoursOfTime(child: Expression)
Seq(child.dataType)
)
- override def inputTypes: Seq[AbstractDataType] =
- Seq(TypeCollection(TimeType.MIN_PRECISION to TimeType.MAX_PRECISION map TimeType.apply: _*))
+ override def inputTypes: Seq[AbstractDataType] = Seq(AnyTimeType)
override def children: Seq[Expression] = Seq(child)
@@ -317,9 +317,11 @@ object HourExpressionBuilder extends ExpressionBuilder {
case class SecondsOfTimeWithFraction(child: Expression)
extends RuntimeReplaceable
with ExpectsInputTypes {
-
override def replacement: Expression = {
-
+ val precision = child.dataType match {
+ case TimeType(p) => p
+ case _ => TimeType.MIN_PRECISION
+ }
StaticInvoke(
classOf[DateTimeUtils.type],
DecimalType(8, 6),
@@ -327,10 +329,8 @@ case class SecondsOfTimeWithFraction(child: Expression)
Seq(child, Literal(precision)),
Seq(child.dataType, IntegerType))
}
- private val precision: Int = child.dataType.asInstanceOf[TimeType].precision
- override def inputTypes: Seq[AbstractDataType] =
- Seq(TimeType(precision))
+ override def inputTypes: Seq[AbstractDataType] = Seq(AnyTimeType)
override def children: Seq[Expression] = Seq(child)
@@ -352,8 +352,7 @@ case class SecondsOfTime(child: Expression)
Seq(child.dataType)
)
- override def inputTypes: Seq[AbstractDataType] =
- Seq(TypeCollection(TimeType.MIN_PRECISION to TimeType.MAX_PRECISION map TimeType.apply: _*))
+ override def inputTypes: Seq[AbstractDataType] = Seq(AnyTimeType)
override def children: Seq[Expression] = Seq(child)
@@ -516,8 +515,8 @@ case class CurrentTime(
override def inputTypes: Seq[AbstractDataType] = Seq(IntegerType)
override def eval(input: InternalRow): Any = {
- val currentTimeOfDayMicros = DateTimeUtils.instantToMicrosOfDay(java.time.Instant.now(), zoneId)
- DateTimeUtils.truncateTimeMicrosToPrecision(currentTimeOfDayMicros, precision)
+ val currentTimeOfDayNanos = DateTimeUtils.instantToNanosOfDay(java.time.Instant.now(), zoneId)
+ DateTimeUtils.truncateTimeToPrecision(currentTimeOfDayNanos, precision)
}
}
@@ -558,7 +557,7 @@ case class MakeTime(
override def replacement: Expression = StaticInvoke(
classOf[DateTimeUtils.type],
TimeType(TimeType.MICROS_PRECISION),
- "timeToMicros",
+ "makeTime",
children,
inputTypes
)
@@ -566,3 +565,187 @@ case class MakeTime(
override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): MakeTime =
copy(hours = newChildren(0), minutes = newChildren(1), secsAndMicros = newChildren(2))
}
+
+/**
+ * Adds day-time interval to time.
+ */
+case class TimeAddInterval(time: Expression, interval: Expression)
+ extends BinaryExpression with RuntimeReplaceable with ExpectsInputTypes {
+ override def nullIntolerant: Boolean = true
+
+ override def left: Expression = time
+ override def right: Expression = interval
+
+ override def toString: String = s"$left + $right"
+ override def sql: String = s"${left.sql} + ${right.sql}"
+ override def inputTypes: Seq[AbstractDataType] = Seq(AnyTimeType, DayTimeIntervalType)
+
+ override def replacement: Expression = {
+ val (timePrecision, intervalEndField) = (time.dataType, interval.dataType) match {
+ case (TimeType(p), DayTimeIntervalType(_, endField)) => (p, endField)
+ case _ => throw SparkException.internalError("Unexpected input types: " +
+ s"time type ${time.dataType.sql}, interval type ${interval.dataType.sql}.")
+ }
+ val intervalPrecision = if (intervalEndField < SECOND) {
+ TimeType.MIN_PRECISION
+ } else {
+ TimeType.MICROS_PRECISION
+ }
+ val targetPrecision = Math.max(timePrecision, intervalPrecision)
+ StaticInvoke(
+ classOf[DateTimeUtils.type],
+ TimeType(targetPrecision),
+ "timeAddInterval",
+ Seq(time, Literal(timePrecision), interval, Literal(intervalEndField),
+ Literal(targetPrecision)),
+ Seq(AnyTimeType, IntegerType, DayTimeIntervalType, ByteType, IntegerType),
+ propagateNull = nullIntolerant)
+ }
+
+ override protected def withNewChildrenInternal(
+ newTime: Expression, newInterval: Expression): TimeAddInterval =
+ copy(time = newTime, interval = newInterval)
+}
+
+/**
+ * Returns a day-time interval between time values.
+ */
+case class SubtractTimes(left: Expression, right: Expression)
+ extends BinaryExpression with RuntimeReplaceable with ExpectsInputTypes {
+ override def nullIntolerant: Boolean = true
+ override def inputTypes: Seq[AbstractDataType] = Seq(AnyTimeType, AnyTimeType)
+
+ override def replacement: Expression = StaticInvoke(
+ classOf[DateTimeUtils.type],
+ DayTimeIntervalType(HOUR, SECOND),
+ "subtractTimes",
+ children,
+ inputTypes,
+ propagateNull = nullIntolerant)
+
+ override def toString: String = s"$left - $right"
+ override def sql: String = s"${left.sql} - ${right.sql}"
+
+ override protected def withNewChildrenInternal(
+ newLeft: Expression, newRight: Expression): SubtractTimes =
+ copy(left = newLeft, right = newRight)
+}
+
+/**
+ * Returns the difference between two times, measured in specified units.
+ */
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+ usage = """
+ _FUNC_(unit, start, end) - Gets the difference between the times in the specified units.
+ """,
+ arguments = """
+ Arguments:
+ * unit - the unit of the difference between the given times
+ - "HOUR"
+ - "MINUTE"
+ - "SECOND"
+ - "MILLISECOND"
+ - "MICROSECOND"
+ * start - a starting TIME expression
+ * end - an ending TIME expression
+ """,
+ examples = """
+ Examples:
+ > SELECT _FUNC_('HOUR', TIME'20:30:29', TIME'21:30:28');
+ 0
+ > SELECT _FUNC_('HOUR', TIME'20:30:29', TIME'21:30:29');
+ 1
+ > SELECT _FUNC_('HOUR', TIME'20:30:29', TIME'12:00:00');
+ -8
+ """,
+ group = "datetime_funcs",
+ since = "4.1.0")
+// scalastyle:on line.size.limit
+case class TimeDiff(
+ unit: Expression,
+ start: Expression,
+ end: Expression)
+ extends TernaryExpression
+ with RuntimeReplaceable
+ with ImplicitCastInputTypes {
+
+ override def first: Expression = unit
+ override def second: Expression = start
+ override def third: Expression = end
+
+ override def inputTypes: Seq[AbstractDataType] =
+ Seq(StringTypeWithCollation(supportsTrimCollation = true), AnyTimeType, AnyTimeType)
+
+ override def dataType: DataType = LongType
+
+ override def prettyName: String = "time_diff"
+
+ override protected def withNewChildrenInternal(
+ newUnit: Expression, newStart: Expression, newEnd: Expression): TimeDiff = {
+ copy(unit = newUnit, start = newStart, end = newEnd)
+ }
+
+ override def replacement: Expression = {
+ StaticInvoke(
+ classOf[DateTimeUtils.type],
+ dataType,
+ "timeDiff",
+ Seq(unit, start, end),
+ Seq(unit.dataType, start.dataType, end.dataType)
+ )
+ }
+}
+
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+ usage = """
+ _FUNC_(unit, time) - Returns `time` truncated to the `unit`.
+ """,
+ arguments = """
+ Arguments:
+ * unit - the unit to truncate to
+ - "HOUR" - zero out the minutes and seconds with fraction part
+ - "MINUTE" - zero out the seconds with fraction part
+ - "SECOND" - zero out the fraction part of seconds
+ - "MILLISECOND" - zero out the microseconds
+ - "MICROSECOND" - zero out the nanoseconds
+ * time - a TIME expression
+ """,
+ examples = """
+ Examples:
+ > SELECT _FUNC_('HOUR', TIME'09:32:05.359');
+ 09:00:00
+ > SELECT _FUNC_('MILLISECOND', TIME'09:32:05.123456');
+ 09:32:05.123
+ """,
+ group = "datetime_funcs",
+ since = "4.1.0")
+// scalastyle:on line.size.limit
+case class TimeTrunc(unit: Expression, time: Expression)
+ extends BinaryExpression with RuntimeReplaceable with ImplicitCastInputTypes {
+
+ override def left: Expression = unit
+ override def right: Expression = time
+
+ override def inputTypes: Seq[AbstractDataType] =
+ Seq(StringTypeWithCollation(supportsTrimCollation = true), AnyTimeType)
+
+ override def dataType: DataType = time.dataType
+
+ override def prettyName: String = "time_trunc"
+
+ override protected def withNewChildrenInternal(
+ newUnit: Expression, newTime: Expression): TimeTrunc =
+ copy(unit = newUnit, time = newTime)
+
+ override def replacement: Expression = {
+ StaticInvoke(
+ classOf[DateTimeUtils.type],
+ dataType,
+ "timeTrunc",
+ Seq(unit, time),
+ Seq(unit.dataType, time.dataType)
+ )
+ }
+}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
index ab787663c9923..f26a4dc9a0c0c 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
@@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.expressions
import java.util.Locale
import org.apache.spark.SparkException
+import org.apache.spark.api.python.PythonEvalType
import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, UnresolvedException}
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{DataTypeMismatch, TypeCheckSuccess}
import org.apache.spark.sql.catalyst.dsl.expressions._
@@ -401,6 +402,16 @@ object WindowFunctionType {
// consider its type to be SQL as literal(0) is also a SQL expression.
t.getOrElse(SQL)
}
+
+ def pythonEvalType(windowExpression: NamedExpression): Option[Int] = {
+ windowExpression.collectFirst {
+ case udf: PythonUDAF => udf.evalType match {
+ // Infer the eval type of window operation, from the input aggregation type
+ case PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF => PythonEvalType.SQL_WINDOW_AGG_PANDAS_UDF
+ case PythonEvalType.SQL_GROUPED_AGG_ARROW_UDF => PythonEvalType.SQL_WINDOW_AGG_ARROW_UDF
+ }
+ }
+ }
}
trait OffsetWindowFunction extends WindowFunction {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala
index 62d99f7854891..ad1a1a99b8257 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InlineCTE.scala
@@ -61,10 +61,7 @@ case class InlineCTE(
// 1) It is fine to inline a CTE if it references another CTE that is non-deterministic;
// 2) Any `CTERelationRef` that contains `OuterReference` would have been inlined first.
refCount == 1 ||
- // Don't inline recursive CTEs if not necessary as recursion is very costly.
- // The check if cteDef is recursive is performed by checking if it contains
- // a UnionLoopRef with the same ID.
- (cteDef.deterministic && !cteDef.hasSelfReferenceAsUnionLoopRef) ||
+ cteDef.deterministic ||
cteDef.child.exists(_.expressions.exists(_.isInstanceOf[OuterReference]))
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
index 66c3bfb46530a..ef505a0144113 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
@@ -20,10 +20,9 @@ package org.apache.spark.sql.catalyst.optimizer
import scala.collection.mutable
import org.apache.spark.SparkException
-import org.apache.spark.internal.{LogKeys, MDC}
+import org.apache.spark.internal.{LogKeys}
import org.apache.spark.sql.catalyst.SQLConfHelper
import org.apache.spark.sql.catalyst.analysis._
-import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.SubqueryExpression.hasCorrelatedSubquery
import org.apache.spark.sql.catalyst.expressions.aggregate._
@@ -573,19 +572,6 @@ object EliminateAggregateFilter extends Rule[LogicalPlan] {
}
}
-/**
- * An optimizer used in test code.
- *
- * To ensure extendability, we leave the standard rules in the abstract optimizer rules, while
- * specific rules go to the subclasses
- */
-object SimpleTestOptimizer extends SimpleTestOptimizer
-
-class SimpleTestOptimizer extends Optimizer(
- new CatalogManager(
- FakeV2SessionCatalog,
- new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry, EmptyTableFunctionRegistry)))
-
/**
* Remove redundant aliases from a query plan. A redundant alias is an alias that does not change
* the name or metadata of a column, and does not deduplicate it.
@@ -611,7 +597,7 @@ object RemoveRedundantAliases extends Rule[LogicalPlan] {
// If the alias name is different from attribute name, we can't strip it either, or we
// may accidentally change the output schema name of the root plan.
case a @ Alias(attr: Attribute, name)
- if (a.metadata == Metadata.empty || a.metadata == attr.metadata) &&
+ if (a.metadata == attr.metadata) &&
name == attr.name &&
!excludeList.contains(attr) &&
!excludeList.contains(a) =>
@@ -654,16 +640,23 @@ object RemoveRedundantAliases extends Rule[LogicalPlan] {
case u: Union =>
var first = true
plan.mapChildren { child =>
- if (first) {
- first = false
- // `Union` inherits its first child's outputs. We don't remove those aliases from the
- // first child's tree that prevent aliased attributes to appear multiple times in the
- // `Union`'s output. A parent projection node on the top of an `Union` with non-unique
- // output attributes could return incorrect result.
- removeRedundantAliases(child, excluded ++ child.outputSet)
+ if (!conf.unionIsResolvedWhenDuplicatesPerChildResolved || shouldRemoveAliasesUnderUnion(
+ child
+ )) {
+ if (first) {
+ first = false
+ // `Union` inherits its first child's outputs. We don't remove those aliases from the
+ // first child's tree that prevent aliased attributes to appear multiple times in the
+ // `Union`'s output. A parent projection node on the top of an `Union` with
+ // non-unique output attributes could return incorrect result.
+ removeRedundantAliases(child, excluded ++ child.outputSet)
+ } else {
+ // We don't need to exclude those attributes that `Union` inherits from its first
+ // child.
+ removeRedundantAliases(child, excluded -- u.children.head.outputSet)
+ }
} else {
- // We don't need to exclude those attributes that `Union` inherits from its first child.
- removeRedundantAliases(child, excluded -- u.children.head.outputSet)
+ child
}
}
@@ -708,6 +701,44 @@ object RemoveRedundantAliases extends Rule[LogicalPlan] {
}
}
+ /**
+ * In case a [[Project]], [[Aggregate]] or [[Window]] is a child of [[Union]], we don't remove an
+ * [[Alias]] in case it is on top of an [[Attribute]] which exists in the output set of the
+ * operator. This is needed because otherwise, we end up having an operator with duplicates in
+ * its output. When that happens, [[Union]] is not resolved, and we fail (but we shouldn't).
+ * In this example:
+ *
+ * {{{ SELECT col1 FROM values(1) WHERE 100 IN (SELECT col1 UNION SELECT col1); }}}
+ *
+ * Without `shouldRemoveAliasesUnderUnion` check, we would remove the [[Alias]] introduced in
+ * [[DeduplicateRelations]] rule (in a [[Project]] tagged as
+ * `PROJECT_FOR_EXPRESSION_ID_DEDUPLICATION`), the result is unresolved [[Union]] which causes the
+ * failure. With the check, [[Alias]] stays, and we resolve the plan properly.
+ */
+ private def shouldRemoveAliasesUnderUnion(operator: LogicalPlan): Boolean = {
+ operator match {
+ case project: Project =>
+ project.projectList.forall {
+ case Alias(attribute: Attribute, _) =>
+ !project.outputSet.contains(attribute)
+ case _ => true
+ }
+ case aggregate: Aggregate =>
+ aggregate.aggregateExpressions.forall {
+ case Alias(attribute: Attribute, _) =>
+ !aggregate.outputSet.contains(attribute)
+ case _ => true
+ }
+ case window: Window =>
+ window.windowExpressions.forall {
+ case Alias(attribute: Attribute, _) =>
+ !window.outputSet.contains(attribute)
+ case _ => true
+ }
+ case other => true
+ }
+ }
+
def apply(plan: LogicalPlan): LogicalPlan = removeRedundantAliases(plan, AttributeSet.empty)
}
@@ -936,6 +967,15 @@ object PushProjectionThroughUnion extends Rule[LogicalPlan] {
result.asInstanceOf[A]
}
+ /**
+ * If [[SQLConf.UNION_IS_RESOLVED_WHEN_DUPLICATES_PER_CHILD_RESOLVED]] is true, [[Project]] can
+ * only be pushed down if there are no duplicate [[ExprId]]s in the project list.
+ */
+ def canPushProjectionThroughUnion(project: Project): Boolean = {
+ !conf.unionIsResolvedWhenDuplicatesPerChildResolved ||
+ project.outputSet.size == project.projectList.size
+ }
+
def pushProjectionThroughUnion(projectList: Seq[NamedExpression], u: Union): Seq[LogicalPlan] = {
val newFirstChild = Project(projectList, u.children.head)
val newOtherChildren = u.children.tail.map { child =>
@@ -949,8 +989,9 @@ object PushProjectionThroughUnion extends Rule[LogicalPlan] {
_.containsAllPatterns(UNION, PROJECT)) {
// Push down deterministic projection through UNION ALL
- case Project(projectList, u: Union)
- if projectList.forall(_.deterministic) && u.children.nonEmpty =>
+ case project @ Project(projectList, u: Union)
+ if projectList.forall(_.deterministic) && u.children.nonEmpty &&
+ canPushProjectionThroughUnion(project) =>
u.copy(children = pushProjectionThroughUnion(projectList, u))
}
}
@@ -1041,8 +1082,7 @@ object ColumnPruning extends Rule[LogicalPlan] {
p
}
- // TODO: Pruning `UnionLoop`s needs to take into account both the outer `Project` and the inner
- // `UnionLoopRef` nodes.
+ // Avoid pruning UnionLoop because of its recursive nature.
case p @ Project(_, _: UnionLoop) => p
// Prune unnecessary window expressions
@@ -1280,8 +1320,22 @@ object CollapseProject extends Rule[LogicalPlan] with AliasHelper {
def buildCleanedProjectList(
upper: Seq[NamedExpression],
lower: Seq[NamedExpression]): Seq[NamedExpression] = {
+ val explicitlyPreserveAliasMetadata =
+ conf.getConf(SQLConf.PRESERVE_ALIAS_METADATA_WHEN_COLLAPSING_PROJECTS)
val aliases = getAliasMap(lower)
- upper.map(replaceAliasButKeepName(_, aliases))
+ upper.map {
+ case alias: Alias if !alias.metadata.isEmpty && explicitlyPreserveAliasMetadata =>
+ replaceAliasButKeepName(alias, aliases) match {
+ case newAlias: Alias => Alias(child = newAlias.child, name = newAlias.name)(
+ exprId = newAlias.exprId,
+ qualifier = newAlias.qualifier,
+ explicitMetadata = Some(alias.metadata),
+ nonInheritableMetadataKeys = newAlias.nonInheritableMetadataKeys
+ )
+ case other => other
+ }
+ case other => replaceAliasButKeepName(other, aliases)
+ }
}
/**
@@ -1576,7 +1630,7 @@ object InferFiltersFromConstraints extends Rule[LogicalPlan]
*/
object CombineUnions extends Rule[LogicalPlan] {
import CollapseProject.{buildCleanedProjectList, canCollapseExpressions}
- import PushProjectionThroughUnion.pushProjectionThroughUnion
+ import PushProjectionThroughUnion.{canPushProjectionThroughUnion, pushProjectionThroughUnion}
def apply(plan: LogicalPlan): LogicalPlan = plan.transformDownWithPruning(
_.containsAnyPattern(UNION, DISTINCT_LIKE), ruleId) {
@@ -1621,17 +1675,19 @@ object CombineUnions extends Rule[LogicalPlan] {
stack.pushAll(children.reverse)
// Push down projection through Union and then push pushed plan to Stack if
// there is a Project.
- case Project(projectList, Distinct(u @ Union(children, byName, allowMissingCol)))
+ case project @ Project(projectList, Distinct(u @ Union(children, byName, allowMissingCol)))
if projectList.forall(_.deterministic) && children.nonEmpty &&
- flattenDistinct && byName == topByName && allowMissingCol == topAllowMissingCol =>
+ flattenDistinct && byName == topByName && allowMissingCol == topAllowMissingCol &&
+ canPushProjectionThroughUnion(project) =>
stack.pushAll(pushProjectionThroughUnion(projectList, u).reverse)
- case Project(projectList, Deduplicate(keys: Seq[Attribute], u: Union))
+ case project @ Project(projectList, Deduplicate(keys: Seq[Attribute], u: Union))
if projectList.forall(_.deterministic) && flattenDistinct && u.byName == topByName &&
- u.allowMissingCol == topAllowMissingCol && AttributeSet(keys) == u.outputSet =>
+ u.allowMissingCol == topAllowMissingCol && AttributeSet(keys) == u.outputSet &&
+ canPushProjectionThroughUnion(project) =>
stack.pushAll(pushProjectionThroughUnion(projectList, u).reverse)
- case Project(projectList, u @ Union(children, byName, allowMissingCol))
- if projectList.forall(_.deterministic) && children.nonEmpty &&
- byName == topByName && allowMissingCol == topAllowMissingCol =>
+ case project @ Project(projectList, u @ Union(children, byName, allowMissingCol))
+ if projectList.forall(_.deterministic) && children.nonEmpty && byName == topByName &&
+ allowMissingCol == topAllowMissingCol && canPushProjectionThroughUnion(project) =>
stack.pushAll(pushProjectionThroughUnion(projectList, u).reverse)
case child =>
flattened += child
@@ -2574,6 +2630,28 @@ object GenerateOptimization extends Rule[LogicalPlan] {
p.withNewChildren(Seq(updatedGenerate))
case _ => p
}
+
+ case p @ Project(_, g: Generate) if g.generator.isInstanceOf[JsonTuple] =>
+ val generatorOutput = g.generatorOutput
+ val usedOutputs =
+ AttributeSet(generatorOutput).intersect(AttributeSet(p.projectList.flatMap(_.references)))
+
+ usedOutputs.size match {
+ case 0 =>
+ p.withNewChildren(g.children)
+ case n if n < generatorOutput.size =>
+ val originJsonTuple = g.generator.asInstanceOf[JsonTuple]
+ val (newJsonExpressions, newGeneratorOutput) =
+ generatorOutput.zipWithIndex.collect {
+ case (attr, i) if usedOutputs.contains(attr) =>
+ (originJsonTuple.children(i + 1), attr)
+ }.unzip
+ p.withNewChildren(Seq(g.copy(
+ generator = JsonTuple(originJsonTuple.children.head +: newJsonExpressions),
+ generatorOutput = newGeneratorOutput)))
+ case _ =>
+ p
+ }
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregates.scala
index d6a4bd030c9d6..b4602d0ddcc93 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregates.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregates.scala
@@ -54,7 +54,13 @@ object RemoveRedundantAggregates extends Rule[LogicalPlan] with AliasHelper {
.map(_.toAttribute)
))
- upperHasNoDuplicateSensitiveAgg && upperRefsOnlyDeterministicNonAgg
+ // If the lower aggregation is global, it is not redundant because a project with
+ // non-aggregate expressions is different with global aggregation in semantics.
+ // E.g., if the input relation is empty, a project might be optimized to an empty
+ // relation, while a global aggregation will return a single row.
+ lazy val lowerIsGlobalAgg = lower.groupingExpressions.isEmpty
+
+ upperHasNoDuplicateSensitiveAgg && upperRefsOnlyDeterministicNonAgg && !lowerIsGlobalAgg
}
private def isDuplicateSensitive(ae: AggregateExpression): Boolean = {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicate.scala
index 0358c45815944..7134c3daf3baa 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicate.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicate.scala
@@ -19,7 +19,6 @@ package org.apache.spark.sql.catalyst.optimizer
import org.apache.spark.SparkIllegalArgumentException
import org.apache.spark.internal.LogKeys.{SQL_TEXT, UNSUPPORTED_EXPR}
-import org.apache.spark.internal.MDC
import org.apache.spark.sql.catalyst.expressions.{And, ArrayExists, ArrayFilter, CaseWhen, EqualNullSafe, Expression, If, In, InSet, LambdaFunction, Literal, MapFilter, Not, Or}
import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLiteral}
import org.apache.spark.sql.catalyst.plans.logical.{DeleteAction, DeleteFromTable, Filter, InsertAction, InsertStarAction, Join, LogicalPlan, MergeAction, MergeIntoTable, ReplaceData, UpdateAction, UpdateStarAction, UpdateTable, WriteDelta}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
index b3236bbfa3755..856236750f7bf 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
@@ -59,7 +59,20 @@ object ConstantFolding extends Rule[LogicalPlan] {
case _ => false
}
- private def constantFolding(
+ private def tryFold(expr: Expression, isConditionalBranch: Boolean): Expression = {
+ try {
+ Literal.create(expr.freshCopyIfContainsStatefulExpression().eval(EmptyRow), expr.dataType)
+ } catch {
+ case NonFatal(_) if isConditionalBranch =>
+ // When doing constant folding inside conditional expressions, we should not fail
+ // during expression evaluation, as the branch we are evaluating may not be reached at
+ // runtime, and we shouldn't fail the query, to match the original behavior.
+ expr.setTagValue(FAILED_TO_EVALUATE, ())
+ expr
+ }
+ }
+
+ private[sql] def constantFolding(
e: Expression,
isConditionalBranch: Boolean = false): Expression = e match {
case c: ConditionalExpression if !c.foldable =>
@@ -78,17 +91,7 @@ object ConstantFolding extends Rule[LogicalPlan] {
case e if e.getTagValue(FAILED_TO_EVALUATE).isDefined => e
// Fold expressions that are foldable.
- case e if e.foldable =>
- try {
- Literal.create(e.freshCopyIfContainsStatefulExpression().eval(EmptyRow), e.dataType)
- } catch {
- case NonFatal(_) if isConditionalBranch =>
- // When doing constant folding inside conditional expressions, we should not fail
- // during expression evaluation, as the branch we are evaluating may not be reached at
- // runtime, and we shouldn't fail the query, to match the original behavior.
- e.setTagValue(FAILED_TO_EVALUATE, ())
- e
- }
+ case e if e.foldable => tryFold(e, isConditionalBranch)
// Don't replace ScalarSubquery if its plan is an aggregate that may suffer from a COUNT bug.
case s @ ScalarSubquery(_, _, _, _, _, mayHaveCountBug, _)
@@ -100,7 +103,13 @@ object ConstantFolding extends Rule[LogicalPlan] {
case s: ScalarSubquery if s.plan.maxRows.contains(0) =>
Literal(null, s.dataType)
- case other => other.mapChildren(constantFolding(_, isConditionalBranch))
+ case other =>
+ val newOther = other.mapChildren(constantFolding(_, isConditionalBranch))
+ if (newOther.foldable) {
+ tryFold(newOther, isConditionalBranch)
+ } else {
+ newOther
+ }
}
def apply(plan: LogicalPlan): LogicalPlan = plan.transformWithPruning(AlwaysProcess.fn, ruleId) {
@@ -726,7 +735,7 @@ object SupportedBinaryExpr {
case _: BinaryArithmetic => Some(expr, expr.children.head, expr.children.last)
case _: BinaryMathExpression => Some(expr, expr.children.head, expr.children.last)
case _: AddMonths | _: DateAdd | _: DateAddInterval | _: DateDiff | _: DateSub |
- _: DateAddYMInterval | _: TimestampAddYMInterval | _: TimeAdd =>
+ _: DateAddYMInterval | _: TimestampAddYMInterval | _: TimestampAddInterval =>
Some(expr, expr.children.head, expr.children.last)
case _: FindInSet | _: RoundBase => Some(expr, expr.children.head, expr.children.last)
case BinaryPredicate(expr) =>
@@ -743,10 +752,11 @@ object SupportedBinaryExpr {
object LikeSimplification extends Rule[LogicalPlan] with PredicateHelper {
// if guards below protect from escapes on trailing %.
// Cases like "something\%" are not optimized, but this does not affect correctness.
- private val startsWith = "([^_%]+)%".r
- private val endsWith = "%([^_%]+)".r
- private val startsAndEndsWith = "([^_%]+)%([^_%]+)".r
- private val contains = "%([^_%]+)%".r
+ // Consecutive wildcard characters are equivalent to a single wildcard character.
+ private val startsWith = "([^_%]+)%+".r
+ private val endsWith = "%+([^_%]+)".r
+ private val startsAndEndsWith = "([^_%]+)%+([^_%]+)".r
+ private val contains = "%+([^_%]+)%+".r
private val equalTo = "([^_%]*)".r
private def simplifyLike(
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala
index 21e09f2e56d19..c9c26d473b982 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala
@@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.trees.TreePattern._
import org.apache.spark.sql.catalyst.trees.TreePatternBits
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.catalyst.util.DateTimeUtils.{convertSpecialDate, convertSpecialTimestamp, convertSpecialTimestampNTZ, instantToMicros, localDateTimeToMicros}
-import org.apache.spark.sql.catalyst.util.SparkDateTimeUtils.{instantToMicrosOfDay, truncateTimeMicrosToPrecision}
+import org.apache.spark.sql.catalyst.util.SparkDateTimeUtils.{instantToNanosOfDay, truncateTimeToPrecision}
import org.apache.spark.sql.catalyst.util.TypeUtils.toSQLExpr
import org.apache.spark.sql.connector.catalog.CatalogManager
import org.apache.spark.sql.types._
@@ -114,7 +114,7 @@ object ComputeCurrentTime extends Rule[LogicalPlan] {
val instant = Instant.now()
val currentTimestampMicros = instantToMicros(instant)
val currentTime = Literal.create(currentTimestampMicros, TimestampType)
- val currentTimeOfDayMicros = instantToMicrosOfDay(instant, conf.sessionLocalTimeZone)
+ val currentTimeOfDayNanos = instantToNanosOfDay(instant, conf.sessionLocalTimeZone)
val timezone = Literal.create(conf.sessionLocalTimeZone, StringType)
val currentDates = collection.mutable.HashMap.empty[ZoneId, Literal]
val localTimestamps = collection.mutable.HashMap.empty[ZoneId, Literal]
@@ -132,7 +132,7 @@ object ComputeCurrentTime extends Rule[LogicalPlan] {
DateTimeUtils.microsToDays(currentTimestampMicros, cd.zoneId), DateType)
})
case currentTimeType : CurrentTime =>
- val truncatedTime = truncateTimeMicrosToPrecision(currentTimeOfDayMicros,
+ val truncatedTime = truncateTimeToPrecision(currentTimeOfDayNanos,
currentTimeType.precision)
Literal.create(truncatedTime, TimeType(currentTimeType.precision))
case CurrentTimestamp() | Now() => currentTime
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala
index 9413bd7b454d4..be07b440a118b 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.optimizer
import scala.annotation.tailrec
import scala.util.control.NonFatal
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{HASH_JOIN_KEYS, JOIN_CONDITION}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
index 9b9ff2175457b..4e930280381c1 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
@@ -28,28 +28,30 @@ import org.antlr.v4.runtime.{ParserRuleContext, RuleContext, Token}
import org.antlr.v4.runtime.tree.{ParseTree, RuleNode, TerminalNode}
import org.apache.spark.{SparkArithmeticException, SparkException, SparkIllegalArgumentException, SparkThrowable, SparkThrowableHelper}
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.PARTITION_SPECIFICATION
-import org.apache.spark.sql.catalyst.{EvaluateUnresolvedInlineTable, FunctionIdentifier, SQLConfHelper, TableIdentifier}
+import org.apache.spark.sql.catalyst.{FunctionIdentifier, SQLConfHelper, TableIdentifier}
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FUNC_ALIAS
import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, ClusterBySpec}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate.{AnyValue, First, Last}
+import org.apache.spark.sql.catalyst.expressions.json.JsonPathParser
+import org.apache.spark.sql.catalyst.expressions.json.PathInstruction.Named
import org.apache.spark.sql.catalyst.parser.SqlBaseParser._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, Origin}
import org.apache.spark.sql.catalyst.trees.TreePattern.PARAMETER
import org.apache.spark.sql.catalyst.types.DataTypeUtils
-import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, CollationFactory, DateTimeUtils, IntervalUtils, SparkParserUtils}
+import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, CollationFactory, DateTimeUtils, EvaluateUnresolvedInlineTable, IntervalUtils}
import org.apache.spark.sql.catalyst.util.DateTimeUtils.{convertSpecialDate, convertSpecialTimestamp, convertSpecialTimestampNTZ, getZoneId, stringToDate, stringToTime, stringToTimestamp, stringToTimestampWithoutTimeZone}
import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsNamespaces, TableCatalog, TableWritePrivilege}
import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition
import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, Expression => V2Expression, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform}
import org.apache.spark.sql.errors.{DataTypeErrorsBase, QueryCompilationErrors, QueryParsingErrors, SqlScriptingErrors}
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.internal.SQLConf.LEGACY_BANG_EQUALS_NOT
+import org.apache.spark.sql.internal.SQLConf.{LEGACY_BANG_EQUALS_NOT, LEGACY_CONSECUTIVE_STRING_LITERALS}
import org.apache.spark.sql.types._
import org.apache.spark.sql.util.CaseInsensitiveStringMap
import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
@@ -128,7 +130,7 @@ class AstBuilder extends DataTypeAstBuilder
* @return The original input text, including all whitespaces and formatting.
*/
private def getOriginalText(ctx: ParserRuleContext): String = {
- SparkParserUtils.source(ctx)
+ source(ctx)
}
/**
@@ -555,6 +557,7 @@ class AstBuilder extends DataTypeAstBuilder
val query = withOrigin(queryCtx) {
SingleStatement(visitQuery(queryCtx))
}
+ parsingCtx.labelContext.enterForScope(Option(ctx.multipartIdentifier()))
val varName = Option(ctx.multipartIdentifier()).map(_.getText)
val body = visitCompoundBodyImpl(
ctx.compoundBody(),
@@ -562,6 +565,7 @@ class AstBuilder extends DataTypeAstBuilder
parsingCtx,
isScope = false
)
+ parsingCtx.labelContext.exitForScope(Option(ctx.multipartIdentifier()))
parsingCtx.labelContext.exitLabeledScope(Option(ctx.beginLabel()))
ForStatement(query, varName, body, Some(labelText))
@@ -1823,7 +1827,7 @@ class AstBuilder extends DataTypeAstBuilder
// syntax error here accordingly.
val error: String = (if (n.name != null) n.name else n.identifierList).getText
throw new ParseException(
- command = Some(SparkParserUtils.command(n)),
+ command = Some(command(n)),
start = Origin(),
errorClass = "PARSE_SYNTAX_ERROR",
messageParameters = Map(
@@ -3322,6 +3326,24 @@ class AstBuilder extends DataTypeAstBuilder
}
}
+ /**
+ * Create a [[SemiStructuredExtract]] expression.
+ */
+ override def visitSemiStructuredExtract(
+ ctx: SemiStructuredExtractContext): Expression = withOrigin(ctx) {
+ val field = ctx.path.getText
+ // When `field` starts with a bracket, do not add a `.` as the bracket already implies nesting
+ // Also the bracket will imply case sensitive field extraction.
+ val path = if (field.startsWith("[")) "$" + field else s"$$.$field"
+ val parsedPath = JsonPathParser.parse(path)
+ if (parsedPath.isEmpty) {
+ throw new ParseException(errorClass = "PARSE_SYNTAX_ERROR", ctx = ctx)
+ }
+ val potentialAlias = parsedPath.get.collect { case Named(name) => name }.lastOption
+ val node = SemiStructuredExtract(expression(ctx.col), path)
+ potentialAlias.map { colName => Alias(node, colName)() }.getOrElse(node)
+ }
+
/**
* Create an [[UnresolvedAttribute]] expression or a [[UnresolvedRegex]] if it is a regex
* quoted in ``
@@ -3622,6 +3644,8 @@ class AstBuilder extends DataTypeAstBuilder
private def createString(ctx: StringLiteralContext): String = {
if (conf.escapedStringLiterals) {
ctx.stringLit.asScala.map(x => stringWithoutUnescape(visitStringLit(x))).mkString
+ } else if (conf.getConf(LEGACY_CONSECUTIVE_STRING_LITERALS)) {
+ ctx.stringLit.asScala.map(x => stringIgnoreQuoteQuote(visitStringLit(x))).mkString
} else {
ctx.stringLit.asScala.map(x => string(visitStringLit(x))).mkString
}
@@ -4872,8 +4896,23 @@ class AstBuilder extends DataTypeAstBuilder
.mkString(", ")
throw QueryParsingErrors.multiplePrimaryKeysError(ctx, primaryKeyColumns)
}
+ // If there is a primary key constraint, all the columns in the primary key are not null.
+ val updatedColumns = if (primaryKeys.nonEmpty) {
+ val lowerCasePkColumns = primaryKeys.head.asInstanceOf[PrimaryKeyConstraint].columns
+ .map(_.toLowerCase(Locale.ROOT))
+ columnDefs.map { colDef =>
+ if (colDef.nullable &&
+ lowerCasePkColumns.contains(colDef.name.toLowerCase(Locale.ROOT))) {
+ colDef.copy(nullable = false)
+ } else {
+ colDef
+ }
+ }
+ } else {
+ columnDefs
+ }
- (columnDefs.toSeq, constraints.toSeq)
+ (updatedColumns.toSeq, constraints.toSeq)
}
}
@@ -6340,7 +6379,7 @@ class AstBuilder extends DataTypeAstBuilder
*
* For example:
* {{{
- * DECLARE [OR REPLACE] [VARIABLE] [db_name.]variable_name
+ * DECLARE [OR REPLACE] [VARIABLE] variable_name [COMMA variable_name]*
* [dataType] [defaultExpression];
* }}}
*
@@ -6353,7 +6392,7 @@ class AstBuilder extends DataTypeAstBuilder
throw new ParseException(
errorClass = "INVALID_SQL_SYNTAX.VARIABLE_TYPE_OR_DEFAULT_REQUIRED",
messageParameters = Map.empty,
- ctx.identifierReference)
+ ctx.identifierReferences.get(0))
}
DefaultValueExpression(Literal(null, dataTypeOpt.get), "null")
} else {
@@ -6361,7 +6400,11 @@ class AstBuilder extends DataTypeAstBuilder
dataTypeOpt.map { dt => default.copy(child = Cast(default.child, dt)) }.getOrElse(default)
}
CreateVariable(
- withIdentClause(ctx.identifierReference(), UnresolvedIdentifier(_)),
+ ctx.identifierReferences.asScala.map (
+ identifierReference => {
+ withIdentClause(identifierReference, UnresolvedIdentifier(_))
+ }
+ ).toSeq,
defaultExpression,
ctx.REPLACE() != null
)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala
index 38e92cf9aebdd..a19b4cca28173 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala
@@ -28,7 +28,7 @@ import org.antlr.v4.runtime.tree.{ParseTree, TerminalNodeImpl}
import org.apache.spark.SparkException
import org.apache.spark.sql.catalyst.analysis.UnresolvedIdentifier
-import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{BeginLabelContext, EndLabelContext}
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{BeginLabelContext, EndLabelContext, MultipartIdentifierContext}
import org.apache.spark.sql.catalyst.plans.logical.{CreateVariable, ErrorCondition}
import org.apache.spark.sql.catalyst.trees.CurrentOrigin
import org.apache.spark.sql.catalyst.util.SparkParserUtils
@@ -151,7 +151,8 @@ class CompoundBodyParsingContext {
def variable(createVariable: CreateVariable, allowVarDeclare: Boolean): Unit = {
if (!allowVarDeclare) {
throw SqlScriptingErrors.variableDeclarationNotAllowedInScope(
- createVariable.origin, createVariable.name.asInstanceOf[UnresolvedIdentifier].nameParts)
+ createVariable.origin,
+ createVariable.names(0).asInstanceOf[UnresolvedIdentifier].nameParts)
}
transitionTo(State.VARIABLE, createVariable = Some(createVariable), None)
}
@@ -229,12 +230,12 @@ class CompoundBodyParsingContext {
case (State.STATEMENT, State.VARIABLE) =>
throw SqlScriptingErrors.variableDeclarationOnlyAtBeginning(
createVariable.get.origin,
- createVariable.get.name.asInstanceOf[UnresolvedIdentifier].nameParts)
+ createVariable.get.names(0).asInstanceOf[UnresolvedIdentifier].nameParts)
case (State.HANDLER, State.VARIABLE) =>
throw SqlScriptingErrors.variableDeclarationOnlyAtBeginning(
createVariable.get.origin,
- createVariable.get.name.asInstanceOf[UnresolvedIdentifier].nameParts)
+ createVariable.get.names(0).asInstanceOf[UnresolvedIdentifier].nameParts)
// Invalid transitions to CONDITION state.
case (State.STATEMENT, State.CONDITION) =>
@@ -245,7 +246,7 @@ class CompoundBodyParsingContext {
case (State.HANDLER, State.CONDITION) =>
throw SqlScriptingErrors.variableDeclarationOnlyAtBeginning(
createVariable.get.origin,
- createVariable.get.name.asInstanceOf[UnresolvedIdentifier].nameParts)
+ createVariable.get.names(0).asInstanceOf[UnresolvedIdentifier].nameParts)
// Invalid transitions to HANDLER state.
case (State.STATEMENT, State.HANDLER) =>
@@ -316,6 +317,23 @@ class SqlScriptingLabelContext {
beginLabelCtx.map(_.multipartIdentifier().getText).isDefined
}
+ /**
+ * Assert the identifier is not contained within seenLabels.
+ * If the identifier is contained within seenLabels, raise an exception.
+ */
+ private def assertIdentifierNotInSeenLabels(
+ identifierCtx: Option[MultipartIdentifierContext]): Unit = {
+ identifierCtx.foreach { ctx =>
+ val identifierName = ctx.getText
+ if (seenLabels.contains(identifierName.toLowerCase(Locale.ROOT))) {
+ withOrigin(ctx) {
+ throw SqlScriptingErrors
+ .duplicateLabels(CurrentOrigin.get, identifierName.toLowerCase(Locale.ROOT))
+ }
+ }
+ }
+ }
+
/**
* Enter a labeled scope and return the label text.
* If the label is defined, it will be returned and added to seenLabels.
@@ -342,9 +360,9 @@ class SqlScriptingLabelContext {
// Do not add the label to the seenLabels set if it is not defined.
java.util.UUID.randomUUID.toString.toLowerCase(Locale.ROOT)
}
- if (SqlScriptingLabelContext.isForbiddenLabelName(labelText)) {
+ if (SqlScriptingLabelContext.isForbiddenLabelOrForVariableName(labelText)) {
withOrigin(beginLabelCtx.get) {
- throw SqlScriptingErrors.labelNameForbidden(CurrentOrigin.get, labelText)
+ throw SqlScriptingErrors.labelOrForVariableNameForbidden(CurrentOrigin.get, labelText)
}
}
labelText
@@ -359,13 +377,46 @@ class SqlScriptingLabelContext {
seenLabels.remove(beginLabelCtx.get.multipartIdentifier().getText.toLowerCase(Locale.ROOT))
}
}
+
+ /**
+ * Enter a for loop scope.
+ * If the for loop variable is defined, it will be asserted to not be inside seenLabels;
+ * Then, if the for loop variable is defined, it will be added to seenLabels.
+ */
+ def enterForScope(identifierCtx: Option[MultipartIdentifierContext]): Unit = {
+ identifierCtx.foreach { ctx =>
+ val identifierName = ctx.getText
+ assertIdentifierNotInSeenLabels(identifierCtx)
+ seenLabels.add(identifierName.toLowerCase(Locale.ROOT))
+
+ if (SqlScriptingLabelContext.isForbiddenLabelOrForVariableName(identifierName)) {
+ withOrigin(ctx) {
+ throw SqlScriptingErrors.labelOrForVariableNameForbidden(
+ CurrentOrigin.get,
+ identifierName.toLowerCase(Locale.ROOT))
+ }
+ }
+ }
+ }
+
+ /**
+ * Exit a for loop scope.
+ * If the for loop variable is defined, it will be removed from seenLabels.
+ */
+ def exitForScope(identifierCtx: Option[MultipartIdentifierContext]): Unit = {
+ identifierCtx.foreach { ctx =>
+ val identifierName = ctx.getText
+ seenLabels.remove(identifierName.toLowerCase(Locale.ROOT))
+ }
+ }
+
}
object SqlScriptingLabelContext {
private val forbiddenLabelNames: immutable.Set[Regex] =
immutable.Set("builtin".r, "session".r, "sys.*".r)
- def isForbiddenLabelName(labelName: String): Boolean = {
+ def isForbiddenLabelOrForVariableName(labelName: String): Boolean = {
forbiddenLabelNames.exists(_.matches(labelName.toLowerCase(Locale.ROOT)))
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/NormalizePlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/NormalizePlan.scala
index b679514a26056..df2f95e1cf1e0 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/NormalizePlan.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/NormalizePlan.scala
@@ -19,11 +19,8 @@ package org.apache.spark.sql.catalyst.plans
import java.util.HashMap
-import org.apache.spark.sql.catalyst.analysis.{
- DeduplicateRelations,
- GetViewColumnByNameAndOrdinal,
- NormalizeableRelation
-}
+import org.apache.spark.sql.catalyst.analysis.NormalizeableRelation
+import org.apache.spark.sql.catalyst.analysis.resolver.ResolverTag
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
import org.apache.spark.sql.catalyst.optimizer.ReplaceExpressions
@@ -150,24 +147,42 @@ object NormalizePlan extends PredicateHelper {
.reduce(And)
Join(left, right, newJoinType, Some(newCondition), hint)
case project: Project
- if project
- .getTagValue(DeduplicateRelations.PROJECT_FOR_EXPRESSION_ID_DEDUPLICATION)
- .isDefined =>
+ if project.getTagValue(ResolverTag.PROJECT_FOR_EXPRESSION_ID_DEDUPLICATION).isDefined =>
project.child
+
case aggregate @ Aggregate(_, _, innerProject: Project, _) =>
- val newInnerProject = Project(
- innerProject.projectList.sortBy(_.name),
- innerProject.child
- )
- aggregate.copy(child = newInnerProject)
- case Project(outerProjectList, innerProject: Project) =>
- val newInnerProject = Project(
- innerProject.projectList.sortBy(_.name),
- innerProject.child
+ aggregate.copy(child = normalizeProjectListOrder(innerProject))
+
+ case project @ Project(_, innerProject: Project) =>
+ project.copy(child = normalizeProjectListOrder(innerProject))
+
+ case project @ Project(_, innerAggregate: Aggregate) =>
+ project.copy(child = normalizeAggregateListOrder(innerAggregate))
+
+ /**
+ * ORDER BY covered by an output-retaining project on top of GROUP BY
+ */
+ case project @ Project(_, sort @ Sort(_, _, innerAggregate: Aggregate, _)) =>
+ project.copy(child = sort.copy(child = normalizeAggregateListOrder(innerAggregate)))
+
+ /**
+ * HAVING covered by an output-retaining project on top of GROUP BY
+ */
+ case project @ Project(_, filter @ Filter(_, innerAggregate: Aggregate)) =>
+ project.copy(child = filter.copy(child = normalizeAggregateListOrder(innerAggregate)))
+
+ /**
+ * HAVING ... ORDER BY covered by an output-retaining project on top of GROUP BY
+ */
+ case project @ Project(
+ _,
+ sort @ Sort(_, _, filter @ Filter(_, innerAggregate: Aggregate), _)
+ ) =>
+ project.copy(
+ child =
+ sort.copy(child = filter.copy(child = normalizeAggregateListOrder(innerAggregate)))
)
- Project(normalizeProjectList(outerProjectList), newInnerProject)
- case Project(projectList, child) =>
- Project(normalizeProjectList(projectList), child)
+
case c: KeepAnalyzedQuery => c.storeAnalyzedQuery()
case localRelation: LocalRelation if !localRelation.data.isEmpty =>
/**
@@ -204,14 +219,12 @@ object NormalizePlan extends PredicateHelper {
case _ => condition // Don't reorder.
}
- private def normalizeProjectList(projectList: Seq[NamedExpression]): Seq[NamedExpression] = {
- projectList
- .map { e =>
- e.transformUp {
- case g: GetViewColumnByNameAndOrdinal => g.copy(viewDDL = None)
- }
- }
- .asInstanceOf[Seq[NamedExpression]]
+ private def normalizeProjectListOrder(project: Project): Project = {
+ project.copy(projectList = project.projectList.sortBy(_.name))
+ }
+
+ private def normalizeAggregateListOrder(aggregate: Aggregate): Aggregate = {
+ aggregate.copy(aggregateExpressions = aggregate.aggregateExpressions.sortBy(_.name))
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
index 3eb4cdef9ea9b..7801cd347f7dc 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
@@ -24,6 +24,7 @@ import scala.collection.mutable
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.analysis.UnresolvedException
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.rules.RuleId
import org.apache.spark.sql.catalyst.rules.UnknownRuleId
@@ -55,6 +56,32 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]]
def output: Seq[Attribute]
+ override def nodeWithOutputColumnsString(maxColumns: Int): String = {
+ try {
+ nodeName + {
+ if (this.output.length > maxColumns) {
+ val outputWithNullability = this.output.take(maxColumns).map { attr =>
+ attr.toString + s"[nullable=${attr.nullable}]"
+ }
+
+ outputWithNullability.mkString("
+
+ org.scalameta
+ semanticdb-shared_${scala.binary.version}
+ 4.13.1.1
+
+
+ org.scala-lang
+ scalap
+
+
+ provided
+ org.apache.sparkspark-tags_${scala.binary.version}test-jartest
+
+ commons-io
+ commons-io
+ test
+ org.scalacheckscalacheck_${scala.binary.version}
diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala
index a548ec7007dbe..b760828a1e99c 100644
--- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala
+++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala
@@ -2369,6 +2369,14 @@ class PlanGenerationTestSuite
fn.to_date(fn.col("s"), "yyyy-MM-dd")
}
+ temporalFunctionTest("try_to_date") {
+ fn.try_to_date(fn.col("s"))
+ }
+
+ temporalFunctionTest("try_to_date with format") {
+ fn.try_to_date(fn.col("s"), "yyyy-MM-dd")
+ }
+
temporalFunctionTest("xpath") {
fn.xpath(fn.col("s"), lit("a/b/text()"))
}
@@ -3319,6 +3327,7 @@ class PlanGenerationTestSuite
fn.lit(java.sql.Date.valueOf("2023-02-23")),
fn.lit(java.time.Duration.ofSeconds(200L)),
fn.lit(java.time.Period.ofDays(100)),
+ fn.lit(java.time.LocalTime.of(23, 59, 59, 999999999)),
fn.lit(new CalendarInterval(2, 20, 100L)))
}
@@ -3389,13 +3398,14 @@ class PlanGenerationTestSuite
fn.typedLit(java.sql.Date.valueOf("2023-02-23")),
fn.typedLit(java.time.Duration.ofSeconds(200L)),
fn.typedLit(java.time.Period.ofDays(100)),
+ fn.typedLit(java.time.LocalTime.of(23, 59, 59, 999999999)),
fn.typedLit(new CalendarInterval(2, 20, 100L)),
// Handle parameterized scala types e.g.: List, Seq and Map.
fn.typedLit(Some(1)),
fn.typedLit(Array(1, 2, 3)),
fn.typedLit(Seq(1, 2, 3)),
- fn.typedLit(Map("a" -> 1, "b" -> 2)),
+ fn.typedLit(mutable.LinkedHashMap("a" -> 1, "b" -> 2)),
fn.typedLit(("a", 2, 1.0)),
fn.typedLit[Option[Int]](None),
fn.typedLit[Array[Option[Int]]](Array(Some(1))),
@@ -3404,9 +3414,20 @@ class PlanGenerationTestSuite
fn.typedlit[collection.immutable.Map[Int, Option[Int]]](
collection.immutable.Map(1 -> None)),
fn.typedLit(Seq(Seq(1, 2, 3), Seq(4, 5, 6), Seq(7, 8, 9))),
- fn.typedLit(Seq(Map("a" -> 1, "b" -> 2), Map("a" -> 3, "b" -> 4), Map("a" -> 5, "b" -> 6))),
- fn.typedLit(Map(1 -> Map("a" -> 1, "b" -> 2), 2 -> Map("a" -> 3, "b" -> 4))),
- fn.typedLit((Seq(1, 2, 3), Map("a" -> 1, "b" -> 2), ("a", Map(1 -> "a", 2 -> "b")))))
+ fn.typedLit(
+ Seq(
+ mutable.LinkedHashMap("a" -> 1, "b" -> 2),
+ mutable.LinkedHashMap("a" -> 3, "b" -> 4),
+ mutable.LinkedHashMap("a" -> 5, "b" -> 6))),
+ fn.typedLit(
+ mutable.LinkedHashMap(
+ 1 -> mutable.LinkedHashMap("a" -> 1, "b" -> 2),
+ 2 -> mutable.LinkedHashMap("a" -> 3, "b" -> 4))),
+ fn.typedLit(
+ (
+ Seq(1, 2, 3),
+ mutable.LinkedHashMap("a" -> 1, "b" -> 2),
+ ("a", mutable.LinkedHashMap(1 -> "a", 2 -> "b")))))
}
/* Window API */
diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/CatalogSuite.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/CatalogSuite.scala
index b2c19226dc542..bae569978890b 100644
--- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/CatalogSuite.scala
+++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/CatalogSuite.scala
@@ -19,13 +19,12 @@ package org.apache.spark.sql.connect
import java.io.{File, FilenameFilter}
-import org.apache.commons.io.FileUtils
-
import org.apache.spark.SparkException
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.connect.test.{ConnectFunSuite, RemoteSparkSession, SQLHelper}
import org.apache.spark.sql.types.{DoubleType, LongType, StructType}
import org.apache.spark.storage.StorageLevel
+import org.apache.spark.util.SparkFileUtils
class CatalogSuite extends ConnectFunSuite with RemoteSparkSession with SQLHelper {
@@ -258,7 +257,7 @@ class CatalogSuite extends ConnectFunSuite with RemoteSparkSession with SQLHelpe
spark.catalog.cacheTable(tableName)
assert(spark.table(tableName).collect().length == 1)
- FileUtils.deleteDirectory(dir)
+ SparkFileUtils.deleteRecursively(dir)
assert(spark.table(tableName).collect().length == 1)
spark.catalog.refreshTable(tableName)
diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ClientE2ETestSuite.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ClientE2ETestSuite.scala
index 415960e83f9d1..29dd7d9742f44 100644
--- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ClientE2ETestSuite.scala
+++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ClientE2ETestSuite.scala
@@ -18,7 +18,7 @@ package org.apache.spark.sql.connect
import java.io.{ByteArrayOutputStream, PrintStream}
import java.nio.file.Files
-import java.time.DateTimeException
+import java.time.{DateTimeException, LocalTime}
import java.util.Properties
import scala.collection.mutable
@@ -26,7 +26,6 @@ import scala.concurrent.{ExecutionContext, Future}
import scala.concurrent.duration.{DurationInt, FiniteDuration}
import scala.jdk.CollectionConverters._
-import org.apache.commons.io.FileUtils
import org.apache.commons.io.output.TeeOutputStream
import org.scalactic.TolerantNumerics
import org.scalatest.PrivateMethodTester
@@ -46,7 +45,7 @@ import org.apache.spark.sql.connect.test.SparkConnectServerUtils.port
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SqlApiConf
import org.apache.spark.sql.types._
-import org.apache.spark.util.SparkThreadUtils
+import org.apache.spark.util.{SparkFileUtils, SparkSystemUtils, SparkThreadUtils}
class ClientE2ETestSuite
extends QueryTest
@@ -120,7 +119,7 @@ class ClientE2ETestSuite
import session.implicits._
val throwException =
- udf((_: String) => throw new SparkException("test" * 10000))
+ udf((_: String) => throw new SparkException("test".repeat(10000)))
val ex = intercept[SparkException] {
Seq("1").toDS().withColumn("udf_val", throwException($"value")).collect()
@@ -133,7 +132,7 @@ class ClientE2ETestSuite
val cause = ex.getCause.asInstanceOf[SparkException]
assert(cause.getCondition == null)
assert(cause.getMessageParameters.isEmpty)
- assert(cause.getMessage.contains("test" * 10000))
+ assert(cause.getMessage.contains("test".repeat(10000)))
}
}
@@ -228,7 +227,7 @@ class ClientE2ETestSuite
}
test("spark deep recursion") {
- var recursionDepth = if (System.getProperty("os.arch") == "s390x") 400 else 500
+ var recursionDepth = if (SparkSystemUtils.osArch == "s390x") 400 else 500
var df = spark.range(1)
for (a <- 1 to recursionDepth) {
df = df.union(spark.range(a, a + 1))
@@ -346,7 +345,7 @@ class ClientE2ETestSuite
.listFiles()
.filter(file => file.getPath.endsWith(".csv"))(0)
- assert(FileUtils.contentEquals(testDataPath.toFile, outputFile))
+ assert(SparkFileUtils.contentEquals(testDataPath.toFile, outputFile))
}
test("read path collision") {
@@ -1670,6 +1669,53 @@ class ClientE2ETestSuite
}
checkAnswer(df, (0 until 6).map(i => Row(i)))
}
+
+ test("SPARK-52770: Support Time type") {
+ val df = spark.sql("SELECT TIME '12:13:14'")
+
+ checkAnswer(df, Row(LocalTime.of(12, 13, 14)))
+ }
+
+ test("SPARK-53054: DataFrameReader defaults to spark.sql.sources.default") {
+ withTempPath { file =>
+ val path = file.getAbsoluteFile.toURI.toString
+ spark.range(100).write.parquet(file.toPath.toAbsolutePath.toString)
+
+ spark.conf.set("spark.sql.sources.default", "parquet")
+
+ val df = spark.read.load(path)
+ assert(df.count() == 100)
+ }
+ }
+
+ test("SPARK-52930: the nullability of arrays should be preserved using typedlit") {
+ val arrays = Seq(
+ (typedlit(Array[Int]()), false),
+ (typedlit(Array[Int](1)), false),
+ (typedlit(Array[Integer]()), true),
+ (typedlit(Array[Integer](1)), true))
+ for ((array, containsNull) <- arrays) {
+ val df = spark.sql("select 1").select(array)
+ df.createOrReplaceTempView("test_array_nullability")
+ val schema = spark.sql("select * from test_array_nullability").schema
+ assert(schema.fields.head.dataType.asInstanceOf[ArrayType].containsNull === containsNull)
+ }
+ }
+
+ test("SPARK-52930: the nullability of map values should be preserved using typedlit") {
+ val maps = Seq(
+ (typedlit(Map[String, Int]()), false),
+ (typedlit(Map[String, Int]("a" -> 1)), false),
+ (typedlit(Map[String, Integer]()), true),
+ (typedlit(Map[String, Integer]("a" -> 1)), true))
+ for ((map, valueContainsNull) <- maps) {
+ val df = spark.sql("select 1").select(map)
+ df.createOrReplaceTempView("test_map_nullability")
+ val schema = spark.sql("select * from test_map_nullability").schema
+ assert(
+ schema.fields.head.dataType.asInstanceOf[MapType].valueContainsNull === valueContainsNull)
+ }
+ }
}
private[sql] case class ClassData(a: String, b: Int)
diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ColumnNodeToProtoConverterSuite.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ColumnNodeToProtoConverterSuite.scala
index 02f0c35c44a8f..90da125b49ff0 100644
--- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ColumnNodeToProtoConverterSuite.scala
+++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ColumnNodeToProtoConverterSuite.scala
@@ -79,15 +79,24 @@ class ColumnNodeToProtoConverterSuite extends ConnectFunSuite {
Literal((12.0, "north", 60.0, "west"), Option(dataType)),
expr { b =>
val builder = b.getLiteralBuilder.getStructBuilder
- builder.getStructTypeBuilder.getStructBuilder
- .addFields(structField("_1", ProtoDataTypes.DoubleType))
- .addFields(structField("_2", stringTypeWithCollation))
- .addFields(structField("_3", ProtoDataTypes.DoubleType))
- .addFields(structField("_4", stringTypeWithCollation))
- builder.addElements(proto.Expression.Literal.newBuilder().setDouble(12.0))
- builder.addElements(proto.Expression.Literal.newBuilder().setString("north"))
- builder.addElements(proto.Expression.Literal.newBuilder().setDouble(60.0))
- builder.addElements(proto.Expression.Literal.newBuilder().setString("west"))
+ builder
+ .addElements(proto.Expression.Literal.newBuilder().setDouble(12.0).build())
+ builder
+ .addElements(proto.Expression.Literal.newBuilder().setString("north").build())
+ builder
+ .addElements(proto.Expression.Literal.newBuilder().setDouble(60.0).build())
+ builder
+ .addElements(proto.Expression.Literal.newBuilder().setString("west").build())
+ builder.setDataTypeStruct(
+ proto.DataType.Struct
+ .newBuilder()
+ .addFields(
+ proto.DataType.StructField.newBuilder().setName("_1").setNullable(true).build())
+ .addFields(structField("_2", stringTypeWithCollation))
+ .addFields(
+ proto.DataType.StructField.newBuilder().setName("_3").setNullable(true).build())
+ .addFields(structField("_4", stringTypeWithCollation))
+ .build())
})
}
diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/DataFrameSuite.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/DataFrameSuite.scala
new file mode 100644
index 0000000000000..890245fdd2fba
--- /dev/null
+++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/DataFrameSuite.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.connect
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.connect.test.{QueryTest, RemoteSparkSession}
+import org.apache.spark.sql.functions.{col, concat, lit, when}
+
+class DataFrameSuite extends QueryTest with RemoteSparkSession {
+
+ test("drop") {
+ val sparkSession = spark
+ import sparkSession.implicits._
+
+ val df1 = Seq[(String, String, String)](("a", "b", "c")).toDF("colA", "colB", "colC")
+
+ val df2 = Seq[(String, String, String)](("c", "d", "e")).toDF("colC", "colD", "colE")
+
+ val df3 = df1
+ .join(df2, df1.col("colC") === df2.col("colC"))
+ .withColumn(
+ "colB",
+ when(df1.col("colB") === "b", concat(df1.col("colB").cast("string"), lit("x")))
+ .otherwise(df1.col("colB")))
+
+ val df4 = df3.drop(df1.col("colB"))
+
+ assert(df4.columns === Array("colA", "colB", "colC", "colC", "colD", "colE"))
+ assert(df4.count() === 1)
+ }
+
+ test("drop column from different dataframe") {
+ val sparkSession = spark
+
+ val df1 = spark.range(10)
+ val df2 = df1.select(col("id"), lit(0).as("v0"))
+
+ assert(df2.drop(df2.col("id")).columns === Array("v0"))
+ // drop df1.col("id") from df2, which is semantically equal to df2.col("id")
+ // note that df1.drop(df2.col("id")) works in Classic, but not in Connect
+ assert(df2.drop(df1.col("id")).columns === Array("v0"))
+
+ val df3 = df2.select(col("*"), lit(1).as("v1"))
+ assert(df3.drop(df3.col("id")).columns === Array("v0", "v1"))
+ // drop df2.col("id") from df3, which is semantically equal to df3.col("id")
+ assert(df3.drop(df2.col("id")).columns === Array("v0", "v1"))
+ // drop df1.col("id") from df3, which is semantically equal to df3.col("id")
+ assert(df3.drop(df1.col("id")).columns === Array("v0", "v1"))
+
+ assert(df3.drop(df3.col("v0")).columns === Array("id", "v1"))
+ // drop df2.col("v0") from df3, which is semantically equal to df3.col("v0")
+ assert(df3.drop(df2.col("v0")).columns === Array("id", "v1"))
+ }
+
+ test("lazy column validation") {
+ val session = spark
+ import session.implicits._
+
+ val df1 = Seq(1 -> "y").toDF("a", "y")
+ val df2 = Seq(1 -> "x").toDF("a", "x")
+ val df3 = df1.join(df2, df1("a") === df2("a"))
+ val df4 = df3.select(df1("x")) // <- No exception here
+
+ intercept[AnalysisException] { df4.schema }
+ }
+}
diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/SQLImplicitsTestSuite.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/SQLImplicitsTestSuite.scala
index c7b4748f12221..547d5ca7804af 100644
--- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/SQLImplicitsTestSuite.scala
+++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/SQLImplicitsTestSuite.scala
@@ -23,7 +23,6 @@ import java.util.concurrent.atomic.AtomicLong
import io.grpc.inprocess.InProcessChannelBuilder
import org.apache.arrow.memory.RootAllocator
-import org.apache.commons.lang3.SystemUtils
import org.scalatest.BeforeAndAfterAll
import org.apache.spark.sql.{Column, Encoder, SaveMode}
@@ -31,6 +30,7 @@ import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.agnosticEncoderFo
import org.apache.spark.sql.connect.client.SparkConnectClient
import org.apache.spark.sql.connect.client.arrow.{ArrowDeserializers, ArrowSerializer}
import org.apache.spark.sql.connect.test.ConnectFunSuite
+import org.apache.spark.util.SparkSystemUtils
/**
* Test suite for SQL implicits.
@@ -173,7 +173,7 @@ class SQLImplicitsTestSuite extends ConnectFunSuite with BeforeAndAfterAll {
// Spark always converts them to microseconds, this will cause the
// test fail when using Java 17 on Linux, so add `truncatedTo(ChronoUnit.MICROS)` when
// testing on Linux using Java 17 to ensure the accuracy of input data is microseconds.
- if (SystemUtils.IS_OS_LINUX) {
+ if (SparkSystemUtils.isLinux) {
testImplicit(LocalDateTime.now().truncatedTo(ChronoUnit.MICROS))
testImplicit(Instant.now().truncatedTo(ChronoUnit.MICROS))
testImplicit(Timestamp.from(Instant.now().truncatedTo(ChronoUnit.MICROS)))
diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/ClassFinderSuite.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/ClassFinderSuite.scala
index 92cd1acd45d40..5eef1de0a5437 100644
--- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/ClassFinderSuite.scala
+++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/ClassFinderSuite.scala
@@ -18,8 +18,6 @@ package org.apache.spark.sql.connect.client
import java.nio.file.Paths
-import org.apache.commons.io.FileUtils
-
import org.apache.spark.sql.connect.test.ConnectFunSuite
import org.apache.spark.util.SparkFileUtils
@@ -32,7 +30,7 @@ class ClassFinderSuite extends ConnectFunSuite {
requiredClasses.foreach(className =>
assume(classResourcePath.resolve(className).toFile.exists))
val copyDir = SparkFileUtils.createTempDir().toPath
- FileUtils.copyDirectory(classResourcePath.toFile, copyDir.toFile)
+ SparkFileUtils.copyDirectory(classResourcePath.toFile, copyDir.toFile)
val monitor = new REPLClassDirMonitor(copyDir.toAbsolutePath.toString)
def checkClasses(monitor: REPLClassDirMonitor, additionalClasses: Seq[String] = Nil): Unit = {
@@ -50,7 +48,7 @@ class ClassFinderSuite extends ConnectFunSuite {
val subDir = SparkFileUtils.createTempDir(copyDir.toAbsolutePath.toString)
val classToCopy = copyDir.resolve("Hello.class")
val copyLocation = subDir.toPath.resolve("HelloDup.class")
- FileUtils.copyFile(classToCopy.toFile, copyLocation.toFile)
+ SparkFileUtils.copyFile(classToCopy.toFile, copyLocation.toFile)
checkClasses(monitor, Seq(s"${subDir.getName}/HelloDup.class"))
}
diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientRetriesSuite.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientRetriesSuite.scala
new file mode 100644
index 0000000000000..c0738d7de325b
--- /dev/null
+++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientRetriesSuite.scala
@@ -0,0 +1,282 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.connect.client
+
+import scala.concurrent.duration.FiniteDuration
+
+import com.google.protobuf.{Any, Duration}
+import com.google.rpc
+import io.grpc.{Status, StatusRuntimeException}
+import io.grpc.protobuf.StatusProto
+import org.scalatest.BeforeAndAfterEach
+import org.scalatest.concurrent.Eventually
+
+import org.apache.spark.sql.connect.test.ConnectFunSuite
+
+class SparkConnectClientRetriesSuite
+ extends ConnectFunSuite
+ with BeforeAndAfterEach
+ with Eventually {
+
+ private class DummyFn(e: => Throwable, numFails: Int = 3) {
+ var counter = 0
+ def fn(): Int = {
+ if (counter < numFails) {
+ counter += 1
+ throw e
+ } else {
+ 42
+ }
+ }
+ }
+
+ /** Tracks sleep times in milliseconds for testing purposes. */
+ private class SleepTimeTracker {
+ private val data = scala.collection.mutable.ListBuffer[Long]()
+ def sleep(t: Long): Unit = data.append(t)
+ def times: List[Long] = data.toList
+ def totalSleep: Long = data.sum
+ }
+
+ /** Helper function for creating a test exception with retry_delay */
+ private def createTestExceptionWithDetails(
+ msg: String,
+ code: Status.Code = Status.Code.INTERNAL,
+ retryDelay: FiniteDuration = FiniteDuration(0, "s")): StatusRuntimeException = {
+ // In grpc-java, RetryDelay should be specified as seconds: Long + nanos: Int
+ val seconds = retryDelay.toSeconds
+ val nanos = (retryDelay - FiniteDuration(seconds, "s")).toNanos.toInt
+ val retryDelayMsg = Duration
+ .newBuilder()
+ .setSeconds(seconds)
+ .setNanos(nanos)
+ .build()
+ val retryInfo = rpc.RetryInfo
+ .newBuilder()
+ .setRetryDelay(retryDelayMsg)
+ .build()
+ val status = rpc.Status
+ .newBuilder()
+ .setMessage(msg)
+ .setCode(code.value())
+ .addDetails(Any.pack(retryInfo))
+ .build()
+ StatusProto.toStatusRuntimeException(status)
+ }
+
+ /** helper function for comparing two sequences of sleep times */
+ private def assertLongSequencesAlmostEqual(
+ first: Seq[Long],
+ second: Seq[Long],
+ delta: Long): Unit = {
+ assert(first.length == second.length, "Lists have different lengths.")
+ for ((a, b) <- first.zip(second)) {
+ assert(math.abs(a - b) <= delta, s"Elements $a and $b differ by more than $delta.")
+ }
+ }
+
+ test("SPARK-44721: Retries run for a minimum period") {
+ // repeat test few times to avoid random flakes
+ for (_ <- 1 to 10) {
+ val st = new SleepTimeTracker()
+ val dummyFn = new DummyFn(new StatusRuntimeException(Status.UNAVAILABLE), numFails = 100)
+ val retryHandler = new GrpcRetryHandler(RetryPolicy.defaultPolicies(), st.sleep)
+
+ assertThrows[StatusRuntimeException] {
+ retryHandler.retry {
+ dummyFn.fn()
+ }
+ }
+
+ assert(st.totalSleep >= 10 * 60 * 1000) // waited at least 10 minutes
+ }
+ }
+
+ test("SPARK-44275: retry actually retries") {
+ val dummyFn = new DummyFn(new StatusRuntimeException(Status.UNAVAILABLE))
+ val retryPolicies = RetryPolicy.defaultPolicies()
+ val retryHandler = new GrpcRetryHandler(retryPolicies, sleep = _ => {})
+ val result = retryHandler.retry { dummyFn.fn() }
+
+ assert(result == 42)
+ assert(dummyFn.counter == 3)
+ }
+
+ test("SPARK-44275: default retryException retries only on UNAVAILABLE") {
+ val dummyFn = new DummyFn(new StatusRuntimeException(Status.ABORTED))
+ val retryPolicies = RetryPolicy.defaultPolicies()
+ val retryHandler = new GrpcRetryHandler(retryPolicies, sleep = _ => {})
+
+ assertThrows[StatusRuntimeException] {
+ retryHandler.retry { dummyFn.fn() }
+ }
+ assert(dummyFn.counter == 1)
+ }
+
+ test("SPARK-44275: retry uses canRetry to filter exceptions") {
+ val dummyFn = new DummyFn(new StatusRuntimeException(Status.UNAVAILABLE))
+ val retryPolicy = RetryPolicy(canRetry = _ => false, name = "TestPolicy")
+ val retryHandler = new GrpcRetryHandler(retryPolicy)
+
+ assertThrows[StatusRuntimeException] {
+ retryHandler.retry { dummyFn.fn() }
+ }
+ assert(dummyFn.counter == 1)
+ }
+
+ test("SPARK-44275: retry does not exceed maxRetries") {
+ val dummyFn = new DummyFn(new StatusRuntimeException(Status.UNAVAILABLE))
+ val retryPolicy = RetryPolicy(canRetry = _ => true, maxRetries = Some(1), name = "TestPolicy")
+ val retryHandler = new GrpcRetryHandler(retryPolicy, sleep = _ => {})
+
+ assertThrows[StatusRuntimeException] {
+ retryHandler.retry { dummyFn.fn() }
+ }
+ assert(dummyFn.counter == 2)
+ }
+
+ def testPolicySpecificError(maxRetries: Int, status: Status): RetryPolicy = {
+ RetryPolicy(
+ maxRetries = Some(maxRetries),
+ name = s"Policy for ${status.getCode}",
+ canRetry = {
+ case e: StatusRuntimeException => e.getStatus.getCode == status.getCode
+ case _ => false
+ })
+ }
+
+ test("Test multiple policies") {
+ val policy1 = testPolicySpecificError(maxRetries = 2, status = Status.UNAVAILABLE)
+ val policy2 = testPolicySpecificError(maxRetries = 4, status = Status.INTERNAL)
+
+ // Tolerate 2 UNAVAILABLE errors and 4 INTERNAL errors
+
+ val errors = (List.fill(2)(Status.UNAVAILABLE) ++ List.fill(4)(Status.INTERNAL)).iterator
+
+ new GrpcRetryHandler(List(policy1, policy2), sleep = _ => {}).retry({
+ val e = errors.nextOption()
+ if (e.isDefined) {
+ throw e.get.asRuntimeException()
+ }
+ })
+
+ assert(!errors.hasNext)
+ }
+
+ test("Test multiple policies exceed") {
+ val policy1 = testPolicySpecificError(maxRetries = 2, status = Status.INTERNAL)
+ val policy2 = testPolicySpecificError(maxRetries = 4, status = Status.INTERNAL)
+
+ val errors = List.fill(10)(Status.INTERNAL).iterator
+ var countAttempted = 0
+
+ assertThrows[StatusRuntimeException](
+ new GrpcRetryHandler(List(policy1, policy2), sleep = _ => {}).retry({
+ countAttempted += 1
+ val e = errors.nextOption()
+ if (e.isDefined) {
+ throw e.get.asRuntimeException()
+ }
+ }))
+
+ assert(countAttempted == 3)
+ }
+
+ test("DefaultPolicy retries exceptions with RetryInfo") {
+ // Error contains RetryInfo with retry_delay set to 0
+ val dummyFn =
+ new DummyFn(createTestExceptionWithDetails(msg = "Some error message"), numFails = 100)
+ val retryPolicies = RetryPolicy.defaultPolicies()
+ val retryHandler = new GrpcRetryHandler(retryPolicies, sleep = _ => {})
+ assertThrows[StatusRuntimeException] {
+ retryHandler.retry { dummyFn.fn() }
+ }
+
+ // Should be retried by DefaultPolicy
+ val policy = retryPolicies.find(_.name == "DefaultPolicy").get
+ assert(dummyFn.counter == policy.maxRetries.get + 1)
+ }
+
+ test("retry_delay overrides maxBackoff") {
+ val st = new SleepTimeTracker()
+ val retryDelay = FiniteDuration(5, "min")
+ val dummyFn = new DummyFn(
+ createTestExceptionWithDetails(msg = "Some error message", retryDelay = retryDelay),
+ numFails = 100)
+ val retryPolicies = RetryPolicy.defaultPolicies()
+ val retryHandler = new GrpcRetryHandler(retryPolicies, sleep = st.sleep)
+
+ assertThrows[StatusRuntimeException] {
+ retryHandler.retry { dummyFn.fn() }
+ }
+
+ // Should be retried by DefaultPolicy
+ val policy = retryPolicies.find(_.name == "DefaultPolicy").get
+ // sleep times are higher than maxBackoff and are equal to retryDelay + jitter
+ st.times.foreach(t => assert(t > policy.maxBackoff.get.toMillis + policy.jitter.toMillis))
+ val expectedSleeps = List.fill(policy.maxRetries.get)(retryDelay.toMillis)
+ assertLongSequencesAlmostEqual(st.times, expectedSleeps, policy.jitter.toMillis)
+ }
+
+ test("maxServerRetryDelay limits retry_delay") {
+ val st = new SleepTimeTracker()
+ val retryDelay = FiniteDuration(5, "d")
+ val dummyFn = new DummyFn(
+ createTestExceptionWithDetails(msg = "Some error message", retryDelay = retryDelay),
+ numFails = 100)
+ val retryPolicies = RetryPolicy.defaultPolicies()
+ val retryHandler = new GrpcRetryHandler(retryPolicies, sleep = st.sleep)
+
+ assertThrows[StatusRuntimeException] {
+ retryHandler.retry { dummyFn.fn() }
+ }
+
+ // Should be retried by DefaultPolicy
+ val policy = retryPolicies.find(_.name == "DefaultPolicy").get
+ val expectedSleeps = List.fill(policy.maxRetries.get)(policy.maxServerRetryDelay.get.toMillis)
+ assertLongSequencesAlmostEqual(st.times, expectedSleeps, policy.jitter.toMillis)
+ }
+
+ test("Policy uses to exponential backoff after retry_delay is unset") {
+ val st = new SleepTimeTracker()
+ val retryDelay = FiniteDuration(5, "min")
+ val retryPolicies = RetryPolicy.defaultPolicies()
+ val retryHandler = new GrpcRetryHandler(retryPolicies, sleep = st.sleep)
+ val errors = (
+ List.fill(2)(
+ createTestExceptionWithDetails(
+ msg = "Some error message",
+ retryDelay = retryDelay)) ++ List.fill(3)(
+ createTestExceptionWithDetails(
+ msg = "Some error message",
+ code = Status.Code.UNAVAILABLE))
+ ).iterator
+
+ retryHandler.retry({
+ if (errors.hasNext) {
+ throw errors.next()
+ }
+ })
+ assert(!errors.hasNext)
+
+ // Should be retried by DefaultPolicy
+ val policy = retryPolicies.find(_.name == "DefaultPolicy").get
+ val expectedSleeps = List.fill(2)(retryDelay.toMillis) ++ List.tabulate(3)(i =>
+ policy.initialBackoff.toMillis * math.pow(policy.backoffMultiplier, i + 2).toLong)
+ assertLongSequencesAlmostEqual(st.times, expectedSleeps, delta = policy.jitter.toMillis)
+ }
+}
diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala
index 9bb8f5889d330..a41ea344cbd4c 100644
--- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala
+++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala
@@ -339,130 +339,6 @@ class SparkConnectClientSuite extends ConnectFunSuite with BeforeAndAfterEach {
}
}
- private class DummyFn(e: => Throwable, numFails: Int = 3) {
- var counter = 0
- def fn(): Int = {
- if (counter < numFails) {
- counter += 1
- throw e
- } else {
- 42
- }
- }
- }
-
- test("SPARK-44721: Retries run for a minimum period") {
- // repeat test few times to avoid random flakes
- for (_ <- 1 to 10) {
- var totalSleepMs: Long = 0
-
- def sleep(t: Long): Unit = {
- totalSleepMs += t
- }
-
- val dummyFn = new DummyFn(new StatusRuntimeException(Status.UNAVAILABLE), numFails = 100)
- val retryHandler = new GrpcRetryHandler(RetryPolicy.defaultPolicies(), sleep)
-
- assertThrows[RetriesExceeded] {
- retryHandler.retry {
- dummyFn.fn()
- }
- }
-
- assert(totalSleepMs >= 10 * 60 * 1000) // waited at least 10 minutes
- }
- }
-
- test("SPARK-44275: retry actually retries") {
- val dummyFn = new DummyFn(new StatusRuntimeException(Status.UNAVAILABLE))
- val retryPolicies = RetryPolicy.defaultPolicies()
- val retryHandler = new GrpcRetryHandler(retryPolicies, sleep = _ => {})
- val result = retryHandler.retry { dummyFn.fn() }
-
- assert(result == 42)
- assert(dummyFn.counter == 3)
- }
-
- test("SPARK-44275: default retryException retries only on UNAVAILABLE") {
- val dummyFn = new DummyFn(new StatusRuntimeException(Status.ABORTED))
- val retryPolicies = RetryPolicy.defaultPolicies()
- val retryHandler = new GrpcRetryHandler(retryPolicies, sleep = _ => {})
-
- assertThrows[StatusRuntimeException] {
- retryHandler.retry { dummyFn.fn() }
- }
- assert(dummyFn.counter == 1)
- }
-
- test("SPARK-44275: retry uses canRetry to filter exceptions") {
- val dummyFn = new DummyFn(new StatusRuntimeException(Status.UNAVAILABLE))
- val retryPolicy = RetryPolicy(canRetry = _ => false, name = "TestPolicy")
- val retryHandler = new GrpcRetryHandler(retryPolicy)
-
- assertThrows[StatusRuntimeException] {
- retryHandler.retry { dummyFn.fn() }
- }
- assert(dummyFn.counter == 1)
- }
-
- test("SPARK-44275: retry does not exceed maxRetries") {
- val dummyFn = new DummyFn(new StatusRuntimeException(Status.UNAVAILABLE))
- val retryPolicy = RetryPolicy(canRetry = _ => true, maxRetries = Some(1), name = "TestPolicy")
- val retryHandler = new GrpcRetryHandler(retryPolicy, sleep = _ => {})
-
- assertThrows[RetriesExceeded] {
- retryHandler.retry { dummyFn.fn() }
- }
- assert(dummyFn.counter == 2)
- }
-
- def testPolicySpecificError(maxRetries: Int, status: Status): RetryPolicy = {
- RetryPolicy(
- maxRetries = Some(maxRetries),
- name = s"Policy for ${status.getCode}",
- canRetry = {
- case e: StatusRuntimeException => e.getStatus.getCode == status.getCode
- case _ => false
- })
- }
-
- test("Test multiple policies") {
- val policy1 = testPolicySpecificError(maxRetries = 2, status = Status.UNAVAILABLE)
- val policy2 = testPolicySpecificError(maxRetries = 4, status = Status.INTERNAL)
-
- // Tolerate 2 UNAVAILABLE errors and 4 INTERNAL errors
-
- val errors = (List.fill(2)(Status.UNAVAILABLE) ++ List.fill(4)(Status.INTERNAL)).iterator
-
- new GrpcRetryHandler(List(policy1, policy2), sleep = _ => {}).retry({
- val e = errors.nextOption()
- if (e.isDefined) {
- throw e.get.asRuntimeException()
- }
- })
-
- assert(!errors.hasNext)
- }
-
- test("Test multiple policies exceed") {
- val policy1 = testPolicySpecificError(maxRetries = 2, status = Status.INTERNAL)
- val policy2 = testPolicySpecificError(maxRetries = 4, status = Status.INTERNAL)
-
- val errors = List.fill(10)(Status.INTERNAL).iterator
- var countAttempted = 0
-
- assertThrows[RetriesExceeded](
- new GrpcRetryHandler(List(policy1, policy2), sleep = _ => {}).retry({
- countAttempted += 1
- val e = errors.nextOption()
- if (e.isDefined) {
- throw e.get.asRuntimeException()
- }
- }))
-
- assert(countAttempted == 7)
- }
-
test("ArtifactManager retries errors") {
var attempt = 0
diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderSuite.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderSuite.scala
index 75816a835aaa7..b29d73be359b5 100644
--- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderSuite.scala
+++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderSuite.scala
@@ -36,7 +36,7 @@ import org.apache.spark.sql.catalyst.{DefinedByConstructorParams, JavaTypeInfere
import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, Codec, OuterScopes}
import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{agnosticEncoderFor, BinaryEncoder, BoxedBooleanEncoder, BoxedByteEncoder, BoxedDoubleEncoder, BoxedFloatEncoder, BoxedIntEncoder, BoxedLongEncoder, BoxedShortEncoder, CalendarIntervalEncoder, DateEncoder, DayTimeIntervalEncoder, EncoderField, InstantEncoder, IterableEncoder, JavaDecimalEncoder, LocalDateEncoder, LocalDateTimeEncoder, NullEncoder, PrimitiveBooleanEncoder, PrimitiveByteEncoder, PrimitiveDoubleEncoder, PrimitiveFloatEncoder, PrimitiveIntEncoder, PrimitiveLongEncoder, PrimitiveShortEncoder, RowEncoder, ScalaDecimalEncoder, StringEncoder, TimestampEncoder, TransformingEncoder, UDTEncoder, YearMonthIntervalEncoder}
import org.apache.spark.sql.catalyst.encoders.RowEncoder.{encoderFor => toRowEncoder}
-import org.apache.spark.sql.catalyst.util.{DateFormatter, SparkStringUtils, TimestampFormatter}
+import org.apache.spark.sql.catalyst.util.{DateFormatter, TimestampFormatter}
import org.apache.spark.sql.catalyst.util.DateTimeConstants.MICROS_PER_SECOND
import org.apache.spark.sql.catalyst.util.IntervalStringStyles.ANSI_STYLE
import org.apache.spark.sql.catalyst.util.SparkDateTimeUtils._
@@ -46,6 +46,7 @@ import org.apache.spark.sql.connect.client.arrow.FooEnum.FooEnum
import org.apache.spark.sql.connect.test.ConnectFunSuite
import org.apache.spark.sql.types.{ArrayType, DataType, DayTimeIntervalType, Decimal, DecimalType, IntegerType, Metadata, SQLUserDefinedType, StringType, StructType, UserDefinedType, YearMonthIntervalType}
import org.apache.spark.unsafe.types.VariantVal
+import org.apache.spark.util.SparkStringUtils
/**
* Tests for encoding external data to and from arrow.
diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/streaming/TransformWithStateConnectSuite.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/streaming/TransformWithStateConnectSuite.scala
index 310b50dac1cc3..359486ae8e652 100644
--- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/streaming/TransformWithStateConnectSuite.scala
+++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/streaming/TransformWithStateConnectSuite.scala
@@ -26,17 +26,22 @@ import org.scalatest.concurrent.Eventually.eventually
import org.scalatest.concurrent.Futures.timeout
import org.scalatest.time.SpanSugar._
+import org.apache.spark.SparkUnsupportedOperationException
import org.apache.spark.internal.Logging
import org.apache.spark.sql.{DataFrame, Dataset, Encoders, Row}
import org.apache.spark.sql.connect.SparkSession
import org.apache.spark.sql.connect.test.{QueryTest, RemoteSparkSession}
import org.apache.spark.sql.functions._
-import org.apache.spark.sql.streaming.{ListState, MapState, OutputMode, StatefulProcessor, StatefulProcessorWithInitialState, TimeMode, TimerValues, TTLConfig, ValueState}
+import org.apache.spark.sql.streaming.{ListState, MapState, OutputMode, StatefulProcessor, StatefulProcessorWithInitialState, StreamingQueryException, TimeMode, TimerValues, TTLConfig, ValueState}
import org.apache.spark.sql.types._
+import org.apache.spark.util.SparkFileUtils
case class InputRowForConnectTest(key: String, value: String)
case class OutputRowForConnectTest(key: String, value: String)
+case class StateRowForConnectTestWithIntType(count: Int)
case class StateRowForConnectTest(count: Long)
+case class StateRowForConnectTestWithTwoLongs(count: Long, count2: Long)
+case class StateRowForConnectTestWithReorder(count2: Long, count: Long)
// A basic stateful processor which will return the occurrences of key
class BasicCountStatefulProcessor
@@ -67,6 +72,97 @@ class BasicCountStatefulProcessor
}
}
+// A basic stateful processor which will return the occurrences of key.
+// Count State is a Int type.
+class CountStatefulProcessorWithInt
+ extends StatefulProcessor[String, InputRowForConnectTest, OutputRowForConnectTest]
+ with Logging {
+ @transient protected var _countState: ValueState[StateRowForConnectTestWithIntType] = _
+
+ override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {
+ _countState = getHandle.getValueState[StateRowForConnectTestWithIntType](
+ "countState",
+ Encoders.product[StateRowForConnectTestWithIntType],
+ TTLConfig.NONE)
+ }
+
+ override def handleInputRows(
+ key: String,
+ inputRows: Iterator[InputRowForConnectTest],
+ timerValues: TimerValues): Iterator[OutputRowForConnectTest] = {
+ val count = inputRows.toSeq.length + {
+ if (_countState.exists()) {
+ _countState.get().count
+ } else {
+ 0
+ }
+ }
+ _countState.update(StateRowForConnectTestWithIntType(count))
+ Iterator(OutputRowForConnectTest(key, count.toString))
+ }
+}
+
+// A stateful processor with Two Longs as state
+// which will return the occurrences of key to test TWS schema evolution
+class CountStatefulProcessorTwoLongs
+ extends StatefulProcessor[String, InputRowForConnectTest, OutputRowForConnectTest]
+ with Logging {
+ @transient protected var _countState: ValueState[StateRowForConnectTestWithTwoLongs] = _
+
+ override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {
+ _countState = getHandle.getValueState[StateRowForConnectTestWithTwoLongs](
+ "countState",
+ Encoders.product[StateRowForConnectTestWithTwoLongs],
+ TTLConfig.NONE)
+ }
+
+ override def handleInputRows(
+ key: String,
+ inputRows: Iterator[InputRowForConnectTest],
+ timerValues: TimerValues): Iterator[OutputRowForConnectTest] = {
+ val count = inputRows.toSeq.length + {
+ if (_countState.exists()) {
+ _countState.get().count
+ } else {
+ 0L
+ }
+ }
+ _countState.update(StateRowForConnectTestWithTwoLongs(count, count))
+ Iterator(OutputRowForConnectTest(key, count.toString))
+ }
+}
+
+// A stateful processor with Two Longs as state.
+// Reorder the field Sequence inside StateRowForConnectTestWithTwoLongs.
+// which will return the occurrences of key to test TWS schema evolution
+class CountStatefulProcessorWithReorder
+ extends StatefulProcessor[String, InputRowForConnectTest, OutputRowForConnectTest]
+ with Logging {
+ @transient protected var _countState: ValueState[StateRowForConnectTestWithReorder] = _
+
+ override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {
+ _countState = getHandle.getValueState[StateRowForConnectTestWithReorder](
+ "countState",
+ Encoders.product[StateRowForConnectTestWithReorder],
+ TTLConfig.NONE)
+ }
+
+ override def handleInputRows(
+ key: String,
+ inputRows: Iterator[InputRowForConnectTest],
+ timerValues: TimerValues): Iterator[OutputRowForConnectTest] = {
+ val count = inputRows.toSeq.length + {
+ if (_countState.exists()) {
+ _countState.get().count
+ } else {
+ 0L
+ }
+ }
+ _countState.update(StateRowForConnectTestWithReorder(count, count))
+ Iterator(OutputRowForConnectTest(key, count.toString))
+ }
+}
+
// A stateful processor with initial state which will return the occurrences of key
class TestInitialStatefulProcessor
extends StatefulProcessorWithInitialState[
@@ -488,13 +584,140 @@ class TransformWithStateConnectSuite
}
}
+ private def runSchemaEvolutionTest(
+ firstProcessor: StatefulProcessor[String, InputRowForConnectTest, OutputRowForConnectTest],
+ secondProcessor: StatefulProcessor[String, InputRowForConnectTest, OutputRowForConnectTest])
+ : Unit = {
+ withSQLConf(
+ (twsAdditionalSQLConf ++
+ Seq("spark.sql.streaming.stateStore.encodingFormat" -> "avro")): _*) {
+ val session: SparkSession = spark
+ import session.implicits._
+
+ withTempPath { dir =>
+ val path = dir.getCanonicalPath
+ val checkpointPath = s"$path/cpt"
+ val dataPath = s"$path/data"
+ val targetPath = s"$path/tgt"
+
+ testData
+ .toDS()
+ .toDF("key", "value")
+ .repartition(3)
+ .write
+ .mode("append")
+ .parquet(dataPath)
+
+ val testSchema =
+ StructType(Array(StructField("key", StringType), StructField("value", StringType)))
+
+ val q1 = spark.readStream
+ .schema(testSchema)
+ .option("maxFilesPerTrigger", 1)
+ .parquet(dataPath)
+ .as[InputRowForConnectTest]
+ .groupByKey(x => x.key)
+ .transformWithState[OutputRowForConnectTest](
+ firstProcessor,
+ TimeMode.None(),
+ OutputMode.Update())
+ .writeStream
+ .format("parquet")
+ .option("checkpointLocation", checkpointPath)
+ .option("path", targetPath)
+ .start()
+
+ try {
+ q1.processAllAvailable()
+ eventually(timeout(30.seconds)) {
+ checkDatasetUnorderly(
+ spark.read.format("parquet").load(targetPath).as[(String, String)],
+ ("a", "1"),
+ ("a", "2"),
+ ("b", "1"))
+ }
+ } finally {
+ q1.stop()
+ }
+
+ testData
+ .toDS()
+ .toDF("key", "value")
+ .repartition(3)
+ .write
+ .mode("append")
+ .parquet(dataPath)
+
+ val q2 = spark.readStream
+ .schema(testSchema)
+ .option("maxFilesPerTrigger", 1)
+ .parquet(dataPath)
+ .as[InputRowForConnectTest]
+ .groupByKey(x => x.key)
+ .transformWithState[OutputRowForConnectTest](
+ secondProcessor,
+ TimeMode.None(),
+ OutputMode.Update())
+ .writeStream
+ .format("parquet")
+ .option("checkpointLocation", checkpointPath)
+ .option("path", targetPath)
+ .start()
+
+ try {
+ q2.processAllAvailable()
+ eventually(timeout(30.seconds)) {
+ checkDatasetUnorderly(
+ spark.read.format("parquet").load(targetPath).as[(String, String)],
+ ("a", "1"),
+ ("a", "2"),
+ ("b", "1"),
+ ("a", "3"),
+ ("a", "4"),
+ ("b", "2"))
+ }
+ } finally {
+ q2.stop()
+ }
+ }
+ }
+ }
+
+ test("transformWithState - add fields schema evolution") {
+ runSchemaEvolutionTest(new BasicCountStatefulProcessor, new CountStatefulProcessorTwoLongs)
+ }
+
+ test("transformWithState - remove fields schema evolution") {
+ runSchemaEvolutionTest(new CountStatefulProcessorTwoLongs, new BasicCountStatefulProcessor)
+ }
+
+ test("transformWithState - reorder fields schema evolution") {
+ runSchemaEvolutionTest(
+ new CountStatefulProcessorTwoLongs,
+ new CountStatefulProcessorWithReorder)
+ }
+
+ test("transformWithState - upcast fields schema evolution") {
+ runSchemaEvolutionTest(new CountStatefulProcessorWithInt, new BasicCountStatefulProcessor)
+ }
+
+ test("transformWithState - downcast fields would fail") {
+ val e = intercept[StreamingQueryException] {
+ runSchemaEvolutionTest(new BasicCountStatefulProcessor, new CountStatefulProcessorWithInt)
+ }
+ assert(
+ e.getCause
+ .asInstanceOf[SparkUnsupportedOperationException]
+ .getCondition == "STATE_STORE_INVALID_VALUE_SCHEMA_EVOLUTION")
+ }
+
/* Utils functions for tests */
def prepareInputData(inputPath: String, col1: Seq[String], col2: Seq[Int]): File = {
// Ensure the parent directory exists
val file = Paths.get(inputPath).toFile
val parentDir = file.getParentFile
if (parentDir != null && !parentDir.exists()) {
- parentDir.mkdirs()
+ SparkFileUtils.createDirectory(parentDir)
}
val writer = new BufferedWriter(new FileWriter(inputPath))
diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/test/QueryTest.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/test/QueryTest.scala
index 5ae23368b9729..da3b32b408f58 100644
--- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/test/QueryTest.scala
+++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/test/QueryTest.scala
@@ -25,9 +25,9 @@ import org.scalatest.Assertions
import org.apache.spark.{QueryContextType, SparkThrowable}
import org.apache.spark.sql.Row
-import org.apache.spark.sql.catalyst.util.SparkStringUtils.sideBySide
import org.apache.spark.sql.connect.{DataFrame, Dataset, SparkSession}
import org.apache.spark.util.ArrayImplicits._
+import org.apache.spark.util.SparkStringUtils.sideBySide
abstract class QueryTest extends ConnectFunSuite with SQLHelper {
diff --git a/sql/connect/common/pom.xml b/sql/connect/common/pom.xml
index 18fb06ff3341d..6ff47ec6d68ce 100644
--- a/sql/connect/common/pom.xml
+++ b/sql/connect/common/pom.xml
@@ -105,13 +105,6 @@
-
-
- kr.motd.maven
- os-maven-plugin
- 1.6.2
-
- target/scala-${scala.binary.version}/classestarget/scala-${scala.binary.version}/test-classes
diff --git a/sql/connect/common/src/main/protobuf/spark/connect/expressions.proto b/sql/connect/common/src/main/protobuf/spark/connect/expressions.proto
index df907a84868fe..913622b91a284 100644
--- a/sql/connect/common/src/main/protobuf/spark/connect/expressions.proto
+++ b/sql/connect/common/src/main/protobuf/spark/connect/expressions.proto
@@ -195,6 +195,7 @@ message Expression {
Struct struct = 24;
SpecializedArray specialized_array = 25;
+ Time time = 26;
}
message Decimal {
@@ -214,20 +215,66 @@ message Expression {
}
message Array {
- DataType element_type = 1;
+ // (Deprecated) The element type of the array.
+ //
+ // This field is deprecated since Spark 4.1+ and should only be set
+ // if the data_type field is not set. Use data_type field instead.
+ DataType element_type = 1 [deprecated = true];
+
+ // The literal values that make up the array elements.
repeated Literal elements = 2;
+
+ // The type of the array.
+ //
+ // If the element type can be inferred from the first element of the elements field,
+ // then you don't need to set data_type.element_type to save space. On the other hand,
+ // redundant type information is also acceptable.
+ DataType.Array data_type = 3;
}
message Map {
- DataType key_type = 1;
- DataType value_type = 2;
+ // (Deprecated) The key type of the map.
+ //
+ // This field is deprecated since Spark 4.1+ and should only be set
+ // if the data_type field is not set. Use data_type field instead.
+ DataType key_type = 1 [deprecated = true];
+
+ // (Deprecated) The value type of the map.
+ //
+ // This field is deprecated since Spark 4.1+ and should only be set
+ // if the data_type field is not set. Use data_type field instead.
+ DataType value_type = 2 [deprecated = true];
+
+ // The literal keys that make up the map.
repeated Literal keys = 3;
+
+ // The literal values that make up the map.
repeated Literal values = 4;
+
+ // The type of the map.
+ //
+ // If the key/value types can be inferred from the first element of the keys/values fields,
+ // then you don't need to set data_type.key_type/data_type.value_type to save space.
+ // On the other hand, redundant type information is also acceptable.
+ DataType.Map data_type = 5;
}
message Struct {
- DataType struct_type = 1;
+ // (Deprecated) The type of the struct.
+ //
+ // This field is deprecated since Spark 4.1+ because using DataType as the type of a struct
+ // is ambiguous. This field should only be set if the data_type_struct field is not set.
+ // Use data_type_struct field instead.
+ DataType struct_type = 1 [deprecated = true];
+
+ // (Required) The literal values that make up the struct elements.
repeated Literal elements = 2;
+
+ // The type of the struct.
+ //
+ // Whether data_type_struct.fields.data_type should be set depends on
+ // whether each field's type can be inferred from the elements field.
+ DataType.Struct data_type_struct = 3;
}
message SpecializedArray {
@@ -240,6 +287,12 @@ message Expression {
Strings strings = 6;
}
}
+
+ message Time {
+ int64 nano = 1;
+ // The precision of this time, if omitted, uses the default value of MICROS_PRECISION.
+ optional int32 precision = 2;
+ }
}
// An unresolved attribute that is not explicitly bound to a specific column, but the column
diff --git a/sql/connect/common/src/main/protobuf/spark/connect/ml.proto b/sql/connect/common/src/main/protobuf/spark/connect/ml.proto
index b66c0a186df39..ef5c406dedd26 100644
--- a/sql/connect/common/src/main/protobuf/spark/connect/ml.proto
+++ b/sql/connect/common/src/main/protobuf/spark/connect/ml.proto
@@ -38,6 +38,8 @@ message MlCommand {
Evaluate evaluate = 6;
CleanCache clean_cache = 7;
GetCacheInfo get_cache_info = 8;
+ CreateSummary create_summary = 9;
+ GetModelSize get_model_size = 10;
}
// Command for estimator.fit(dataset)
@@ -54,6 +56,9 @@ message MlCommand {
// or summary evaluated by a model
message Delete {
repeated ObjectRef obj_refs = 1;
+ // if set `evict_only` to true, only evict the cached model from memory,
+ // but keep the offloaded model in Spark driver local disk.
+ optional bool evict_only = 2;
}
// Force to clean up all the ML cached objects
@@ -98,6 +103,18 @@ message MlCommand {
// (Required) the evaluating dataset
Relation dataset = 3;
}
+
+ // This is for re-creating the model summary when the model summary is lost
+ // (model summary is lost when the model is offloaded and then loaded back)
+ message CreateSummary {
+ ObjectRef model_ref = 1;
+ Relation dataset = 2;
+ }
+
+ // This is for query the model estimated in-memory size
+ message GetModelSize {
+ ObjectRef model_ref = 1;
+ }
}
// The result of MlCommand
diff --git a/sql/connect/common/src/main/protobuf/spark/connect/pipelines.proto b/sql/connect/common/src/main/protobuf/spark/connect/pipelines.proto
index 2f685c6da42c9..57e1ffc7dbe76 100644
--- a/sql/connect/common/src/main/protobuf/spark/connect/pipelines.proto
+++ b/sql/connect/common/src/main/protobuf/spark/connect/pipelines.proto
@@ -25,6 +25,7 @@ import "spark/connect/types.proto";
option java_multiple_files = true;
option java_package = "org.apache.spark.connect.proto";
+option go_package = "internal/generated";
// Dispatch object for pipelines commands. See each individual command for documentation.
message PipelineCommand {
@@ -101,13 +102,10 @@ message PipelineCommand {
optional string target_dataset_name = 3;
// An unresolved relation that defines the dataset's flow.
- optional spark.connect.Relation plan = 4;
+ optional spark.connect.Relation relation = 4;
// SQL configurations set when running this flow.
map sql_conf = 5;
-
- // If true, this flow will only be run once per full refresh.
- optional bool once = 6;
}
// Resolves all datasets and flows and start a pipeline update. Should be called after all
@@ -115,6 +113,19 @@ message PipelineCommand {
message StartRun {
// The graph to start.
optional string dataflow_graph_id = 1;
+
+ // List of dataset to reset and recompute.
+ repeated string full_refresh_selection = 2;
+
+ // Perform a full graph reset and recompute.
+ optional bool full_refresh_all = 3;
+
+ // List of dataset to update.
+ repeated string refresh_selection = 4;
+
+ // If true, the run will not actually execute any flows, but will only validate the graph and
+ // check for any errors. This is useful for testing and validation purposes.
+ optional bool dry = 5;
}
// Parses the SQL file and registers all datasets and flows.
diff --git a/sql/connect/common/src/main/protobuf/spark/connect/relations.proto b/sql/connect/common/src/main/protobuf/spark/connect/relations.proto
index 70a52a2111494..ccb674e812dc0 100644
--- a/sql/connect/common/src/main/protobuf/spark/connect/relations.proto
+++ b/sql/connect/common/src/main/protobuf/spark/connect/relations.proto
@@ -115,6 +115,9 @@ message MlRelation {
Transform transform = 1;
Fetch fetch = 2;
}
+ // (Optional) the dataset for restoring the model summary
+ optional Relation model_summary_dataset = 3;
+
// Relation to represent transform(input) of the operator
// which could be a cached model or a new transformer
message Transform {
diff --git a/sql/connect/common/src/main/protobuf/spark/connect/types.proto b/sql/connect/common/src/main/protobuf/spark/connect/types.proto
index e1a111e5d6917..1800e3885774f 100644
--- a/sql/connect/common/src/main/protobuf/spark/connect/types.proto
+++ b/sql/connect/common/src/main/protobuf/spark/connect/types.proto
@@ -69,8 +69,13 @@ message DataType {
// UnparsedDataType
Unparsed unparsed = 24;
+
+ Time time = 28;
}
+ // Reserved for geometry and geography types
+ reserved 26, 27;
+
message Boolean {
uint32 type_variation_reference = 1;
}
@@ -124,6 +129,11 @@ message DataType {
uint32 type_variation_reference = 1;
}
+ message Time {
+ optional int32 precision = 1;
+ uint32 type_variation_reference = 2;
+ }
+
message CalendarInterval {
uint32 type_variation_reference = 1;
}
diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/DataFrameReader.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/DataFrameReader.scala
index 0af603e0f6cc9..67a4d983f56bf 100644
--- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/DataFrameReader.scala
+++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/DataFrameReader.scala
@@ -79,8 +79,7 @@ class DataFrameReader private[sql] (sparkSession: SparkSession) extends sql.Data
def load(paths: String*): DataFrame = {
sparkSession.newDataFrame { builder =>
val dataSourceBuilder = builder.getReadBuilder.getDataSourceBuilder
- assertSourceFormatSpecified()
- dataSourceBuilder.setFormat(source)
+ Option(source).foreach(dataSourceBuilder.setFormat)
userSpecifiedSchema.foreach(schema => dataSourceBuilder.setSchema(schema.toDDL))
extraOptions.foreach { case (k, v) =>
dataSourceBuilder.putOptions(k, v)
@@ -211,12 +210,6 @@ class DataFrameReader private[sql] (sparkSession: SparkSession) extends sql.Data
@scala.annotation.varargs
override def textFile(paths: String*): Dataset[String] = super.textFile(paths: _*)
- private def assertSourceFormatSpecified(): Unit = {
- if (source == null) {
- throw new IllegalArgumentException("The source format must be specified.")
- }
- }
-
private def parse(ds: Dataset[String], format: ParseFormat): DataFrame = {
sparkSession.newDataFrame { builder =>
val parseBuilder = builder.getParseBuilder
diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/KeyValueGroupedDataset.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/KeyValueGroupedDataset.scala
index af036218455fa..49298a5b39fc6 100644
--- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/KeyValueGroupedDataset.scala
+++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/KeyValueGroupedDataset.scala
@@ -24,6 +24,7 @@ import org.apache.spark.api.java.function._
import org.apache.spark.connect.proto
import org.apache.spark.sql
import org.apache.spark.sql.{Column, Encoder, TypedColumn}
+import org.apache.spark.sql.catalyst.ScalaReflection
import org.apache.spark.sql.catalyst.encoders.AgnosticEncoder
import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{agnosticEncoderFor, ProductEncoder, StructEncoder}
import org.apache.spark.sql.connect.ColumnNodeToProtoConverter.{toExpr, toExprWithTransformation, toTypedExpr}
@@ -658,8 +659,14 @@ private class KeyValueGroupedDatasetImpl[K, V, IK, IV](
initialState: Option[sql.KeyValueGroupedDataset[K, S]] = None,
eventTimeColumnName: String = ""): Dataset[U] = {
val outputEncoder = agnosticEncoderFor[U]
- val stateEncoder = agnosticEncoderFor[S]
- val inputEncoders: Seq[AgnosticEncoder[_]] = Seq(kEncoder, stateEncoder, ivEncoder)
+ val initialStateEncoder = if (initialState.isDefined) {
+ agnosticEncoderFor[S]
+ } else {
+ // Cannot use `agnosticEncoderFor[S]` here because it points to incorrect encoder
+ // when the initial state is not provided. Using an empty state encoder instead.
+ ScalaReflection.encoderFor[EmptyInitialStateStruct]
+ }
+ val inputEncoders: Seq[AgnosticEncoder[_]] = Seq(kEncoder, initialStateEncoder, ivEncoder)
// SparkUserDefinedFunction is creating a udfPacket where the input function are
// being java serialized into bytes; we pass in `statefulProcessor` as function so it can be
@@ -780,3 +787,14 @@ private object KeyValueGroupedDatasetImpl {
case _ => false
}
}
+
+/**
+ * A marker case class used as a placeholder type for initial state encoders when no actual
+ * initial state is provided to stateful streaming operations.
+ *
+ * In the `transformWithStateHelper` method, when `initialState` is not provided, we cannot use
+ * `agnosticEncoderFor[S]` for the initial state encoder because it would incorrectly point to the
+ * other encoders. Instead, we use `EmptyStruct` as a sentinel type to create a proper encoder
+ * that represents the absence of initial state data.
+ */
+case class EmptyInitialStateStruct()
diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/SparkSession.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/SparkSession.scala
index 739b0318759e5..646db83981fe2 100644
--- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/SparkSession.scala
+++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/SparkSession.scala
@@ -40,7 +40,7 @@ import org.apache.spark.api.java.JavaRDD
import org.apache.spark.connect.proto
import org.apache.spark.connect.proto.ExecutePlanResponse
import org.apache.spark.connect.proto.ExecutePlanResponse.ObservedMetrics
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{CONFIG, PATH}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql
diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/StreamingQueryListenerBus.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/StreamingQueryListenerBus.scala
index 30cdf2b5cadb4..52b0ea24e9e33 100644
--- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/StreamingQueryListenerBus.scala
+++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/StreamingQueryListenerBus.scala
@@ -22,7 +22,7 @@ import java.util.concurrent.CopyOnWriteArrayList
import scala.jdk.CollectionConverters._
import org.apache.spark.connect.proto.{Command, ExecutePlanResponse, Plan, StreamingQueryEventType}
-import org.apache.spark.internal.{Logging, LogKeys, MDC}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.sql.connect.client.CloseableIterator
import org.apache.spark.sql.streaming.StreamingQueryListener
import org.apache.spark.sql.streaming.StreamingQueryListener.{Event, QueryIdleEvent, QueryProgressEvent, QueryStartedEvent, QueryTerminatedEvent}
diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ArtifactManager.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ArtifactManager.scala
index e9411dc3db61b..213cd1d2e8673 100644
--- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ArtifactManager.scala
+++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ArtifactManager.scala
@@ -33,7 +33,6 @@ import com.google.protobuf.ByteString
import io.grpc.StatusRuntimeException
import io.grpc.stub.StreamObserver
import org.apache.commons.codec.digest.DigestUtils.sha256Hex
-import org.apache.commons.lang3.StringUtils
import org.apache.spark.SparkException
import org.apache.spark.connect.proto
@@ -41,7 +40,7 @@ import org.apache.spark.connect.proto.AddArtifactsResponse
import org.apache.spark.connect.proto.AddArtifactsResponse.ArtifactSummary
import org.apache.spark.sql.Artifact
import org.apache.spark.sql.Artifact.{newCacheArtifact, newIvyArtifacts}
-import org.apache.spark.util.{SparkFileUtils, SparkThreadUtils}
+import org.apache.spark.util.{SparkFileUtils, SparkStringUtils, SparkThreadUtils}
/**
* The Artifact Manager is responsible for handling and transferring artifacts from the local
@@ -173,7 +172,8 @@ class ArtifactManager(
.addAllNames(Arrays.asList(artifactName))
.build()
val response = bstub.artifactStatus(request)
- if (StringUtils.isNotEmpty(response.getSessionId) && response.getSessionId != sessionId) {
+ if (SparkStringUtils.isNotEmpty(response.getSessionId) &&
+ response.getSessionId != sessionId) {
// In older versions of the Spark cluster, the session ID is not set in the response.
// Ignore this check to keep compatibility.
throw new IllegalStateException(
@@ -248,7 +248,7 @@ class ArtifactManager(
val responseHandler = new StreamObserver[proto.AddArtifactsResponse] {
private val summaries = mutable.Buffer.empty[ArtifactSummary]
override def onNext(v: AddArtifactsResponse): Unit = {
- if (StringUtils.isNotEmpty(v.getSessionId) && v.getSessionId != sessionId) {
+ if (SparkStringUtils.isNotEmpty(v.getSessionId) && v.getSessionId != sessionId) {
// In older versions of the Spark cluster, the session ID is not set in the response.
// Ignore this check to keep compatibility.
throw new IllegalStateException(s"Session ID mismatch: $sessionId != ${v.getSessionId}")
diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcRetryHandler.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcRetryHandler.scala
index 7e0a356b9e493..3f4558ee97dad 100644
--- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcRetryHandler.scala
+++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcRetryHandler.scala
@@ -23,7 +23,6 @@ import io.grpc.stub.StreamObserver
import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{ERROR, NUM_RETRY, POLICY, RETRY_WAIT_TIME}
-import org.apache.spark.internal.MDC
private[sql] class GrpcRetryHandler(
private val policies: Seq[RetryPolicy],
@@ -194,15 +193,17 @@ private[sql] object GrpcRetryHandler extends Logging {
return
}
- for (policy <- policies if policy.canRetry(lastException)) {
- val time = policy.nextAttempt()
-
+ // find a policy to wait with
+ val matchedPolicyOpt = policies.find(_.canRetry(lastException))
+ if (matchedPolicyOpt.isDefined) {
+ val matchedPolicy = matchedPolicyOpt.get
+ val time = matchedPolicy.nextAttempt(lastException)
if (time.isDefined) {
logWarning(
log"Non-Fatal error during RPC execution: ${MDC(ERROR, lastException)}, " +
log"retrying (wait=${MDC(RETRY_WAIT_TIME, time.get.toMillis)} ms, " +
log"currentRetryNum=${MDC(NUM_RETRY, currentRetryNum)}, " +
- log"policy=${MDC(POLICY, policy.getName)}).")
+ log"policy=${MDC(POLICY, matchedPolicy.getName)}).")
sleep(time.get.toMillis)
return
}
@@ -212,9 +213,8 @@ private[sql] object GrpcRetryHandler extends Logging {
log"Non-Fatal error during RPC execution: ${MDC(ERROR, lastException)}, " +
log"exceeded retries (currentRetryNum=${MDC(NUM_RETRY, currentRetryNum)})")
- val error = new RetriesExceeded()
- exceptionList.foreach(error.addSuppressed)
- throw error
+ logWarning(log"[RETRIES_EXCEEDED] The maximum number of retries has been exceeded.")
+ throw lastException
}
def retry(): T = {
diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/RetryPolicy.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/RetryPolicy.scala
index 8c8472d780dbc..5b5c4b517923e 100644
--- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/RetryPolicy.scala
+++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/RetryPolicy.scala
@@ -18,9 +18,14 @@
package org.apache.spark.sql.connect.client
import scala.concurrent.duration.{Duration, FiniteDuration}
+import scala.jdk.CollectionConverters._
import scala.util.Random
+import com.google.rpc.RetryInfo
import io.grpc.{Status, StatusRuntimeException}
+import io.grpc.protobuf.StatusProto
+
+import org.apache.spark.internal.Logging
/**
* [[RetryPolicy]] configure the retry mechanism in [[GrpcRetryHandler]]
@@ -33,8 +38,27 @@ import io.grpc.{Status, StatusRuntimeException}
* Maximal value of the exponential backoff (ms).
* @param backoffMultiplier
* Multiplicative base of the exponential backoff.
+ * @param jitter
+ * Sample a random value uniformly from the range [0, jitter] and add it to the backoff.
+ * @param minJitterThreshold
+ * Minimal value of the backoff to add random jitter.
* @param canRetry
* Function that determines whether a retry is to be performed in the event of an error.
+ * @param name
+ * Name of the policy.
+ * @param recognizeServerRetryDelay
+ * Per gRPC standard, the server can send error messages that contain `RetryInfo` message with
+ * `retry_delay` field indicating that the client should wait for at least `retry_delay` amount
+ * of time before retrying again, see:
+ * https://github.com/googleapis/googleapis/blob/master/google/rpc/error_details.proto#L91
+ *
+ * If this flag is set to true, RetryPolicy will use `RetryInfo.retry_delay` field in the backoff
+ * computation. Server's `retry_delay` can override client's `maxBackoff`.
+ *
+ * This flag does not change which errors are retried, only how the backoff is computed.
+ * `DefaultPolicy` additionally has a rule for retrying any error that contains `RetryInfo`.
+ * @param maxServerRetryDelay
+ * Limit for the server-provided `retry_delay`.
*/
case class RetryPolicy(
maxRetries: Option[Int] = None,
@@ -44,14 +68,16 @@ case class RetryPolicy(
jitter: FiniteDuration = FiniteDuration(0, "s"),
minJitterThreshold: FiniteDuration = FiniteDuration(0, "s"),
canRetry: Throwable => Boolean,
- name: String) {
+ name: String,
+ recognizeServerRetryDelay: Boolean = false,
+ maxServerRetryDelay: Option[FiniteDuration] = None) {
def getName: String = name
def toState: RetryPolicy.RetryPolicyState = new RetryPolicy.RetryPolicyState(this)
}
-object RetryPolicy {
+object RetryPolicy extends Logging {
def defaultPolicy(): RetryPolicy = RetryPolicy(
name = "DefaultPolicy",
// Please synchronize changes here with Python side:
@@ -65,7 +91,9 @@ object RetryPolicy {
backoffMultiplier = 4.0,
jitter = FiniteDuration(500, "ms"),
minJitterThreshold = FiniteDuration(2, "s"),
- canRetry = defaultPolicyRetryException)
+ canRetry = defaultPolicyRetryException,
+ recognizeServerRetryDelay = true,
+ maxServerRetryDelay = Some(FiniteDuration(10, "min")))
// list of policies to be used by this client
def defaultPolicies(): Seq[RetryPolicy] = List(defaultPolicy())
@@ -77,7 +105,7 @@ object RetryPolicy {
private var nextWait: Duration = policy.initialBackoff
// return waiting time until next attempt, or None if has exceeded max retries
- def nextAttempt(): Option[Duration] = {
+ def nextAttempt(e: Throwable): Option[Duration] = {
if (policy.maxRetries.isDefined && numberAttempts >= policy.maxRetries.get) {
return None
}
@@ -90,6 +118,14 @@ object RetryPolicy {
nextWait = nextWait min policy.maxBackoff.get
}
+ if (policy.recognizeServerRetryDelay) {
+ extractRetryDelay(e).foreach { retryDelay =>
+ logDebug(s"The server has sent a retry delay of $retryDelay ms.")
+ val retryDelayLimited = retryDelay min policy.maxServerRetryDelay.getOrElse(retryDelay)
+ currentWait = currentWait max retryDelayLimited
+ }
+ }
+
if (currentWait >= policy.minJitterThreshold) {
currentWait += Random.nextDouble() * policy.jitter
}
@@ -127,8 +163,33 @@ object RetryPolicy {
if (statusCode == Status.Code.UNAVAILABLE) {
return true
}
+
+ // All errors messages containing `RetryInfo` should be retried.
+ if (extractRetryInfo(e).isDefined) {
+ return true
+ }
+
false
case _ => false
}
}
+
+ private def extractRetryInfo(e: Throwable): Option[RetryInfo] = {
+ e match {
+ case e: StatusRuntimeException =>
+ Option(StatusProto.fromThrowable(e))
+ .flatMap(status =>
+ status.getDetailsList.asScala
+ .find(_.is(classOf[RetryInfo]))
+ .map(_.unpack(classOf[RetryInfo])))
+ case _ => None
+ }
+ }
+
+ private def extractRetryDelay(e: Throwable): Option[FiniteDuration] = {
+ extractRetryInfo(e)
+ .flatMap(retryInfo => Option(retryInfo.getRetryDelay))
+ .map(retryDelay =>
+ FiniteDuration(retryDelay.getSeconds, "s") + FiniteDuration(retryDelay.getNanos, "ns"))
+ }
}
diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala
index e844237a3bb44..48f01a8042a64 100644
--- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala
+++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala
@@ -33,6 +33,7 @@ import org.apache.spark.connect.proto
import org.apache.spark.connect.proto.UserContext
import org.apache.spark.sql.connect.common.ProtoUtils
import org.apache.spark.sql.connect.common.config.ConnectCommon
+import org.apache.spark.util.SparkSystemUtils
/**
* Conceptually the remote spark session that communicates with the server.
@@ -707,12 +708,11 @@ object SparkConnectClient {
*/
private def genUserAgent(value: String): String = {
val scalaVersion = Properties.versionNumberString
- val jvmVersion = System.getProperty("java.version").split("_")(0)
+ val jvmVersion = SparkSystemUtils.javaVersion.split("_")(0)
val osName = {
- val os = System.getProperty("os.name").toLowerCase(Locale.ROOT)
- if (os.contains("mac")) "darwin"
- else if (os.contains("linux")) "linux"
- else if (os.contains("win")) "windows"
+ if (SparkSystemUtils.isMac) "darwin"
+ else if (SparkSystemUtils.isLinux) "linux"
+ else if (SparkSystemUtils.isWindows) "windows"
else "unknown"
}
List(
diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala
index ceeece073da65..7597a0ceeb8cd 100644
--- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala
+++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala
@@ -200,6 +200,10 @@ object ArrowDeserializers {
new LeafFieldDeserializer[LocalDateTime](encoder, v, timeZoneId) {
override def value(i: Int): LocalDateTime = reader.getLocalDateTime(i)
}
+ case (LocalTimeEncoder, v: FieldVector) =>
+ new LeafFieldDeserializer[LocalTime](encoder, v, timeZoneId) {
+ override def value(i: Int): LocalTime = reader.getLocalTime(i)
+ }
case (OptionEncoder(value), v) =>
val deserializer = deserializerFor(value, v, timeZoneId)
diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowSerializer.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowSerializer.scala
index d79fb25ec1a0b..4acb11f014d19 100644
--- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowSerializer.scala
+++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowSerializer.scala
@@ -20,7 +20,7 @@ import java.io.{ByteArrayOutputStream, OutputStream}
import java.lang.invoke.{MethodHandles, MethodType}
import java.math.{BigDecimal => JBigDecimal, BigInteger => JBigInteger}
import java.nio.channels.Channels
-import java.time.{Duration, Instant, LocalDate, LocalDateTime, Period}
+import java.time.{Duration, Instant, LocalDate, LocalDateTime, LocalTime, Period}
import java.util.{Map => JMap, Objects}
import scala.jdk.CollectionConverters._
@@ -392,6 +392,11 @@ object ArrowSerializer {
override def set(index: Int, value: LocalDateTime): Unit =
vector.setSafe(index, SparkDateTimeUtils.localDateTimeToMicros(value))
}
+ case (LocalTimeEncoder, v: TimeNanoVector) =>
+ new FieldSerializer[LocalTime, TimeNanoVector](v) {
+ override def set(index: Int, value: LocalTime): Unit =
+ vector.setSafe(index, SparkDateTimeUtils.localTimeToNanos(value))
+ }
case (OptionEncoder(value), v) =>
new Serializer {
diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowVectorReader.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowVectorReader.scala
index 3dbfce18e7b48..ea57e0e1c77fa 100644
--- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowVectorReader.scala
+++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowVectorReader.scala
@@ -18,17 +18,18 @@ package org.apache.spark.sql.connect.client.arrow
import java.math.{BigDecimal => JBigDecimal}
import java.sql.{Date, Timestamp}
-import java.time.{Duration, Instant, LocalDate, LocalDateTime, Period, ZoneOffset}
+import java.time.{Duration, Instant, LocalDate, LocalDateTime, LocalTime, Period, ZoneOffset}
import org.apache.arrow.vector._
import org.apache.arrow.vector.util.Text
-import org.apache.spark.sql.catalyst.util.{DateFormatter, SparkIntervalUtils, SparkStringUtils, TimestampFormatter}
+import org.apache.spark.sql.catalyst.util.{DateFormatter, SparkIntervalUtils, TimeFormatter, TimestampFormatter}
import org.apache.spark.sql.catalyst.util.DateTimeConstants.MICROS_PER_SECOND
import org.apache.spark.sql.catalyst.util.IntervalStringStyles.ANSI_STYLE
import org.apache.spark.sql.catalyst.util.SparkDateTimeUtils._
import org.apache.spark.sql.types.{DataType, DayTimeIntervalType, Decimal, UpCastRule, YearMonthIntervalType}
import org.apache.spark.sql.util.ArrowUtils
+import org.apache.spark.util.SparkStringUtils
/**
* Base class for reading leaf values from an arrow vector. This reader has read methods for all
@@ -59,6 +60,7 @@ private[arrow] abstract class ArrowVectorReader {
def getInstant(i: Int): java.time.Instant = unsupported()
def getLocalDate(i: Int): java.time.LocalDate = unsupported()
def getLocalDateTime(i: Int): java.time.LocalDateTime = unsupported()
+ def getLocalTime(i: Int): java.time.LocalTime = unsupported()
private def unsupported(): Nothing = throw new UnsupportedOperationException()
}
@@ -90,6 +92,7 @@ object ArrowVectorReader {
case v: DateDayVector => new DateDayVectorReader(v, timeZoneId)
case v: TimeStampMicroTZVector => new TimeStampMicroTZVectorReader(v)
case v: TimeStampMicroVector => new TimeStampMicroVectorReader(v, timeZoneId)
+ case v: TimeNanoVector => new TimeVectorReader(v)
case _: NullVector => NullVectorReader
case _ => throw new RuntimeException("Unsupported Vector Type: " + vector.getClass)
}
@@ -275,3 +278,11 @@ private[arrow] class TimeStampMicroVectorReader(v: TimeStampMicroVector, timeZon
override def getLocalDateTime(i: Int): LocalDateTime = microsToLocalDateTime(utcMicros(i))
override def getString(i: Int): String = formatter.format(utcMicros(i))
}
+
+private[arrow] class TimeVectorReader(v: TimeNanoVector)
+ extends TypedArrowVectorReader[TimeNanoVector](v) {
+ private lazy val formatter = TimeFormatter.getFractionFormatter()
+ private def nanos(i: Int): Long = vector.get(i)
+ override def getLocalTime(i: Int): LocalTime = nanosToLocalTime(nanos(i))
+ override def getString(i: Int): String = formatter.format(nanos(i))
+}
diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/columnNodeSupport.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/columnNodeSupport.scala
index 1e798387726bb..cbbec0599b77b 100644
--- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/columnNodeSupport.scala
+++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/columnNodeSupport.scala
@@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.encoders.RowEncoder
import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, Origin}
import org.apache.spark.sql.connect.ConnectConversions._
import org.apache.spark.sql.connect.common.DataTypeProtoConverter
-import org.apache.spark.sql.connect.common.LiteralValueProtoConverter.toLiteralProtoBuilder
+import org.apache.spark.sql.connect.common.LiteralValueProtoConverter.{toLiteralProtoBuilderWithOptions, ToLiteralProtoOptions}
import org.apache.spark.sql.expressions.{Aggregator, UserDefinedAggregateFunction, UserDefinedAggregator, UserDefinedFunction}
import org.apache.spark.sql.internal.{Alias, CaseWhenOtherwise, Cast, ColumnNode, ColumnNodeLike, InvokeInlineUserDefinedFunction, LambdaFunction, LazyExpression, Literal, SortOrder, SqlExpression, SubqueryExpression, SubqueryType, UnresolvedAttribute, UnresolvedExtractValue, UnresolvedFunction, UnresolvedNamedLambdaVariable, UnresolvedRegex, UnresolvedStar, UpdateFields, Window, WindowFrame}
@@ -65,11 +65,12 @@ object ColumnNodeToProtoConverter extends (ColumnNode => proto.Expression) {
val builder = proto.Expression.newBuilder()
val n = additionalTransformation.map(_(node)).getOrElse(node)
n match {
- case Literal(value, None, _) =>
- builder.setLiteral(toLiteralProtoBuilder(value))
-
- case Literal(value, Some(dataType), _) =>
- builder.setLiteral(toLiteralProtoBuilder(value, dataType))
+ case Literal(value, dataTypeOpt, _) =>
+ builder.setLiteral(
+ toLiteralProtoBuilderWithOptions(
+ value,
+ dataTypeOpt,
+ ToLiteralProtoOptions(useDeprecatedDataTypeFields = false)))
case u @ UnresolvedAttribute(unparsedIdentifier, planId, isMetadataColumn, _) =>
val escapedName = u.sql
diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/DataTypeProtoConverter.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/DataTypeProtoConverter.scala
index 8c83ad3d1f550..4ff555c5645bd 100644
--- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/DataTypeProtoConverter.scala
+++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/DataTypeProtoConverter.scala
@@ -53,6 +53,12 @@ object DataTypeProtoConverter {
case proto.DataType.KindCase.DATE => DateType
case proto.DataType.KindCase.TIMESTAMP => TimestampType
case proto.DataType.KindCase.TIMESTAMP_NTZ => TimestampNTZType
+ case proto.DataType.KindCase.TIME =>
+ if (t.getTime.hasPrecision) {
+ TimeType(t.getTime.getPrecision)
+ } else {
+ TimeType()
+ }
case proto.DataType.KindCase.CALENDAR_INTERVAL => CalendarIntervalType
case proto.DataType.KindCase.YEAR_MONTH_INTERVAL =>
@@ -204,6 +210,12 @@ object DataTypeProtoConverter {
case TimestampNTZType => ProtoDataTypes.TimestampNTZType
+ case TimeType(precision) =>
+ proto.DataType
+ .newBuilder()
+ .setTime(proto.DataType.Time.newBuilder().setPrecision(precision).build())
+ .build()
+
case CalendarIntervalType => ProtoDataTypes.CalendarIntervalType
case YearMonthIntervalType(startField, endField) =>
diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/LiteralValueProtoConverter.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/LiteralValueProtoConverter.scala
index 1f3496fa89847..870a452e85ecd 100644
--- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/LiteralValueProtoConverter.scala
+++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/LiteralValueProtoConverter.scala
@@ -23,7 +23,6 @@ import java.sql.{Date, Timestamp}
import java.time._
import scala.collection.{immutable, mutable}
-import scala.jdk.CollectionConverters._
import scala.reflect.ClassTag
import scala.reflect.runtime.universe.TypeTag
import scala.util.Try
@@ -40,14 +39,10 @@ import org.apache.spark.util.SparkClassUtils
object LiteralValueProtoConverter {
- /**
- * Transforms literal value to the `proto.Expression.Literal.Builder`.
- *
- * @return
- * proto.Expression.Literal.Builder
- */
@scala.annotation.tailrec
- def toLiteralProtoBuilder(literal: Any): proto.Expression.Literal.Builder = {
+ private def toLiteralProtoBuilderInternal(
+ literal: Any,
+ options: ToLiteralProtoOptions): proto.Expression.Literal.Builder = {
val builder = proto.Expression.Literal.newBuilder()
def decimalBuilder(precision: Int, scale: Int, value: String) = {
@@ -63,8 +58,17 @@ object LiteralValueProtoConverter {
def arrayBuilder(array: Array[_]) = {
val ab = builder.getArrayBuilder
- .setElementType(toConnectProtoType(toDataType(array.getClass.getComponentType)))
- array.foreach(x => ab.addElements(toLiteralProto(x)))
+ if (options.useDeprecatedDataTypeFields) {
+ ab.setElementType(toConnectProtoType(toDataType(array.getClass.getComponentType)))
+ } else {
+ ab.setDataType(
+ proto.DataType.Array
+ .newBuilder()
+ .setElementType(toConnectProtoType(toDataType(array.getClass.getComponentType)))
+ .setContainsNull(true)
+ .build())
+ }
+ array.foreach(x => ab.addElements(toLiteralProtoWithOptions(x, None, options)))
ab
}
@@ -84,8 +88,8 @@ object LiteralValueProtoConverter {
case v: Char => builder.setString(v.toString)
case v: Array[Char] => builder.setString(String.valueOf(v))
case v: Array[Byte] => builder.setBinary(ByteString.copyFrom(v))
- case v: mutable.ArraySeq[_] => toLiteralProtoBuilder(v.array)
- case v: immutable.ArraySeq[_] => toLiteralProtoBuilder(v.unsafeArray)
+ case v: mutable.ArraySeq[_] => toLiteralProtoBuilderInternal(v.array, options)
+ case v: immutable.ArraySeq[_] => toLiteralProtoBuilderInternal(v.unsafeArray, options)
case v: LocalDate => builder.setDate(v.toEpochDay.toInt)
case v: Decimal =>
builder.setDecimal(decimalBuilder(Math.max(v.precision, v.scale), v.scale, v.toString))
@@ -96,6 +100,11 @@ object LiteralValueProtoConverter {
case v: Date => builder.setDate(SparkDateTimeUtils.fromJavaDate(v))
case v: Duration => builder.setDayTimeInterval(SparkIntervalUtils.durationToMicros(v))
case v: Period => builder.setYearMonthInterval(SparkIntervalUtils.periodToMonths(v))
+ case v: LocalTime =>
+ builder.setTime(
+ builder.getTimeBuilder
+ .setNano(SparkDateTimeUtils.localTimeToNanos(v))
+ .setPrecision(TimeType.DEFAULT_PRECISION))
case v: Array[_] => builder.setArray(arrayBuilder(v))
case v: CalendarInterval =>
builder.setCalendarInterval(calendarIntervalBuilder(v.months, v.days, v.microseconds))
@@ -105,19 +114,31 @@ object LiteralValueProtoConverter {
}
@scala.annotation.tailrec
- def toLiteralProtoBuilder(
+ private def toLiteralProtoBuilderInternal(
literal: Any,
- dataType: DataType): proto.Expression.Literal.Builder = {
+ dataType: DataType,
+ options: ToLiteralProtoOptions): proto.Expression.Literal.Builder = {
val builder = proto.Expression.Literal.newBuilder()
- def arrayBuilder(scalaValue: Any, elementType: DataType) = {
- val ab = builder.getArrayBuilder.setElementType(toConnectProtoType(elementType))
-
+ def arrayBuilder(scalaValue: Any, elementType: DataType, containsNull: Boolean) = {
+ val ab = builder.getArrayBuilder
+ if (options.useDeprecatedDataTypeFields) {
+ ab.setElementType(toConnectProtoType(elementType))
+ } else {
+ ab.setDataType(
+ proto.DataType.Array
+ .newBuilder()
+ .setElementType(toConnectProtoType(elementType))
+ .setContainsNull(containsNull)
+ .build())
+ }
scalaValue match {
case a: Array[_] =>
- a.foreach(item => ab.addElements(toLiteralProto(item, elementType)))
+ a.foreach(item =>
+ ab.addElements(toLiteralProtoWithOptions(item, Some(elementType), options)))
case s: scala.collection.Seq[_] =>
- s.foreach(item => ab.addElements(toLiteralProto(item, elementType)))
+ s.foreach(item =>
+ ab.addElements(toLiteralProtoWithOptions(item, Some(elementType), options)))
case other =>
throw new IllegalArgumentException(s"literal $other not supported (yet).")
}
@@ -125,16 +146,30 @@ object LiteralValueProtoConverter {
ab
}
- def mapBuilder(scalaValue: Any, keyType: DataType, valueType: DataType) = {
+ def mapBuilder(
+ scalaValue: Any,
+ keyType: DataType,
+ valueType: DataType,
+ valueContainsNull: Boolean) = {
val mb = builder.getMapBuilder
- .setKeyType(toConnectProtoType(keyType))
- .setValueType(toConnectProtoType(valueType))
+ if (options.useDeprecatedDataTypeFields) {
+ mb.setKeyType(toConnectProtoType(keyType))
+ mb.setValueType(toConnectProtoType(valueType))
+ } else {
+ mb.setDataType(
+ proto.DataType.Map
+ .newBuilder()
+ .setKeyType(toConnectProtoType(keyType))
+ .setValueType(toConnectProtoType(valueType))
+ .setValueContainsNull(valueContainsNull)
+ .build())
+ }
scalaValue match {
case map: scala.collection.Map[_, _] =>
map.foreach { case (k, v) =>
- mb.addKeys(toLiteralProto(k, keyType))
- mb.addValues(toLiteralProto(v, valueType))
+ mb.addKeys(toLiteralProtoWithOptions(k, Some(keyType), options))
+ mb.addValues(toLiteralProtoWithOptions(v, Some(valueType), options))
}
case other =>
throw new IllegalArgumentException(s"literal $other not supported (yet).")
@@ -144,16 +179,47 @@ object LiteralValueProtoConverter {
}
def structBuilder(scalaValue: Any, structType: StructType) = {
- val sb = builder.getStructBuilder.setStructType(toConnectProtoType(structType))
- val dataTypes = structType.fields.map(_.dataType)
+ val sb = builder.getStructBuilder
+ val fields = structType.fields
scalaValue match {
case p: Product =>
val iter = p.productIterator
var idx = 0
- while (idx < structType.size) {
- sb.addElements(toLiteralProto(iter.next(), dataTypes(idx)))
- idx += 1
+ if (options.useDeprecatedDataTypeFields) {
+ while (idx < structType.size) {
+ val field = fields(idx)
+ val literalProto =
+ toLiteralProtoWithOptions(iter.next(), Some(field.dataType), options)
+ sb.addElements(literalProto)
+ idx += 1
+ }
+ sb.setStructType(toConnectProtoType(structType))
+ } else {
+ val dataTypeStruct = proto.DataType.Struct.newBuilder()
+ while (idx < structType.size) {
+ val field = fields(idx)
+ val literalProto =
+ toLiteralProtoWithOptions(iter.next(), Some(field.dataType), options)
+ sb.addElements(literalProto)
+
+ val fieldBuilder = dataTypeStruct
+ .addFieldsBuilder()
+ .setName(field.name)
+ .setNullable(field.nullable)
+
+ if (LiteralValueProtoConverter.getInferredDataType(literalProto).isEmpty) {
+ fieldBuilder.setDataType(toConnectProtoType(field.dataType))
+ }
+
+ // Set metadata if available
+ if (field.metadata != Metadata.empty) {
+ fieldBuilder.setMetadata(field.metadata.json)
+ }
+
+ idx += 1
+ }
+ sb.setDataTypeStruct(dataTypeStruct.build())
}
case other =>
throw new IllegalArgumentException(s"literal $other not supported (yet).")
@@ -164,24 +230,63 @@ object LiteralValueProtoConverter {
(literal, dataType) match {
case (v: mutable.ArraySeq[_], ArrayType(_, _)) =>
- toLiteralProtoBuilder(v.array, dataType)
+ toLiteralProtoBuilderInternal(v.array, dataType, options)
case (v: immutable.ArraySeq[_], ArrayType(_, _)) =>
- toLiteralProtoBuilder(v.unsafeArray, dataType)
+ toLiteralProtoBuilderInternal(v.unsafeArray, dataType, options)
case (v: Array[Byte], ArrayType(_, _)) =>
- toLiteralProtoBuilder(v)
- case (v, ArrayType(elementType, _)) =>
- builder.setArray(arrayBuilder(v, elementType))
- case (v, MapType(keyType, valueType, _)) =>
- builder.setMap(mapBuilder(v, keyType, valueType))
+ toLiteralProtoBuilderInternal(v, options)
+ case (v, ArrayType(elementType, containsNull)) =>
+ builder.setArray(arrayBuilder(v, elementType, containsNull))
+ case (v, MapType(keyType, valueType, valueContainsNull)) =>
+ builder.setMap(mapBuilder(v, keyType, valueType, valueContainsNull))
case (v, structType: StructType) =>
builder.setStruct(structBuilder(v, structType))
case (v: Option[_], _: DataType) =>
if (v.isDefined) {
- toLiteralProtoBuilder(v.get)
+ toLiteralProtoBuilderInternal(v.get, options)
} else {
builder.setNull(toConnectProtoType(dataType))
}
- case _ => toLiteralProtoBuilder(literal)
+ case (v: LocalTime, timeType: TimeType) =>
+ builder.setTime(
+ builder.getTimeBuilder
+ .setNano(SparkDateTimeUtils.localTimeToNanos(v))
+ .setPrecision(timeType.precision))
+ case _ => toLiteralProtoBuilderInternal(literal, options)
+ }
+
+ }
+
+ /**
+ * Transforms literal value to the `proto.Expression.Literal.Builder`.
+ *
+ * @return
+ * proto.Expression.Literal.Builder
+ */
+ def toLiteralProtoBuilder(literal: Any): proto.Expression.Literal.Builder = {
+ toLiteralProtoBuilderInternal(
+ literal,
+ ToLiteralProtoOptions(useDeprecatedDataTypeFields = true))
+ }
+
+ def toLiteralProtoBuilder(
+ literal: Any,
+ dataType: DataType): proto.Expression.Literal.Builder = {
+ toLiteralProtoBuilderInternal(
+ literal,
+ dataType,
+ ToLiteralProtoOptions(useDeprecatedDataTypeFields = true))
+ }
+
+ def toLiteralProtoBuilderWithOptions(
+ literal: Any,
+ dataTypeOpt: Option[DataType],
+ options: ToLiteralProtoOptions): proto.Expression.Literal.Builder = {
+ dataTypeOpt match {
+ case Some(dataType) =>
+ toLiteralProtoBuilderInternal(literal, dataType, options)
+ case None =>
+ toLiteralProtoBuilderInternal(literal, options)
}
}
@@ -192,6 +297,8 @@ object LiteralValueProtoConverter {
toLiteralProtoBuilder(v)
}
+ case class ToLiteralProtoOptions(useDeprecatedDataTypeFields: Boolean)
+
/**
* Transforms literal value to the `proto.Expression.Literal`.
*
@@ -199,10 +306,27 @@ object LiteralValueProtoConverter {
* proto.Expression.Literal
*/
def toLiteralProto(literal: Any): proto.Expression.Literal =
- toLiteralProtoBuilder(literal).build()
+ toLiteralProtoBuilderInternal(
+ literal,
+ ToLiteralProtoOptions(useDeprecatedDataTypeFields = true)).build()
def toLiteralProto(literal: Any, dataType: DataType): proto.Expression.Literal =
- toLiteralProtoBuilder(literal, dataType).build()
+ toLiteralProtoBuilderInternal(
+ literal,
+ dataType,
+ ToLiteralProtoOptions(useDeprecatedDataTypeFields = true)).build()
+
+ def toLiteralProtoWithOptions(
+ literal: Any,
+ dataTypeOpt: Option[DataType],
+ options: ToLiteralProtoOptions): proto.Expression.Literal = {
+ dataTypeOpt match {
+ case Some(dataType) =>
+ toLiteralProtoBuilderInternal(literal, dataType, options).build()
+ case None =>
+ toLiteralProtoBuilderInternal(literal, options).build()
+ }
+ }
private[sql] def toDataType(clz: Class[_]): DataType = clz match {
// primitive types
@@ -290,6 +414,9 @@ object LiteralValueProtoConverter {
case proto.Expression.Literal.LiteralTypeCase.ARRAY =>
toCatalystArray(literal.getArray)
+ case proto.Expression.Literal.LiteralTypeCase.STRUCT =>
+ toCatalystStruct(literal.getStruct)
+
case other =>
throw new UnsupportedOperationException(
s"Unsupported Literal Type: ${other.getNumber} (${other.name})")
@@ -297,86 +424,250 @@ object LiteralValueProtoConverter {
}
private def getConverter(dataType: proto.DataType): proto.Expression.Literal => Any = {
- if (dataType.hasShort) { v =>
- v.getShort.toShort
- } else if (dataType.hasInteger) { v =>
- v.getInteger
- } else if (dataType.hasLong) { v =>
- v.getLong
- } else if (dataType.hasDouble) { v =>
- v.getDouble
- } else if (dataType.hasByte) { v =>
- v.getByte.toByte
- } else if (dataType.hasFloat) { v =>
- v.getFloat
- } else if (dataType.hasBoolean) { v =>
- v.getBoolean
- } else if (dataType.hasString) { v =>
- v.getString
- } else if (dataType.hasBinary) { v =>
- v.getBinary.toByteArray
- } else if (dataType.hasDate) { v =>
- v.getDate
- } else if (dataType.hasTimestamp) { v =>
- v.getTimestamp
- } else if (dataType.hasTimestampNtz) { v =>
- v.getTimestampNtz
- } else if (dataType.hasDayTimeInterval) { v =>
- v.getDayTimeInterval
- } else if (dataType.hasYearMonthInterval) { v =>
- v.getYearMonthInterval
- } else if (dataType.hasDecimal) { v =>
- Decimal(v.getDecimal.getValue)
- } else if (dataType.hasCalendarInterval) { v =>
- val interval = v.getCalendarInterval
- new CalendarInterval(interval.getMonths, interval.getDays, interval.getMicroseconds)
- } else if (dataType.hasArray) { v =>
- toCatalystArray(v.getArray)
- } else if (dataType.hasMap) { v =>
- toCatalystMap(v.getMap)
- } else if (dataType.hasStruct) { v =>
- toCatalystStruct(v.getStruct)
- } else {
- throw InvalidPlanInput(s"Unsupported Literal Type: $dataType)")
+ dataType.getKindCase match {
+ case proto.DataType.KindCase.SHORT => v => v.getShort.toShort
+ case proto.DataType.KindCase.INTEGER => v => v.getInteger
+ case proto.DataType.KindCase.LONG => v => v.getLong
+ case proto.DataType.KindCase.DOUBLE => v => v.getDouble
+ case proto.DataType.KindCase.BYTE => v => v.getByte.toByte
+ case proto.DataType.KindCase.FLOAT => v => v.getFloat
+ case proto.DataType.KindCase.BOOLEAN => v => v.getBoolean
+ case proto.DataType.KindCase.STRING => v => v.getString
+ case proto.DataType.KindCase.BINARY => v => v.getBinary.toByteArray
+ case proto.DataType.KindCase.DATE => v => v.getDate
+ case proto.DataType.KindCase.TIMESTAMP => v => v.getTimestamp
+ case proto.DataType.KindCase.TIMESTAMP_NTZ => v => v.getTimestampNtz
+ case proto.DataType.KindCase.DAY_TIME_INTERVAL => v => v.getDayTimeInterval
+ case proto.DataType.KindCase.YEAR_MONTH_INTERVAL => v => v.getYearMonthInterval
+ case proto.DataType.KindCase.DECIMAL => v => Decimal(v.getDecimal.getValue)
+ case proto.DataType.KindCase.CALENDAR_INTERVAL =>
+ v =>
+ val interval = v.getCalendarInterval
+ new CalendarInterval(interval.getMonths, interval.getDays, interval.getMicroseconds)
+ case proto.DataType.KindCase.ARRAY =>
+ v => toCatalystArrayInternal(v.getArray, dataType.getArray)
+ case proto.DataType.KindCase.MAP =>
+ v => toCatalystMapInternal(v.getMap, dataType.getMap)
+ case proto.DataType.KindCase.STRUCT =>
+ v => toCatalystStructInternal(v.getStruct, dataType.getStruct)
+ case _ =>
+ throw InvalidPlanInput(s"Unsupported Literal Type: ${dataType.getKindCase}")
}
}
- def toCatalystArray(array: proto.Expression.Literal.Array): Array[_] = {
+ private def getInferredDataType(
+ literal: proto.Expression.Literal,
+ recursive: Boolean = false): Option[proto.DataType] = {
+ if (literal.hasNull) {
+ return Some(literal.getNull)
+ }
+
+ val builder = proto.DataType.newBuilder()
+ literal.getLiteralTypeCase match {
+ case proto.Expression.Literal.LiteralTypeCase.BINARY =>
+ builder.setBinary(proto.DataType.Binary.newBuilder.build())
+ case proto.Expression.Literal.LiteralTypeCase.BOOLEAN =>
+ builder.setBoolean(proto.DataType.Boolean.newBuilder.build())
+ case proto.Expression.Literal.LiteralTypeCase.BYTE =>
+ builder.setByte(proto.DataType.Byte.newBuilder.build())
+ case proto.Expression.Literal.LiteralTypeCase.SHORT =>
+ builder.setShort(proto.DataType.Short.newBuilder.build())
+ case proto.Expression.Literal.LiteralTypeCase.INTEGER =>
+ builder.setInteger(proto.DataType.Integer.newBuilder.build())
+ case proto.Expression.Literal.LiteralTypeCase.LONG =>
+ builder.setLong(proto.DataType.Long.newBuilder.build())
+ case proto.Expression.Literal.LiteralTypeCase.FLOAT =>
+ builder.setFloat(proto.DataType.Float.newBuilder.build())
+ case proto.Expression.Literal.LiteralTypeCase.DOUBLE =>
+ builder.setDouble(proto.DataType.Double.newBuilder.build())
+ case proto.Expression.Literal.LiteralTypeCase.DATE =>
+ builder.setDate(proto.DataType.Date.newBuilder.build())
+ case proto.Expression.Literal.LiteralTypeCase.TIMESTAMP =>
+ builder.setTimestamp(proto.DataType.Timestamp.newBuilder.build())
+ case proto.Expression.Literal.LiteralTypeCase.TIMESTAMP_NTZ =>
+ builder.setTimestampNtz(proto.DataType.TimestampNTZ.newBuilder.build())
+ case proto.Expression.Literal.LiteralTypeCase.CALENDAR_INTERVAL =>
+ builder.setCalendarInterval(proto.DataType.CalendarInterval.newBuilder.build())
+ case proto.Expression.Literal.LiteralTypeCase.STRUCT =>
+ if (recursive) {
+ val struct = literal.getStruct
+ val size = struct.getElementsCount
+ val structTypeBuilder = proto.DataType.Struct.newBuilder
+ var i = 0
+ while (i < size) {
+ val field = struct.getDataTypeStruct.getFields(i)
+ if (field.hasDataType) {
+ structTypeBuilder.addFields(field)
+ } else {
+ val element = struct.getElements(i)
+ getInferredDataType(element, recursive = true) match {
+ case Some(dataType) =>
+ val fieldBuilder = structTypeBuilder.addFieldsBuilder()
+ fieldBuilder.setName(field.getName)
+ fieldBuilder.setDataType(dataType)
+ fieldBuilder.setNullable(field.getNullable)
+ if (field.hasMetadata) {
+ fieldBuilder.setMetadata(field.getMetadata)
+ }
+ case None => return None
+ }
+ }
+ i += 1
+ }
+ builder.setStruct(structTypeBuilder.build())
+ } else {
+ builder.setStruct(proto.DataType.Struct.newBuilder.build())
+ }
+ case proto.Expression.Literal.LiteralTypeCase.ARRAY =>
+ if (recursive) {
+ val arrayType = literal.getArray.getDataType
+ val elementTypeOpt = if (arrayType.hasElementType) {
+ Some(arrayType.getElementType)
+ } else if (literal.getArray.getElementsCount > 0) {
+ getInferredDataType(literal.getArray.getElements(0), recursive = true)
+ } else {
+ None
+ }
+ if (elementTypeOpt.isDefined) {
+ builder.setArray(
+ proto.DataType.Array
+ .newBuilder()
+ .setElementType(elementTypeOpt.get)
+ .setContainsNull(arrayType.getContainsNull)
+ .build())
+ } else {
+ return None
+ }
+ } else {
+ builder.setArray(proto.DataType.Array.newBuilder.build())
+ }
+ case proto.Expression.Literal.LiteralTypeCase.MAP =>
+ if (recursive) {
+ val mapType = literal.getMap.getDataType
+ val keyTypeOpt = if (mapType.hasKeyType) {
+ Some(mapType.getKeyType)
+ } else if (literal.getMap.getKeysCount > 0) {
+ getInferredDataType(literal.getMap.getKeys(0), recursive = true)
+ } else {
+ None
+ }
+ val valueTypeOpt = if (mapType.hasValueType) {
+ Some(mapType.getValueType)
+ } else if (literal.getMap.getValuesCount > 0) {
+ getInferredDataType(literal.getMap.getValues(0), recursive = true)
+ } else {
+ None
+ }
+ if (keyTypeOpt.isDefined && valueTypeOpt.isDefined) {
+ builder.setMap(
+ proto.DataType.Map.newBuilder
+ .setKeyType(keyTypeOpt.get)
+ .setValueType(valueTypeOpt.get)
+ .setValueContainsNull(mapType.getValueContainsNull)
+ .build())
+ } else {
+ return None
+ }
+ } else {
+ builder.setMap(proto.DataType.Map.newBuilder.build())
+ }
+ case _ =>
+ // Not all data types support inferring the data type from the literal at the moment.
+ // e.g. the type of DayTimeInterval contains extra information like start_field and
+ // end_field and cannot be inferred from the literal.
+ return None
+ }
+ Some(builder.build())
+ }
+
+ private def toCatalystArrayInternal(
+ array: proto.Expression.Literal.Array,
+ arrayType: proto.DataType.Array): Array[_] = {
def makeArrayData[T](converter: proto.Expression.Literal => T)(implicit
tag: ClassTag[T]): Array[T] = {
- val builder = mutable.ArrayBuilder.make[T]
- val elementList = array.getElementsList
- builder.sizeHint(elementList.size())
- val iter = elementList.iterator()
- while (iter.hasNext) {
- builder += converter(iter.next())
+ val size = array.getElementsCount
+ if (size > 0) {
+ Array.tabulate(size)(i => converter(array.getElements(i)))
+ } else {
+ Array.empty[T]
}
- builder.result()
}
- makeArrayData(getConverter(array.getElementType))
+ makeArrayData(getConverter(arrayType.getElementType))
}
- def toCatalystMap(map: proto.Expression.Literal.Map): mutable.Map[_, _] = {
+ def getProtoArrayType(array: proto.Expression.Literal.Array): proto.DataType.Array = {
+ if (array.hasDataType) {
+ val literal = proto.Expression.Literal.newBuilder().setArray(array).build()
+ getInferredDataType(literal, recursive = true) match {
+ case Some(dataType) => dataType.getArray
+ case None => throw InvalidPlanInput("Cannot infer data type from this array literal.")
+ }
+ } else if (array.hasElementType) {
+ // For backward compatibility, we still support the old way to
+ // define the type of the array.
+ proto.DataType.Array.newBuilder
+ .setElementType(array.getElementType)
+ .setContainsNull(true)
+ .build()
+ } else {
+ throw InvalidPlanInput("Data type information is missing in the array literal.")
+ }
+ }
+
+ def toCatalystArray(array: proto.Expression.Literal.Array): Array[_] = {
+ toCatalystArrayInternal(array, getProtoArrayType(array))
+ }
+
+ private def toCatalystMapInternal(
+ map: proto.Expression.Literal.Map,
+ mapType: proto.DataType.Map): mutable.Map[_, _] = {
def makeMapData[K, V](
keyConverter: proto.Expression.Literal => K,
valueConverter: proto.Expression.Literal => V)(implicit
tagK: ClassTag[K],
tagV: ClassTag[V]): mutable.Map[K, V] = {
- val builder = mutable.HashMap.empty[K, V]
- val keys = map.getKeysList.asScala
- val values = map.getValuesList.asScala
- builder.sizeHint(keys.size)
- keys.zip(values).foreach { case (key, value) =>
- builder += ((keyConverter(key), valueConverter(value)))
+ val size = map.getKeysCount
+ if (size > 0) {
+ val m = mutable.LinkedHashMap.empty[K, V]
+ m.sizeHint(size)
+ m.addAll(Iterator.tabulate(size)(i =>
+ (keyConverter(map.getKeys(i)), valueConverter(map.getValues(i)))))
+ } else {
+ mutable.Map.empty[K, V]
}
- builder
}
- makeMapData(getConverter(map.getKeyType), getConverter(map.getValueType))
+ makeMapData(getConverter(mapType.getKeyType), getConverter(mapType.getValueType))
}
- def toCatalystStruct(struct: proto.Expression.Literal.Struct): Any = {
+ def getProtoMapType(map: proto.Expression.Literal.Map): proto.DataType.Map = {
+ if (map.hasDataType) {
+ val literal = proto.Expression.Literal.newBuilder().setMap(map).build()
+ getInferredDataType(literal, recursive = true) match {
+ case Some(dataType) => dataType.getMap
+ case None => throw InvalidPlanInput("Cannot infer data type from this map literal.")
+ }
+ } else if (map.hasKeyType && map.hasValueType) {
+ // For backward compatibility, we still support the old way to
+ // define the type of the map.
+ proto.DataType.Map.newBuilder
+ .setKeyType(map.getKeyType)
+ .setValueType(map.getValueType)
+ .setValueContainsNull(true)
+ .build()
+ } else {
+ throw InvalidPlanInput("Data type information is missing in the map literal.")
+ }
+ }
+
+ def toCatalystMap(map: proto.Expression.Literal.Map): mutable.Map[_, _] = {
+ toCatalystMapInternal(map, getProtoMapType(map))
+ }
+
+ private def toCatalystStructInternal(
+ struct: proto.Expression.Literal.Struct,
+ structType: proto.DataType.Struct): Any = {
def toTuple[A <: Object](data: Seq[A]): Product = {
try {
val tupleClass = SparkClassUtils.classForName(s"scala.Tuple${data.length}")
@@ -387,16 +678,32 @@ object LiteralValueProtoConverter {
}
}
- val elements = struct.getElementsList.asScala
- val dataTypes = struct.getStructType.getStruct.getFieldsList.asScala.map(_.getDataType)
- val structData = elements
- .zip(dataTypes)
- .map { case (element, dataType) =>
- getConverter(dataType)(element)
+ val size = struct.getElementsCount
+ val structData = Seq.tabulate(size) { i =>
+ val element = struct.getElements(i)
+ val dataType = structType.getFields(i).getDataType
+ getConverter(dataType)(element).asInstanceOf[Object]
+ }
+ toTuple(structData)
+ }
+
+ def getProtoStructType(struct: proto.Expression.Literal.Struct): proto.DataType.Struct = {
+ if (struct.hasDataTypeStruct) {
+ val literal = proto.Expression.Literal.newBuilder().setStruct(struct).build()
+ getInferredDataType(literal, recursive = true) match {
+ case Some(dataType) => dataType.getStruct
+ case None => throw InvalidPlanInput("Cannot infer data type from this struct literal.")
}
- .asInstanceOf[scala.collection.Seq[Object]]
- .toSeq
+ } else if (struct.hasStructType) {
+ // For backward compatibility, we still support the old way to
+ // define and convert struct types.
+ struct.getStructType.getStruct
+ } else {
+ throw InvalidPlanInput("Data type information is missing in the struct literal.")
+ }
+ }
- toTuple(structData)
+ def toCatalystStruct(struct: proto.Expression.Literal.Struct): Any = {
+ toCatalystStructInternal(struct, getProtoStructType(struct))
}
}
diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/ProtoSpecializedArray.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/ProtoSpecializedArray.scala
new file mode 100644
index 0000000000000..eeccb1f77469c
--- /dev/null
+++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/ProtoSpecializedArray.scala
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.connect.common
+
+import org.apache.spark.connect.proto
+
+private[spark] object ProtoSpecializedArray {
+
+ def toArray(array: proto.Bools): Array[Boolean] = {
+ val size = array.getValuesCount
+ if (size > 0) {
+ val a = Array.ofDim[Boolean](size)
+ var i = 0
+ while (i < size) {
+ a(i) = array.getValues(i)
+ i += 1
+ }
+ a
+ } else {
+ Array.emptyBooleanArray
+ }
+ }
+
+ def toArray(array: proto.Ints): Array[Int] = {
+ val size = array.getValuesCount
+ if (size > 0) {
+ val a = Array.ofDim[Int](size)
+ var i = 0
+ while (i < size) {
+ a(i) = array.getValues(i)
+ i += 1
+ }
+ a
+ } else {
+ Array.emptyIntArray
+ }
+ }
+
+ def toArray(array: proto.Longs): Array[Long] = {
+ val size = array.getValuesCount
+ if (size > 0) {
+ val a = Array.ofDim[Long](size)
+ var i = 0
+ while (i < size) {
+ a(i) = array.getValues(i)
+ i += 1
+ }
+ a
+ } else {
+ Array.emptyLongArray
+ }
+ }
+
+ def toArray(array: proto.Floats): Array[Float] = {
+ val size = array.getValuesCount
+ if (size > 0) {
+ val a = Array.ofDim[Float](size)
+ var i = 0
+ while (i < size) {
+ a(i) = array.getValues(i)
+ i += 1
+ }
+ a
+ } else {
+ Array.emptyFloatArray
+ }
+ }
+
+ def toArray(array: proto.Doubles): Array[Double] = {
+ val size = array.getValuesCount
+ if (size > 0) {
+ val a = Array.ofDim[Double](size)
+ var i = 0
+ while (i < size) {
+ a(i) = array.getValues(i)
+ i += 1
+ }
+ a
+ } else {
+ Array.emptyDoubleArray
+ }
+ }
+
+ def toArray(array: proto.Strings): Array[String] = {
+ val size = array.getValuesCount
+ if (size > 0) {
+ val a = Array.ofDim[String](size)
+ var i = 0
+ while (i < size) {
+ a(i) = array.getValues(i)
+ i += 1
+ }
+ a
+ } else {
+ Array.empty[String]
+ }
+ }
+
+ def fromArray(array: Array[Boolean]): proto.Bools = {
+ if (array.nonEmpty) {
+ val builder = proto.Bools.newBuilder()
+ array.foreach(builder.addValues)
+ builder.build()
+ } else {
+ proto.Bools.getDefaultInstance
+ }
+ }
+
+ def fromArray(array: Array[Int]): proto.Ints = {
+ if (array.nonEmpty) {
+ val builder = proto.Ints.newBuilder()
+ array.foreach(builder.addValues)
+ builder.build()
+ } else {
+ proto.Ints.getDefaultInstance
+ }
+ }
+
+ def fromArray(array: Array[Long]): proto.Longs = {
+ if (array.nonEmpty) {
+ val builder = proto.Longs.newBuilder()
+ array.foreach(builder.addValues)
+ builder.build()
+ } else {
+ proto.Longs.getDefaultInstance
+ }
+ }
+
+ def fromArray(array: Array[Float]): proto.Floats = {
+ if (array.nonEmpty) {
+ val builder = proto.Floats.newBuilder()
+ array.foreach(builder.addValues)
+ builder.build()
+ } else {
+ proto.Floats.getDefaultInstance
+ }
+ }
+
+ def fromArray(array: Array[Double]): proto.Doubles = {
+ if (array.nonEmpty) {
+ val builder = proto.Doubles.newBuilder()
+ array.foreach(builder.addValues)
+ builder.build()
+ } else {
+ proto.Doubles.getDefaultInstance
+ }
+ }
+
+ def fromArray(array: Array[String]): proto.Strings = {
+ if (array.nonEmpty) {
+ val builder = proto.Strings.newBuilder()
+ array.foreach(builder.addValues)
+ builder.build()
+ } else {
+ proto.Strings.getDefaultInstance
+ }
+ }
+}
diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_lit.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_lit.explain
index 4491b6166afae..888fdf71586fb 100644
--- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_lit.explain
+++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_lit.explain
@@ -1,2 +1,2 @@
-Project [id#0L, id#0L, true AS true#0, 68 AS 68#0, 9872 AS 9872#0, -8726532 AS -8726532#0, 7834609328726532 AS 7834609328726532#0L, 2.718281828459045 AS 2.718281828459045#0, -0.8 AS -0.8#0, 89.97620 AS 89.97620#0, 89889.7667231 AS 89889.7667231#0, connect! AS connect!#0, T AS T#0, ABCDEFGHIJ AS ABCDEFGHIJ#0, 0x78797A7B7C7D7E7F808182838485868788898A8B8C8D8E AS X'78797A7B7C7D7E7F808182838485868788898A8B8C8D8E'#0, 0x0806 AS X'0806'#0, [8,6] AS ARRAY(8, 6)#0, null AS NULL#0, 2020-10-10 AS DATE '2020-10-10'#0, 8.997620 AS 8.997620#0, 2023-02-23 04:31:59.808 AS TIMESTAMP '2023-02-23 04:31:59.808'#0, 1969-12-31 16:00:12.345 AS TIMESTAMP '1969-12-31 16:00:12.345'#0, 2023-02-23 20:36:00 AS TIMESTAMP_NTZ '2023-02-23 20:36:00'#0, 2023-02-23 AS DATE '2023-02-23'#0, INTERVAL '0 00:03:20' DAY TO SECOND AS INTERVAL '0 00:03:20' DAY TO SECOND#0, ... 2 more fields]
+Project [id#0L, id#0L, true AS true#0, 68 AS 68#0, 9872 AS 9872#0, -8726532 AS -8726532#0, 7834609328726532 AS 7834609328726532#0L, 2.718281828459045 AS 2.718281828459045#0, -0.8 AS -0.8#0, 89.97620 AS 89.97620#0, 89889.7667231 AS 89889.7667231#0, connect! AS connect!#0, T AS T#0, ABCDEFGHIJ AS ABCDEFGHIJ#0, 0x78797A7B7C7D7E7F808182838485868788898A8B8C8D8E AS X'78797A7B7C7D7E7F808182838485868788898A8B8C8D8E'#0, 0x0806 AS X'0806'#0, [8,6] AS ARRAY(8, 6)#0, null AS NULL#0, 2020-10-10 AS DATE '2020-10-10'#0, 8.997620 AS 8.997620#0, 2023-02-23 04:31:59.808 AS TIMESTAMP '2023-02-23 04:31:59.808'#0, 1969-12-31 16:00:12.345 AS TIMESTAMP '1969-12-31 16:00:12.345'#0, 2023-02-23 20:36:00 AS TIMESTAMP_NTZ '2023-02-23 20:36:00'#0, 2023-02-23 AS DATE '2023-02-23'#0, INTERVAL '0 00:03:20' DAY TO SECOND AS INTERVAL '0 00:03:20' DAY TO SECOND#0, INTERVAL '0-0' YEAR TO MONTH AS INTERVAL '0-0' YEAR TO MONTH#0, 23:59:59.999999999 AS TIME '23:59:59.999999999'#0, 2 months 20 days 0.0001 seconds AS INTERVAL '2 months 20 days 0.0001 seconds'#0]
+- LocalRelation , [id#0L, a#0, b#0]
diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_try_to_date.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_try_to_date.explain
new file mode 100644
index 0000000000000..b154d54f57533
--- /dev/null
+++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_try_to_date.explain
@@ -0,0 +1,2 @@
+Project [cast(s#0 as date) AS try_to_date(s)#0]
++- LocalRelation , [d#0, t#0, s#0, x#0L, wt#0]
diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_try_to_date_with_format.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_try_to_date_with_format.explain
new file mode 100644
index 0000000000000..ab3e72abc21da
--- /dev/null
+++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_try_to_date_with_format.explain
@@ -0,0 +1,2 @@
+Project [cast(gettimestamp(s#0, yyyy-MM-dd, TimestampType, try_to_date, Some(America/Los_Angeles), false) as date) AS try_to_date(s, yyyy-MM-dd)#0]
++- LocalRelation , [d#0, t#0, s#0, x#0L, wt#0]
diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_typedLit.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_typedLit.explain
index 6d854da250fcc..943b353a14ccf 100644
--- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_typedLit.explain
+++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_typedLit.explain
@@ -1,2 +1,2 @@
-Project [id#0L, id#0L, 1 AS 1#0, null AS NULL#0, true AS true#0, 68 AS 68#0, 9872 AS 9872#0, -8726532 AS -8726532#0, 7834609328726532 AS 7834609328726532#0L, 2.718281828459045 AS 2.718281828459045#0, -0.8 AS -0.8#0, 89.97620 AS 89.97620#0, 89889.7667231 AS 89889.7667231#0, connect! AS connect!#0, T AS T#0, ABCDEFGHIJ AS ABCDEFGHIJ#0, 0x78797A7B7C7D7E7F808182838485868788898A8B8C8D8E AS X'78797A7B7C7D7E7F808182838485868788898A8B8C8D8E'#0, 0x0806 AS X'0806'#0, [8,6] AS ARRAY(8, 6)#0, null AS NULL#0, 2020-10-10 AS DATE '2020-10-10'#0, 8.997620 AS 8.997620#0, 2023-02-23 04:31:59.808 AS TIMESTAMP '2023-02-23 04:31:59.808'#0, 1969-12-31 16:00:12.345 AS TIMESTAMP '1969-12-31 16:00:12.345'#0, 2023-02-23 20:36:00 AS TIMESTAMP_NTZ '2023-02-23 20:36:00'#0, ... 18 more fields]
+Project [id#0L, id#0L, 1 AS 1#0, null AS NULL#0, true AS true#0, 68 AS 68#0, 9872 AS 9872#0, -8726532 AS -8726532#0, 7834609328726532 AS 7834609328726532#0L, 2.718281828459045 AS 2.718281828459045#0, -0.8 AS -0.8#0, 89.97620 AS 89.97620#0, 89889.7667231 AS 89889.7667231#0, connect! AS connect!#0, T AS T#0, ABCDEFGHIJ AS ABCDEFGHIJ#0, 0x78797A7B7C7D7E7F808182838485868788898A8B8C8D8E AS X'78797A7B7C7D7E7F808182838485868788898A8B8C8D8E'#0, 0x0806 AS X'0806'#0, [8,6] AS ARRAY(8, 6)#0, null AS NULL#0, 2020-10-10 AS DATE '2020-10-10'#0, 8.997620 AS 8.997620#0, 2023-02-23 04:31:59.808 AS TIMESTAMP '2023-02-23 04:31:59.808'#0, 1969-12-31 16:00:12.345 AS TIMESTAMP '1969-12-31 16:00:12.345'#0, 2023-02-23 20:36:00 AS TIMESTAMP_NTZ '2023-02-23 20:36:00'#0, 2023-02-23 AS DATE '2023-02-23'#0, INTERVAL '0 00:03:20' DAY TO SECOND AS INTERVAL '0 00:03:20' DAY TO SECOND#0, INTERVAL '0-0' YEAR TO MONTH AS INTERVAL '0-0' YEAR TO MONTH#0, 23:59:59.999999999 AS TIME '23:59:59.999999999'#0, 2 months 20 days 0.0001 seconds AS INTERVAL '2 months 20 days 0.0001 seconds'#0, 1 AS 1#0, [1,2,3] AS ARRAY(1, 2, 3)#0, [1,2,3] AS ARRAY(1, 2, 3)#0, map(keys: [a,b], values: [1,2]) AS MAP('a', 1, 'b', 2)#0, [a,2,1.0] AS NAMED_STRUCT('_1', 'a', '_2', 2, '_3', 1.0D)#0, null AS NULL#0, [1] AS ARRAY(1)#0, map(keys: [1], values: [0]) AS MAP(1, 0)#0, map(keys: [1], values: [0]) AS MAP(1, 0)#0, map(keys: [1], values: [0]) AS MAP(1, 0)#0, [[1,2,3],[4,5,6],[7,8,9]] AS ARRAY(ARRAY(1, 2, 3), ARRAY(4, 5, 6), ARRAY(7, 8, 9))#0, [keys: [a,b], values: [1,2],keys: [a,b], values: [3,4],keys: [a,b], values: [5,6]] AS ARRAY(MAP('a', 1, 'b', 2), MAP('a', 3, 'b', 4), MAP('a', 5, 'b', 6))#0, map(keys: [1,2], values: [keys: [a,b], values: [1,2],keys: [a,b], values: [3,4]]) AS MAP(1, MAP('a', 1, 'b', 2), 2, MAP('a', 3, 'b', 4))#0, [[1,2,3],keys: [a,b], values: [1,2],[a,keys: [1,2], values: [a,b]]] AS NAMED_STRUCT('_1', ARRAY(1, 2, 3), '_2', MAP('a', 1, 'b', 2), '_3', NAMED_STRUCT('_1', 'a', '_2', MAP(1, 'a', 2, 'b')))#0]
+- LocalRelation , [id#0L, a#0, b#0]
diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/union.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/union.explain
index 4d5d1f53b8412..252774510896c 100644
--- a/sql/connect/common/src/test/resources/query-tests/explain-results/union.explain
+++ b/sql/connect/common/src/test/resources/query-tests/explain-results/union.explain
@@ -1,3 +1,3 @@
-Union false, false
+'Union false, false
:- LocalRelation , [id#0L, a#0, b#0]
+- LocalRelation , [id#0L, a#0, b#0]
diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/unionAll.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/unionAll.explain
index 4d5d1f53b8412..252774510896c 100644
--- a/sql/connect/common/src/test/resources/query-tests/explain-results/unionAll.explain
+++ b/sql/connect/common/src/test/resources/query-tests/explain-results/unionAll.explain
@@ -1,3 +1,3 @@
-Union false, false
+'Union false, false
:- LocalRelation , [id#0L, a#0, b#0]
+- LocalRelation , [id#0L, a#0, b#0]
diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/unionByName.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/unionByName.explain
index 6ec8eb37f50ed..2877c7cef0fda 100644
--- a/sql/connect/common/src/test/resources/query-tests/explain-results/unionByName.explain
+++ b/sql/connect/common/src/test/resources/query-tests/explain-results/unionByName.explain
@@ -1,4 +1,4 @@
-Union false, false
+'Union false, false
:- Project [id#0L, a#0]
: +- LocalRelation , [id#0L, a#0, b#0]
+- Project [id#0L, a#0]
diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/unionByName_allowMissingColumns.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/unionByName_allowMissingColumns.explain
index 96bd9f281c15e..dc0d1d94f85c1 100644
--- a/sql/connect/common/src/test/resources/query-tests/explain-results/unionByName_allowMissingColumns.explain
+++ b/sql/connect/common/src/test/resources/query-tests/explain-results/unionByName_allowMissingColumns.explain
@@ -1,4 +1,4 @@
-Union false, false
+'Union false, false
:- Project [id#0L, a#0, b#0, null AS payload#0]
: +- LocalRelation , [id#0L, a#0, b#0]
+- Project [id#0L, a#0, null AS b#0, payload#0]
diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_lit.json b/sql/connect/common/src/test/resources/query-tests/queries/function_lit.json
index 96bdfcc8105ba..cedf7572a1fd3 100644
--- a/sql/connect/common/src/test/resources/query-tests/queries/function_lit.json
+++ b/sql/connect/common/src/test/resources/query-tests/queries/function_lit.json
@@ -358,15 +358,18 @@
}, {
"literal": {
"array": {
- "elementType": {
- "integer": {
- }
- },
"elements": [{
"integer": 8
}, {
"integer": 6
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "integer": {
+ }
+ },
+ "containsNull": true
+ }
}
},
"common": {
@@ -582,6 +585,30 @@
}
}
}
+ }, {
+ "literal": {
+ "time": {
+ "nano": "86399999999999",
+ "precision": 6
+ }
+ },
+ "common": {
+ "origin": {
+ "jvmOrigin": {
+ "stackTrace": [{
+ "classLoaderName": "app",
+ "declaringClass": "org.apache.spark.sql.functions$",
+ "methodName": "lit",
+ "fileName": "functions.scala"
+ }, {
+ "classLoaderName": "app",
+ "declaringClass": "org.apache.spark.sql.PlanGenerationTestSuite",
+ "methodName": "~~trimmed~anonfun~~",
+ "fileName": "PlanGenerationTestSuite.scala"
+ }]
+ }
+ }
+ }
}, {
"literal": {
"calendarInterval": {
diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_lit.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_lit.proto.bin
index 8036cda6765f8..5d30f4fca159b 100644
Binary files a/sql/connect/common/src/test/resources/query-tests/queries/function_lit.proto.bin and b/sql/connect/common/src/test/resources/query-tests/queries/function_lit.proto.bin differ
diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_lit_array.json b/sql/connect/common/src/test/resources/query-tests/queries/function_lit_array.json
index 337b3366649f7..53b1a7b3947f9 100644
--- a/sql/connect/common/src/test/resources/query-tests/queries/function_lit_array.json
+++ b/sql/connect/common/src/test/resources/query-tests/queries/function_lit_array.json
@@ -14,9 +14,12 @@
"expressions": [{
"literal": {
"array": {
- "elementType": {
- "double": {
- }
+ "dataType": {
+ "elementType": {
+ "double": {
+ }
+ },
+ "containsNull": true
}
}
},
@@ -40,46 +43,58 @@
}, {
"literal": {
"array": {
- "elementType": {
- "array": {
- "elementType": {
- "integer": {
- }
- },
- "containsNull": true
- }
- },
"elements": [{
"array": {
- "elementType": {
- "integer": {
- }
- },
"elements": [{
"integer": 1
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "integer": {
+ }
+ },
+ "containsNull": true
+ }
}
}, {
"array": {
- "elementType": {
- "integer": {
- }
- },
"elements": [{
"integer": 2
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "integer": {
+ }
+ },
+ "containsNull": true
+ }
}
}, {
"array": {
- "elementType": {
- "integer": {
- }
- },
"elements": [{
"integer": 3
- }]
- }
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "integer": {
+ }
+ },
+ "containsNull": true
+ }
+ }
+ }],
+ "dataType": {
+ "elementType": {
+ "array": {
+ "elementType": {
+ "integer": {
+ }
+ },
+ "containsNull": true
+ }
+ },
+ "containsNull": true
+ }
}
},
"common": {
@@ -102,90 +117,111 @@
}, {
"literal": {
"array": {
- "elementType": {
- "array": {
- "elementType": {
- "array": {
- "elementType": {
- "integer": {
- }
- },
- "containsNull": true
- }
- },
- "containsNull": true
- }
- },
"elements": [{
"array": {
- "elementType": {
- "array": {
- "elementType": {
- "integer": {
- }
- },
- "containsNull": true
- }
- },
"elements": [{
"array": {
- "elementType": {
- "integer": {
- }
- },
"elements": [{
"integer": 1
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "integer": {
+ }
+ },
+ "containsNull": true
+ }
}
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "array": {
+ "elementType": {
+ "integer": {
+ }
+ },
+ "containsNull": true
+ }
+ },
+ "containsNull": true
+ }
}
}, {
"array": {
- "elementType": {
- "array": {
- "elementType": {
- "integer": {
- }
- },
- "containsNull": true
- }
- },
"elements": [{
"array": {
- "elementType": {
- "integer": {
- }
- },
"elements": [{
"integer": 2
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "integer": {
+ }
+ },
+ "containsNull": true
+ }
}
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "array": {
+ "elementType": {
+ "integer": {
+ }
+ },
+ "containsNull": true
+ }
+ },
+ "containsNull": true
+ }
}
}, {
"array": {
- "elementType": {
- "array": {
- "elementType": {
- "integer": {
- }
- },
- "containsNull": true
- }
- },
"elements": [{
"array": {
- "elementType": {
- "integer": {
- }
- },
"elements": [{
"integer": 3
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "integer": {
+ }
+ },
+ "containsNull": true
+ }
}
- }]
- }
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "array": {
+ "elementType": {
+ "integer": {
+ }
+ },
+ "containsNull": true
+ }
+ },
+ "containsNull": true
+ }
+ }
+ }],
+ "dataType": {
+ "elementType": {
+ "array": {
+ "elementType": {
+ "array": {
+ "elementType": {
+ "integer": {
+ }
+ },
+ "containsNull": true
+ }
+ },
+ "containsNull": true
+ }
+ },
+ "containsNull": true
+ }
}
},
"common": {
@@ -208,15 +244,18 @@
}, {
"literal": {
"array": {
- "elementType": {
- "boolean": {
- }
- },
"elements": [{
"boolean": true
}, {
"boolean": false
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "boolean": {
+ }
+ },
+ "containsNull": true
+ }
}
},
"common": {
@@ -260,17 +299,20 @@
}, {
"literal": {
"array": {
- "elementType": {
- "short": {
- }
- },
"elements": [{
"short": 9872
}, {
"short": 9873
}, {
"short": 9874
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "short": {
+ }
+ },
+ "containsNull": true
+ }
}
},
"common": {
@@ -293,17 +335,20 @@
}, {
"literal": {
"array": {
- "elementType": {
- "integer": {
- }
- },
"elements": [{
"integer": -8726532
}, {
"integer": 8726532
}, {
"integer": -8726533
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "integer": {
+ }
+ },
+ "containsNull": true
+ }
}
},
"common": {
@@ -326,17 +371,20 @@
}, {
"literal": {
"array": {
- "elementType": {
- "long": {
- }
- },
"elements": [{
"long": "7834609328726531"
}, {
"long": "7834609328726532"
}, {
"long": "7834609328726533"
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "long": {
+ }
+ },
+ "containsNull": true
+ }
}
},
"common": {
@@ -359,17 +407,20 @@
}, {
"literal": {
"array": {
- "elementType": {
- "double": {
- }
- },
"elements": [{
"double": 2.718281828459045
}, {
"double": 1.0
}, {
"double": 2.0
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "double": {
+ }
+ },
+ "containsNull": true
+ }
}
},
"common": {
@@ -392,17 +443,20 @@
}, {
"literal": {
"array": {
- "elementType": {
- "float": {
- }
- },
"elements": [{
"float": -0.8
}, {
"float": -0.7
}, {
"float": -0.9
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "float": {
+ }
+ },
+ "containsNull": true
+ }
}
},
"common": {
@@ -425,12 +479,6 @@
}, {
"literal": {
"array": {
- "elementType": {
- "decimal": {
- "scale": 18,
- "precision": 38
- }
- },
"elements": [{
"decimal": {
"value": "89.97620",
@@ -443,7 +491,16 @@
"precision": 7,
"scale": 5
}
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "decimal": {
+ "scale": 18,
+ "precision": 38
+ }
+ },
+ "containsNull": true
+ }
}
},
"common": {
@@ -466,12 +523,6 @@
}, {
"literal": {
"array": {
- "elementType": {
- "decimal": {
- "scale": 18,
- "precision": 38
- }
- },
"elements": [{
"decimal": {
"value": "89889.7667231",
@@ -484,7 +535,16 @@
"precision": 12,
"scale": 7
}
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "decimal": {
+ "scale": 18,
+ "precision": 38
+ }
+ },
+ "containsNull": true
+ }
}
},
"common": {
@@ -507,16 +567,19 @@
}, {
"literal": {
"array": {
- "elementType": {
- "string": {
- "collation": "UTF8_BINARY"
- }
- },
"elements": [{
"string": "connect!"
}, {
"string": "disconnect!"
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "string": {
+ "collation": "UTF8_BINARY"
+ }
+ },
+ "containsNull": true
+ }
}
},
"common": {
@@ -560,16 +623,19 @@
}, {
"literal": {
"array": {
- "elementType": {
- "string": {
- "collation": "UTF8_BINARY"
- }
- },
"elements": [{
"string": "ABCDEFGHIJ"
}, {
"string": "BCDEFGHIJK"
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "string": {
+ "collation": "UTF8_BINARY"
+ }
+ },
+ "containsNull": true
+ }
}
},
"common": {
@@ -592,15 +658,18 @@
}, {
"literal": {
"array": {
- "elementType": {
- "date": {
- }
- },
"elements": [{
"date": 18545
}, {
"date": 18546
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "date": {
+ }
+ },
+ "containsNull": true
+ }
}
},
"common": {
@@ -623,15 +692,18 @@
}, {
"literal": {
"array": {
- "elementType": {
- "timestamp": {
- }
- },
"elements": [{
"timestamp": "1677155519808000"
}, {
"timestamp": "1677155519809000"
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "timestamp": {
+ }
+ },
+ "containsNull": true
+ }
}
},
"common": {
@@ -654,15 +726,18 @@
}, {
"literal": {
"array": {
- "elementType": {
- "timestamp": {
- }
- },
"elements": [{
"timestamp": "12345000"
}, {
"timestamp": "23456000"
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "timestamp": {
+ }
+ },
+ "containsNull": true
+ }
}
},
"common": {
@@ -685,15 +760,18 @@
}, {
"literal": {
"array": {
- "elementType": {
- "timestampNtz": {
- }
- },
"elements": [{
"timestampNtz": "1677184560000000"
}, {
"timestampNtz": "1677188160000000"
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "timestampNtz": {
+ }
+ },
+ "containsNull": true
+ }
}
},
"common": {
@@ -716,15 +794,18 @@
}, {
"literal": {
"array": {
- "elementType": {
- "date": {
- }
- },
"elements": [{
"date": 19411
}, {
"date": 19417
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "date": {
+ }
+ },
+ "containsNull": true
+ }
}
},
"common": {
@@ -747,17 +828,20 @@
}, {
"literal": {
"array": {
- "elementType": {
- "dayTimeInterval": {
- "startField": 0,
- "endField": 3
- }
- },
"elements": [{
"dayTimeInterval": "100000000"
}, {
"dayTimeInterval": "200000000"
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "dayTimeInterval": {
+ "startField": 0,
+ "endField": 3
+ }
+ },
+ "containsNull": true
+ }
}
},
"common": {
@@ -780,17 +864,20 @@
}, {
"literal": {
"array": {
- "elementType": {
- "yearMonthInterval": {
- "startField": 0,
- "endField": 1
- }
- },
"elements": [{
"yearMonthInterval": 0
}, {
"yearMonthInterval": 0
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "yearMonthInterval": {
+ "startField": 0,
+ "endField": 1
+ }
+ },
+ "containsNull": true
+ }
}
},
"common": {
@@ -813,10 +900,6 @@
}, {
"literal": {
"array": {
- "elementType": {
- "calendarInterval": {
- }
- },
"elements": [{
"calendarInterval": {
"months": 2,
@@ -829,7 +912,14 @@
"days": 21,
"microseconds": "200"
}
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "calendarInterval": {
+ }
+ },
+ "containsNull": true
+ }
}
},
"common": {
diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_lit_array.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_lit_array.proto.bin
index 320da10258180..8cb965dd25a0b 100644
Binary files a/sql/connect/common/src/test/resources/query-tests/queries/function_lit_array.proto.bin and b/sql/connect/common/src/test/resources/query-tests/queries/function_lit_array.proto.bin differ
diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_date.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_date.json
new file mode 100644
index 0000000000000..fa22071a05e5e
--- /dev/null
+++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_date.json
@@ -0,0 +1,60 @@
+{
+ "common": {
+ "planId": "1"
+ },
+ "project": {
+ "input": {
+ "common": {
+ "planId": "0"
+ },
+ "localRelation": {
+ "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e"
+ }
+ },
+ "expressions": [{
+ "unresolvedFunction": {
+ "functionName": "try_to_date",
+ "arguments": [{
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "s"
+ },
+ "common": {
+ "origin": {
+ "jvmOrigin": {
+ "stackTrace": [{
+ "classLoaderName": "app",
+ "declaringClass": "org.apache.spark.sql.functions$",
+ "methodName": "col",
+ "fileName": "functions.scala"
+ }, {
+ "classLoaderName": "app",
+ "declaringClass": "org.apache.spark.sql.PlanGenerationTestSuite",
+ "methodName": "~~trimmed~anonfun~~",
+ "fileName": "PlanGenerationTestSuite.scala"
+ }]
+ }
+ }
+ }
+ }],
+ "isInternal": false
+ },
+ "common": {
+ "origin": {
+ "jvmOrigin": {
+ "stackTrace": [{
+ "classLoaderName": "app",
+ "declaringClass": "org.apache.spark.sql.functions$",
+ "methodName": "try_to_date",
+ "fileName": "functions.scala"
+ }, {
+ "classLoaderName": "app",
+ "declaringClass": "org.apache.spark.sql.PlanGenerationTestSuite",
+ "methodName": "~~trimmed~anonfun~~",
+ "fileName": "PlanGenerationTestSuite.scala"
+ }]
+ }
+ }
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_date.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_date.proto.bin
new file mode 100644
index 0000000000000..d322dec35417a
Binary files /dev/null and b/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_date.proto.bin differ
diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_date_with_format.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_date_with_format.json
new file mode 100644
index 0000000000000..ccf5e8970ce9f
--- /dev/null
+++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_date_with_format.json
@@ -0,0 +1,81 @@
+{
+ "common": {
+ "planId": "1"
+ },
+ "project": {
+ "input": {
+ "common": {
+ "planId": "0"
+ },
+ "localRelation": {
+ "schema": "struct\u003cd:date,t:timestamp,s:string,x:bigint,wt:struct\u003cstart:timestamp,end:timestamp\u003e\u003e"
+ }
+ },
+ "expressions": [{
+ "unresolvedFunction": {
+ "functionName": "try_to_date",
+ "arguments": [{
+ "unresolvedAttribute": {
+ "unparsedIdentifier": "s"
+ },
+ "common": {
+ "origin": {
+ "jvmOrigin": {
+ "stackTrace": [{
+ "classLoaderName": "app",
+ "declaringClass": "org.apache.spark.sql.functions$",
+ "methodName": "col",
+ "fileName": "functions.scala"
+ }, {
+ "classLoaderName": "app",
+ "declaringClass": "org.apache.spark.sql.PlanGenerationTestSuite",
+ "methodName": "~~trimmed~anonfun~~",
+ "fileName": "PlanGenerationTestSuite.scala"
+ }]
+ }
+ }
+ }
+ }, {
+ "literal": {
+ "string": "yyyy-MM-dd"
+ },
+ "common": {
+ "origin": {
+ "jvmOrigin": {
+ "stackTrace": [{
+ "classLoaderName": "app",
+ "declaringClass": "org.apache.spark.sql.functions$",
+ "methodName": "try_to_date",
+ "fileName": "functions.scala"
+ }, {
+ "classLoaderName": "app",
+ "declaringClass": "org.apache.spark.sql.PlanGenerationTestSuite",
+ "methodName": "~~trimmed~anonfun~~",
+ "fileName": "PlanGenerationTestSuite.scala"
+ }]
+ }
+ }
+ }
+ }],
+ "isInternal": false
+ },
+ "common": {
+ "origin": {
+ "jvmOrigin": {
+ "stackTrace": [{
+ "classLoaderName": "app",
+ "declaringClass": "org.apache.spark.sql.functions$",
+ "methodName": "try_to_date",
+ "fileName": "functions.scala"
+ }, {
+ "classLoaderName": "app",
+ "declaringClass": "org.apache.spark.sql.PlanGenerationTestSuite",
+ "methodName": "~~trimmed~anonfun~~",
+ "fileName": "PlanGenerationTestSuite.scala"
+ }]
+ }
+ }
+ }
+ }]
+ }
+}
\ No newline at end of file
diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_date_with_format.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_date_with_format.proto.bin
new file mode 100644
index 0000000000000..0c252b8c37a93
Binary files /dev/null and b/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_date_with_format.proto.bin differ
diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_typedLit.json b/sql/connect/common/src/test/resources/query-tests/queries/function_typedLit.json
index e56b6e1f3ee09..66bf31d670f9f 100644
--- a/sql/connect/common/src/test/resources/query-tests/queries/function_typedLit.json
+++ b/sql/connect/common/src/test/resources/query-tests/queries/function_typedLit.json
@@ -403,15 +403,17 @@
}, {
"literal": {
"array": {
- "elementType": {
- "integer": {
- }
- },
"elements": [{
"integer": 8
}, {
"integer": 6
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "integer": {
+ }
+ }
+ }
}
},
"common": {
@@ -627,6 +629,30 @@
}
}
}
+ }, {
+ "literal": {
+ "time": {
+ "nano": "86399999999999",
+ "precision": 6
+ }
+ },
+ "common": {
+ "origin": {
+ "jvmOrigin": {
+ "stackTrace": [{
+ "classLoaderName": "app",
+ "declaringClass": "org.apache.spark.sql.functions$",
+ "methodName": "typedLit",
+ "fileName": "functions.scala"
+ }, {
+ "classLoaderName": "app",
+ "declaringClass": "org.apache.spark.sql.PlanGenerationTestSuite",
+ "methodName": "~~trimmed~anonfun~~",
+ "fileName": "PlanGenerationTestSuite.scala"
+ }]
+ }
+ }
+ }
}, {
"literal": {
"calendarInterval": {
@@ -676,17 +702,19 @@
}, {
"literal": {
"array": {
- "elementType": {
- "integer": {
- }
- },
"elements": [{
"integer": 1
}, {
"integer": 2
}, {
"integer": 3
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "integer": {
+ }
+ }
+ }
}
},
"common": {
@@ -709,17 +737,19 @@
}, {
"literal": {
"array": {
- "elementType": {
- "integer": {
- }
- },
"elements": [{
"integer": 1
}, {
"integer": 2
}, {
"integer": 3
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "integer": {
+ }
+ }
+ }
}
},
"common": {
@@ -742,15 +772,6 @@
}, {
"literal": {
"map": {
- "keyType": {
- "string": {
- "collation": "UTF8_BINARY"
- }
- },
- "valueType": {
- "integer": {
- }
- },
"keys": [{
"string": "a"
}, {
@@ -760,7 +781,18 @@
"integer": 1
}, {
"integer": 2
- }]
+ }],
+ "dataType": {
+ "keyType": {
+ "string": {
+ "collation": "UTF8_BINARY"
+ }
+ },
+ "valueType": {
+ "integer": {
+ }
+ }
+ }
}
},
"common": {
@@ -783,38 +815,28 @@
}, {
"literal": {
"struct": {
- "structType": {
- "struct": {
- "fields": [{
- "name": "_1",
- "dataType": {
- "string": {
- "collation": "UTF8_BINARY"
- }
- },
- "nullable": true
- }, {
- "name": "_2",
- "dataType": {
- "integer": {
- }
- }
- }, {
- "name": "_3",
- "dataType": {
- "double": {
- }
- }
- }]
- }
- },
"elements": [{
"string": "a"
}, {
"integer": 2
}, {
"double": 1.0
- }]
+ }],
+ "dataTypeStruct": {
+ "fields": [{
+ "name": "_1",
+ "dataType": {
+ "string": {
+ "collation": "UTF8_BINARY"
+ }
+ },
+ "nullable": true
+ }, {
+ "name": "_2"
+ }, {
+ "name": "_3"
+ }]
+ }
}
},
"common": {
@@ -861,13 +883,16 @@
}, {
"literal": {
"array": {
- "elementType": {
- "integer": {
- }
- },
"elements": [{
"integer": 1
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "integer": {
+ }
+ },
+ "containsNull": true
+ }
}
},
"common": {
@@ -890,14 +915,6 @@
}, {
"literal": {
"map": {
- "keyType": {
- "integer": {
- }
- },
- "valueType": {
- "integer": {
- }
- },
"keys": [{
"integer": 1
}],
@@ -906,7 +923,18 @@
"integer": {
}
}
- }]
+ }],
+ "dataType": {
+ "keyType": {
+ "integer": {
+ }
+ },
+ "valueType": {
+ "integer": {
+ }
+ },
+ "valueContainsNull": true
+ }
}
},
"common": {
@@ -929,14 +957,6 @@
}, {
"literal": {
"map": {
- "keyType": {
- "integer": {
- }
- },
- "valueType": {
- "integer": {
- }
- },
"keys": [{
"integer": 1
}],
@@ -945,7 +965,18 @@
"integer": {
}
}
- }]
+ }],
+ "dataType": {
+ "keyType": {
+ "integer": {
+ }
+ },
+ "valueType": {
+ "integer": {
+ }
+ },
+ "valueContainsNull": true
+ }
}
},
"common": {
@@ -968,14 +999,6 @@
}, {
"literal": {
"map": {
- "keyType": {
- "integer": {
- }
- },
- "valueType": {
- "integer": {
- }
- },
"keys": [{
"integer": 1
}],
@@ -984,7 +1007,18 @@
"integer": {
}
}
- }]
+ }],
+ "dataType": {
+ "keyType": {
+ "integer": {
+ }
+ },
+ "valueType": {
+ "integer": {
+ }
+ },
+ "valueContainsNull": true
+ }
}
},
"common": {
@@ -1007,57 +1041,66 @@
}, {
"literal": {
"array": {
- "elementType": {
- "array": {
- "elementType": {
- "integer": {
- }
- }
- }
- },
"elements": [{
"array": {
- "elementType": {
- "integer": {
- }
- },
"elements": [{
"integer": 1
}, {
"integer": 2
}, {
"integer": 3
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "integer": {
+ }
+ }
+ }
}
}, {
"array": {
- "elementType": {
- "integer": {
- }
- },
"elements": [{
"integer": 4
}, {
"integer": 5
}, {
"integer": 6
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "integer": {
+ }
+ }
+ }
}
}, {
"array": {
- "elementType": {
- "integer": {
- }
- },
"elements": [{
"integer": 7
}, {
"integer": 8
}, {
"integer": 9
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "integer": {
+ }
+ }
+ }
}
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "array": {
+ "elementType": {
+ "integer": {
+ }
+ }
+ }
+ },
+ "containsNull": true
+ }
}
},
"common": {
@@ -1080,30 +1123,8 @@
}, {
"literal": {
"array": {
- "elementType": {
- "map": {
- "keyType": {
- "string": {
- "collation": "UTF8_BINARY"
- }
- },
- "valueType": {
- "integer": {
- }
- }
- }
- },
"elements": [{
"map": {
- "keyType": {
- "string": {
- "collation": "UTF8_BINARY"
- }
- },
- "valueType": {
- "integer": {
- }
- },
"keys": [{
"string": "a"
}, {
@@ -1113,19 +1134,21 @@
"integer": 1
}, {
"integer": 2
- }]
+ }],
+ "dataType": {
+ "keyType": {
+ "string": {
+ "collation": "UTF8_BINARY"
+ }
+ },
+ "valueType": {
+ "integer": {
+ }
+ }
+ }
}
}, {
"map": {
- "keyType": {
- "string": {
- "collation": "UTF8_BINARY"
- }
- },
- "valueType": {
- "integer": {
- }
- },
"keys": [{
"string": "a"
}, {
@@ -1135,19 +1158,21 @@
"integer": 3
}, {
"integer": 4
- }]
+ }],
+ "dataType": {
+ "keyType": {
+ "string": {
+ "collation": "UTF8_BINARY"
+ }
+ },
+ "valueType": {
+ "integer": {
+ }
+ }
+ }
}
}, {
"map": {
- "keyType": {
- "string": {
- "collation": "UTF8_BINARY"
- }
- },
- "valueType": {
- "integer": {
- }
- },
"keys": [{
"string": "a"
}, {
@@ -1157,9 +1182,36 @@
"integer": 5
}, {
"integer": 6
- }]
+ }],
+ "dataType": {
+ "keyType": {
+ "string": {
+ "collation": "UTF8_BINARY"
+ }
+ },
+ "valueType": {
+ "integer": {
+ }
+ }
+ }
}
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "map": {
+ "keyType": {
+ "string": {
+ "collation": "UTF8_BINARY"
+ }
+ },
+ "valueType": {
+ "integer": {
+ }
+ }
+ }
+ },
+ "containsNull": true
+ }
}
},
"common": {
@@ -1182,23 +1234,6 @@
}, {
"literal": {
"map": {
- "keyType": {
- "integer": {
- }
- },
- "valueType": {
- "map": {
- "keyType": {
- "string": {
- "collation": "UTF8_BINARY"
- }
- },
- "valueType": {
- "integer": {
- }
- }
- }
- },
"keys": [{
"integer": 1
}, {
@@ -1206,15 +1241,6 @@
}],
"values": [{
"map": {
- "keyType": {
- "string": {
- "collation": "UTF8_BINARY"
- }
- },
- "valueType": {
- "integer": {
- }
- },
"keys": [{
"string": "a"
}, {
@@ -1224,19 +1250,21 @@
"integer": 1
}, {
"integer": 2
- }]
+ }],
+ "dataType": {
+ "keyType": {
+ "string": {
+ "collation": "UTF8_BINARY"
+ }
+ },
+ "valueType": {
+ "integer": {
+ }
+ }
+ }
}
}, {
"map": {
- "keyType": {
- "string": {
- "collation": "UTF8_BINARY"
- }
- },
- "valueType": {
- "integer": {
- }
- },
"keys": [{
"string": "a"
}, {
@@ -1246,9 +1274,40 @@
"integer": 3
}, {
"integer": 4
- }]
+ }],
+ "dataType": {
+ "keyType": {
+ "string": {
+ "collation": "UTF8_BINARY"
+ }
+ },
+ "valueType": {
+ "integer": {
+ }
+ }
+ }
}
- }]
+ }],
+ "dataType": {
+ "keyType": {
+ "integer": {
+ }
+ },
+ "valueType": {
+ "map": {
+ "keyType": {
+ "string": {
+ "collation": "UTF8_BINARY"
+ }
+ },
+ "valueType": {
+ "integer": {
+ }
+ }
+ }
+ },
+ "valueContainsNull": true
+ }
}
},
"common": {
@@ -1271,96 +1330,24 @@
}, {
"literal": {
"struct": {
- "structType": {
- "struct": {
- "fields": [{
- "name": "_1",
- "dataType": {
- "array": {
- "elementType": {
- "integer": {
- }
- }
- }
- },
- "nullable": true
- }, {
- "name": "_2",
- "dataType": {
- "map": {
- "keyType": {
- "string": {
- "collation": "UTF8_BINARY"
- }
- },
- "valueType": {
- "integer": {
- }
- }
- }
- },
- "nullable": true
- }, {
- "name": "_3",
- "dataType": {
- "struct": {
- "fields": [{
- "name": "_1",
- "dataType": {
- "string": {
- "collation": "UTF8_BINARY"
- }
- },
- "nullable": true
- }, {
- "name": "_2",
- "dataType": {
- "map": {
- "keyType": {
- "integer": {
- }
- },
- "valueType": {
- "string": {
- "collation": "UTF8_BINARY"
- }
- },
- "valueContainsNull": true
- }
- },
- "nullable": true
- }]
- }
- },
- "nullable": true
- }]
- }
- },
"elements": [{
"array": {
- "elementType": {
- "integer": {
- }
- },
"elements": [{
"integer": 1
}, {
"integer": 2
}, {
"integer": 3
- }]
+ }],
+ "dataType": {
+ "elementType": {
+ "integer": {
+ }
+ }
+ }
}
}, {
"map": {
- "keyType": {
- "string": {
- "collation": "UTF8_BINARY"
- }
- },
- "valueType": {
- "integer": {
- }
- },
"keys": [{
"string": "a"
}, {
@@ -1370,53 +1357,25 @@
"integer": 1
}, {
"integer": 2
- }]
+ }],
+ "dataType": {
+ "keyType": {
+ "string": {
+ "collation": "UTF8_BINARY"
+ }
+ },
+ "valueType": {
+ "integer": {
+ }
+ }
+ }
}
}, {
"struct": {
- "structType": {
- "struct": {
- "fields": [{
- "name": "_1",
- "dataType": {
- "string": {
- "collation": "UTF8_BINARY"
- }
- },
- "nullable": true
- }, {
- "name": "_2",
- "dataType": {
- "map": {
- "keyType": {
- "integer": {
- }
- },
- "valueType": {
- "string": {
- "collation": "UTF8_BINARY"
- }
- },
- "valueContainsNull": true
- }
- },
- "nullable": true
- }]
- }
- },
"elements": [{
"string": "a"
}, {
"map": {
- "keyType": {
- "integer": {
- }
- },
- "valueType": {
- "string": {
- "collation": "UTF8_BINARY"
- }
- },
"keys": [{
"integer": 1
}, {
@@ -1426,11 +1385,49 @@
"string": "a"
}, {
"string": "b"
- }]
+ }],
+ "dataType": {
+ "keyType": {
+ "integer": {
+ }
+ },
+ "valueType": {
+ "string": {
+ "collation": "UTF8_BINARY"
+ }
+ },
+ "valueContainsNull": true
+ }
}
- }]
+ }],
+ "dataTypeStruct": {
+ "fields": [{
+ "name": "_1",
+ "dataType": {
+ "string": {
+ "collation": "UTF8_BINARY"
+ }
+ },
+ "nullable": true
+ }, {
+ "name": "_2",
+ "nullable": true
+ }]
+ }
}
- }]
+ }],
+ "dataTypeStruct": {
+ "fields": [{
+ "name": "_1",
+ "nullable": true
+ }, {
+ "name": "_2",
+ "nullable": true
+ }, {
+ "name": "_3",
+ "nullable": true
+ }]
+ }
}
},
"common": {
diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_typedLit.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_typedLit.proto.bin
index 38a6ce6300567..b3ebe8a79e3ec 100644
Binary files a/sql/connect/common/src/test/resources/query-tests/queries/function_typedLit.proto.bin and b/sql/connect/common/src/test/resources/query-tests/queries/function_typedLit.proto.bin differ
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala
index 5fe62295d1a5a..1887e4ede04db 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala
@@ -304,7 +304,7 @@ object Connect {
.version("4.0.0")
.internal()
.intConf
- .createWithDefault(16)
+ .createWithDefault(32)
val CONNECT_SESSION_PLAN_CACHE_ENABLED =
buildConf("spark.connect.session.planCache.enabled")
@@ -317,6 +317,17 @@ object Connect {
.booleanConf
.createWithDefault(true)
+ val CONNECT_ALWAYS_CACHE_DATA_SOURCE_READS_ENABLED =
+ buildConf("spark.connect.session.planCache.alwaysCacheDataSourceReadsEnabled")
+ .doc("When true, always cache the translation of Read.DataSource plans" +
+ " in the plan cache. This massively improves the performance of queries that reuse the" +
+ " same Read.DataSource within the same session, since these translations/analyses" +
+ " are usually quite costly.")
+ .version("4.1.0")
+ .internal()
+ .booleanConf
+ .createWithDefault(true)
+
val CONNECT_AUTHENTICATE_TOKEN =
buildStaticConf("spark.connect.authenticate.token")
.doc("A pre-shared token that will be used to authenticate clients. This secret must be" +
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala
index ff59789980ce7..632a2aecee050 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala
@@ -24,7 +24,7 @@ import io.grpc.stub.{ServerCallStreamObserver, StreamObserver}
import org.apache.spark.{SparkEnv, SparkSQLException}
import org.apache.spark.connect.proto.ExecutePlanResponse
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys._
import org.apache.spark.sql.catalyst.util.DateTimeConstants.NANOS_PER_MILLIS
import org.apache.spark.sql.connect.common.ProtoUtils
@@ -232,7 +232,8 @@ private[connect] class ExecuteGrpcResponseSender[T <: Message](
// 2. has a response to send
def gotResponse = response.nonEmpty
// 3. sent everything from the stream and the stream is finished
- def streamFinished = executionObserver.getLastResponseIndex().exists(nextIndex > _)
+ def streamFinished = executionObserver.getLastResponseIndex().exists(nextIndex > _) ||
+ executionObserver.isCleaned()
// 4. time deadline or size limit reached
def deadlineLimitReached =
sentResponsesSize > maximumResponseSize || deadlineTimeNs < System.nanoTime()
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteResponseObserver.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteResponseObserver.scala
index 9d0cc2128dd41..2473df0e53f1e 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteResponseObserver.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteResponseObserver.scala
@@ -26,7 +26,7 @@ import io.grpc.stub.StreamObserver
import org.apache.spark.{SparkEnv, SparkSQLException}
import org.apache.spark.connect.proto
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys
import org.apache.spark.sql.connect.config.Connect.CONNECT_EXECUTE_REATTACHABLE_OBSERVER_RETRY_BUFFER_SIZE
import org.apache.spark.sql.connect.service.ExecuteHolder
@@ -260,6 +260,11 @@ private[connect] class ExecuteResponseObserver[T <: Message](val executeHolder:
finalProducedIndex.isDefined
}
+ // Returns if this observer has already been cleaned
+ def isCleaned(): Boolean = responseLock.synchronized {
+ completed() && responses.isEmpty
+ }
+
// For testing.
private[connect] def undoCompletion(): Unit = responseLock.synchronized {
finalProducedIndex = None
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala
index 13857e066a8fa..7c4ad7df66fc8 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala
@@ -23,13 +23,12 @@ import scala.jdk.CollectionConverters._
import scala.util.control.NonFatal
import com.google.protobuf.Message
-import org.apache.commons.lang3.StringUtils
import org.apache.spark.SparkSQLException
import org.apache.spark.connect.proto
-import org.apache.spark.internal.{Logging, LogKeys, MDC}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.sql.connect.common.ProtoUtils
-import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.connect.planner.InvalidInputErrors
import org.apache.spark.sql.connect.service.{ExecuteHolder, ExecuteSessionTag, SparkConnectService}
import org.apache.spark.sql.connect.utils.ErrorUtils
import org.apache.spark.util.Utils
@@ -209,23 +208,23 @@ private[connect] class ExecuteThreadRunner(executeHolder: ExecuteHolder) extends
tag))
}
session.sparkContext.setJobDescription(
- s"Spark Connect - ${StringUtils.abbreviate(debugString, 128)}")
+ s"Spark Connect - ${Utils.abbreviate(debugString, 128)}")
session.sparkContext.setInterruptOnCancel(true)
// Add debug information to the query execution so that the jobs are traceable.
session.sparkContext.setLocalProperty(
"callSite.short",
- s"Spark Connect - ${StringUtils.abbreviate(debugString, 128)}")
- session.sparkContext.setLocalProperty(
- "callSite.long",
- StringUtils.abbreviate(debugString, 2048))
+ s"Spark Connect - ${Utils.abbreviate(debugString, 128)}")
+ session.sparkContext.setLocalProperty("callSite.long", Utils.abbreviate(debugString, 2048))
executeHolder.request.getPlan.getOpTypeCase match {
- case proto.Plan.OpTypeCase.COMMAND => handleCommand(executeHolder.request)
- case proto.Plan.OpTypeCase.ROOT => handlePlan(executeHolder.request)
- case _ =>
- throw new UnsupportedOperationException(
- s"${executeHolder.request.getPlan.getOpTypeCase} not supported.")
+ case proto.Plan.OpTypeCase.ROOT | proto.Plan.OpTypeCase.COMMAND =>
+ val execution = new SparkConnectPlanExecution(executeHolder)
+ execution.handlePlan(executeHolder.responseObserver)
+ case other =>
+ throw InvalidInputErrors.invalidOneOfField(
+ other,
+ executeHolder.request.getPlan.getDescriptorForType)
}
val observedMetrics: Map[String, Seq[(Option[String], Any)]] = {
@@ -307,21 +306,6 @@ private[connect] class ExecuteThreadRunner(executeHolder: ExecuteHolder) extends
proto.StreamingQueryListenerBusCommand.CommandCase.ADD_LISTENER_BUS_LISTENER
}
- private def handlePlan(request: proto.ExecutePlanRequest): Unit = {
- val responseObserver = executeHolder.responseObserver
-
- val execution = new SparkConnectPlanExecution(executeHolder)
- execution.handlePlan(responseObserver)
- }
-
- private def handleCommand(request: proto.ExecutePlanRequest): Unit = {
- val responseObserver = executeHolder.responseObserver
-
- val command = request.getPlan.getCommand
- val planner = new SparkConnectPlanner(executeHolder)
- planner.process(command = command, responseObserver = responseObserver)
- }
-
private def requestString(request: Message) = {
try {
Utils.redact(
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala
index 65b9863ca9543..388fd6a575db8 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala
@@ -32,10 +32,10 @@ import org.apache.spark.sql.classic.{DataFrame, Dataset}
import org.apache.spark.sql.connect.common.DataTypeProtoConverter
import org.apache.spark.sql.connect.common.LiteralValueProtoConverter.toLiteralProto
import org.apache.spark.sql.connect.config.Connect.CONNECT_GRPC_ARROW_MAX_BATCH_SIZE
-import org.apache.spark.sql.connect.planner.SparkConnectPlanner
+import org.apache.spark.sql.connect.planner.{InvalidInputErrors, SparkConnectPlanner}
import org.apache.spark.sql.connect.service.ExecuteHolder
import org.apache.spark.sql.connect.utils.MetricGenerator
-import org.apache.spark.sql.execution.{DoNotCleanup, LocalTableScanExec, RemoveShuffleFiles, SkipMigration, SQLExecution}
+import org.apache.spark.sql.execution.{DoNotCleanup, LocalTableScanExec, QueryExecution, RemoveShuffleFiles, SkipMigration, SQLExecution}
import org.apache.spark.sql.execution.arrow.ArrowConverters
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.StructType
@@ -53,34 +53,49 @@ private[execution] class SparkConnectPlanExecution(executeHolder: ExecuteHolder)
def handlePlan(responseObserver: ExecuteResponseObserver[proto.ExecutePlanResponse]): Unit = {
val request = executeHolder.request
- if (request.getPlan.getOpTypeCase != proto.Plan.OpTypeCase.ROOT) {
- throw new IllegalStateException(
- s"Illegal operation type ${request.getPlan.getOpTypeCase} to be handled here.")
- }
val planner = new SparkConnectPlanner(executeHolder)
val tracker = executeHolder.eventsManager.createQueryPlanningTracker()
val conf = session.sessionState.conf
val shuffleCleanupMode =
- if (conf.getConf(SQLConf.SHUFFLE_DEPENDENCY_FILE_CLEANUP_ENABLED)) {
+ if (conf.getConf(SQLConf.CONNECT_SHUFFLE_DEPENDENCY_FILE_CLEANUP_ENABLED)) {
RemoveShuffleFiles
} else if (conf.getConf(SQLConf.SHUFFLE_DEPENDENCY_SKIP_MIGRATION_ENABLED)) {
SkipMigration
} else {
DoNotCleanup
}
- val dataframe =
- Dataset.ofRows(
- sessionHolder.session,
- planner.transformRelation(request.getPlan.getRoot, cachePlan = true),
- tracker,
- shuffleCleanupMode)
- responseObserver.onNext(createSchemaResponse(request.getSessionId, dataframe.schema))
- processAsArrowBatches(dataframe, responseObserver, executeHolder)
- responseObserver.onNext(MetricGenerator.createMetricsResponse(sessionHolder, dataframe))
- createObservedMetricsResponse(
- request.getSessionId,
- executeHolder.allObservationAndPlanIds,
- dataframe).foreach(responseObserver.onNext)
+ request.getPlan.getOpTypeCase match {
+ case proto.Plan.OpTypeCase.ROOT =>
+ val dataframe =
+ Dataset.ofRows(
+ sessionHolder.session,
+ planner.transformRelation(request.getPlan.getRoot, cachePlan = true),
+ tracker,
+ shuffleCleanupMode)
+ responseObserver.onNext(createSchemaResponse(request.getSessionId, dataframe.schema))
+ processAsArrowBatches(dataframe, responseObserver, executeHolder)
+ responseObserver.onNext(MetricGenerator.createMetricsResponse(sessionHolder, dataframe))
+ createObservedMetricsResponse(
+ request.getSessionId,
+ executeHolder.allObservationAndPlanIds,
+ dataframe).foreach(responseObserver.onNext)
+ case proto.Plan.OpTypeCase.COMMAND =>
+ val command = request.getPlan.getCommand
+ planner.transformCommand(command) match {
+ case Some(transformer) =>
+ val qe = new QueryExecution(
+ session,
+ transformer(tracker),
+ tracker,
+ shuffleCleanupMode = shuffleCleanupMode)
+ qe.assertCommandExecuted()
+ executeHolder.eventsManager.postFinished()
+ case None =>
+ planner.process(command, responseObserver)
+ }
+ case other =>
+ throw InvalidInputErrors.invalidOneOfField(other, request.getPlan.getDescriptorForType)
+ }
}
type Batch = (Array[Byte], Long)
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLCache.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLCache.scala
index ef1b17dc2221e..7761c0078b27c 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLCache.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLCache.scala
@@ -25,14 +25,14 @@ import java.util.concurrent.atomic.AtomicLong
import scala.collection.mutable
import com.google.common.cache.{CacheBuilder, RemovalNotification}
-import org.apache.commons.io.FileUtils
import org.apache.spark.SparkException
import org.apache.spark.internal.Logging
import org.apache.spark.ml.Model
-import org.apache.spark.ml.util.{ConnectHelper, MLWritable, Summary}
+import org.apache.spark.ml.util.{ConnectHelper, HasTrainingSummary, MLWritable, Summary}
import org.apache.spark.sql.connect.config.Connect
import org.apache.spark.sql.connect.service.SessionHolder
+import org.apache.spark.util.SparkFileUtils
/**
* MLCache is for caching ML objects, typically for models and summaries evaluated by a model.
@@ -115,6 +115,12 @@ private[connect] class MLCache(sessionHolder: SessionHolder) extends Logging {
}
}
+ private[spark] def getModelOffloadingPath(refId: String): Path = {
+ val path = offloadedModelsDir.resolve(refId)
+ require(path.startsWith(offloadedModelsDir))
+ path
+ }
+
/**
* Cache an object into a map of MLCache, and return its key
* @param obj
@@ -122,7 +128,7 @@ private[connect] class MLCache(sessionHolder: SessionHolder) extends Logging {
* @return
* the key
*/
- def register(obj: Object): String = {
+ def register(obj: Object): String = this.synchronized {
val objectId = UUID.randomUUID().toString
if (obj.isInstanceOf[Summary]) {
@@ -137,9 +143,14 @@ private[connect] class MLCache(sessionHolder: SessionHolder) extends Logging {
}
cachedModel.put(objectId, CacheItem(obj, sizeBytes))
if (getMemoryControlEnabled) {
- val savePath = offloadedModelsDir.resolve(objectId)
- require(savePath.startsWith(offloadedModelsDir))
+ val savePath = getModelOffloadingPath(objectId)
obj.asInstanceOf[MLWritable].write.saveToLocal(savePath.toString)
+ if (obj.isInstanceOf[HasTrainingSummary[_]]
+ && obj.asInstanceOf[HasTrainingSummary[_]].hasSummary) {
+ obj
+ .asInstanceOf[HasTrainingSummary[_]]
+ .saveSummary(savePath.resolve("summary").toString)
+ }
Files.writeString(savePath.resolve(modelClassNameFile), obj.getClass.getName)
totalMLCacheInMemorySizeBytes.addAndGet(sizeBytes)
totalMLCacheSizeBytes.addAndGet(sizeBytes)
@@ -169,15 +180,14 @@ private[connect] class MLCache(sessionHolder: SessionHolder) extends Logging {
* @return
* the cached object
*/
- def get(refId: String): Object = {
+ def get(refId: String): Object = this.synchronized {
if (refId == helperID) {
helper
} else {
verifyObjectId(refId)
var obj: Object = Option(cachedModel.get(refId)).map(_.obj).getOrElse(null)
if (obj == null && getMemoryControlEnabled) {
- val loadPath = offloadedModelsDir.resolve(refId)
- require(loadPath.startsWith(offloadedModelsDir))
+ val loadPath = getModelOffloadingPath(refId)
if (Files.isDirectory(loadPath)) {
val className = Files.readString(loadPath.resolve(modelClassNameFile))
obj = MLUtils.loadTransformer(
@@ -194,17 +204,16 @@ private[connect] class MLCache(sessionHolder: SessionHolder) extends Logging {
}
}
- def _removeModel(refId: String): Boolean = {
+ def _removeModel(refId: String, evictOnly: Boolean): Boolean = {
verifyObjectId(refId)
val removedModel = cachedModel.remove(refId)
val removedFromMem = removedModel != null
- val removedFromDisk = if (removedModel != null && getMemoryControlEnabled) {
+ val removedFromDisk = if (!evictOnly && removedModel != null && getMemoryControlEnabled) {
totalMLCacheSizeBytes.addAndGet(-removedModel.sizeBytes)
- val removePath = offloadedModelsDir.resolve(refId)
- require(removePath.startsWith(offloadedModelsDir))
+ val removePath = getModelOffloadingPath(refId)
val offloadingPath = new File(removePath.toString)
if (offloadingPath.exists()) {
- FileUtils.deleteDirectory(offloadingPath)
+ SparkFileUtils.deleteRecursively(offloadingPath)
true
} else {
false
@@ -220,8 +229,8 @@ private[connect] class MLCache(sessionHolder: SessionHolder) extends Logging {
* @param refId
* the key used to look up the corresponding object
*/
- def remove(refId: String): Boolean = {
- val modelIsRemoved = _removeModel(refId)
+ def remove(refId: String, evictOnly: Boolean = false): Boolean = this.synchronized {
+ val modelIsRemoved = _removeModel(refId, evictOnly)
modelIsRemoved
}
@@ -229,16 +238,17 @@ private[connect] class MLCache(sessionHolder: SessionHolder) extends Logging {
/**
* Clear all the caches
*/
- def clear(): Int = {
+ def clear(): Int = this.synchronized {
val size = cachedModel.size()
cachedModel.clear()
+ totalMLCacheSizeBytes.set(0)
if (getMemoryControlEnabled) {
- FileUtils.cleanDirectory(new File(offloadedModelsDir.toString))
+ SparkFileUtils.cleanDirectory(new File(offloadedModelsDir.toString))
}
size
}
- def getInfo(): Array[String] = {
+ def getInfo(): Array[String] = this.synchronized {
val info = mutable.ArrayBuilder.make[String]
cachedModel.forEach { case (key, value) =>
info += s"id: $key, obj: ${value.obj.getClass}, size: ${value.sizeBytes}"
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLException.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLException.scala
index a017c719ed16e..847052be98a98 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLException.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLException.scala
@@ -51,3 +51,9 @@ private[spark] case class MLCacheSizeOverflowException(mlCacheMaxSize: Long)
errorClass = "CONNECT_ML.ML_CACHE_SIZE_OVERFLOW_EXCEPTION",
messageParameters = Map("mlCacheMaxSize" -> mlCacheMaxSize.toString),
cause = null)
+
+private[spark] case class MLModelSummaryLostException(objectName: String)
+ extends SparkException(
+ errorClass = "CONNECT_ML.MODEL_SUMMARY_LOST",
+ messageParameters = Map("objectName" -> objectName),
+ cause = null)
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLHandler.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLHandler.scala
index d40b70ba0813c..40f1172677a50 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLHandler.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLHandler.scala
@@ -17,11 +17,14 @@
package org.apache.spark.sql.connect.ml
+import java.io.{PrintWriter, StringWriter}
import java.lang.ThreadLocal
+import java.util.concurrent.ConcurrentHashMap
import scala.collection.mutable
import scala.jdk.CollectionConverters.CollectionHasAsScala
+import org.apache.spark.SparkException
import org.apache.spark.connect.proto
import org.apache.spark.internal.Logging
import org.apache.spark.ml.{Estimator, EstimatorUtils, Model, Transformer}
@@ -121,6 +124,9 @@ private[connect] object MLHandler extends Logging {
override def initialValue: SessionHolder = null
}
+ // A map of thread-id -> handler execution start time (UNIX timestamp)
+ val handlerExecutionStartTimeMap = new ConcurrentHashMap[Long, Long]()
+
private val allowlistedMLClasses = {
val transformerClasses = MLUtils.loadOperators(classOf[Transformer])
val estimatorClasses = MLUtils.loadOperators(classOf[Estimator[_]])
@@ -150,7 +156,43 @@ private[connect] object MLHandler extends Logging {
}
}
- def handleMlCommand(
+ def startHangingHandlerReaper(): Unit = {
+ val handlerInterruptionTimeoutMinutes = {
+ try {
+ val envValue = System.getenv("SPARK_CONNECT_ML_HANDLER_INTERRUPTION_TIMEOUT_MINUTES")
+ if (envValue != null) {
+ envValue.toInt
+ } else { 0 }
+ } catch {
+ case _: Exception => 0
+ }
+ }
+
+ if (handlerInterruptionTimeoutMinutes > 0) {
+ val handlerInterruptionTimeoutMillis = handlerInterruptionTimeoutMinutes * 60 * 1000
+ val thread = new Thread(() => {
+ while (true) {
+ handlerExecutionStartTimeMap.forEach { (threadId, startTime) =>
+ val execTime = System.currentTimeMillis() - startTime
+ if (execTime > handlerInterruptionTimeoutMillis) {
+ for (t <- Thread.getAllStackTraces().keySet().asScala) {
+ if (t.getId() == threadId) {
+ t.interrupt()
+ }
+ }
+ }
+ }
+ Thread.sleep(60 * 1000)
+ }
+ })
+ thread.setDaemon(true)
+ thread.start()
+ }
+ }
+
+ startHangingHandlerReaper()
+
+ def _handleMlCommand(
sessionHolder: SessionHolder,
mlCommand: proto.MlCommand): proto.MlCommandResult = {
@@ -229,9 +271,7 @@ private[connect] object MLHandler extends Logging {
if (obj != null && obj.isInstanceOf[HasTrainingSummary[_]]
&& methods(0).getMethod == "summary"
&& !obj.asInstanceOf[HasTrainingSummary[_]].hasSummary) {
- throw MLCacheInvalidException(
- objRefId,
- sessionHolder.mlCache.getOffloadingTimeoutMinute)
+ throw MLModelSummaryLostException(objRefId)
}
val helper = AttributeHelper(sessionHolder, objRefId, methods)
val attrResult = helper.getAttribute
@@ -264,9 +304,13 @@ private[connect] object MLHandler extends Logging {
case proto.MlCommand.CommandCase.DELETE =>
val ids = mutable.ArrayBuilder.make[String]
- mlCommand.getDelete.getObjRefsList.asScala.toArray.foreach { objId =>
+ val deleteCmd = mlCommand.getDelete
+ val evictOnly = if (deleteCmd.hasEvictOnly) {
+ deleteCmd.getEvictOnly
+ } else { false }
+ deleteCmd.getObjRefsList.asScala.toArray.foreach { objId =>
if (!objId.getId.contains(".")) {
- if (mlCache.remove(objId.getId)) {
+ if (mlCache.remove(objId.getId, evictOnly)) {
ids += objId.getId
}
}
@@ -400,11 +444,80 @@ private[connect] object MLHandler extends Logging {
.setParam(LiteralValueProtoConverter.toLiteralProto(metric))
.build()
+ case proto.MlCommand.CommandCase.CREATE_SUMMARY =>
+ val createSummaryCmd = mlCommand.getCreateSummary
+ createModelSummary(sessionHolder, createSummaryCmd)
+
+ case proto.MlCommand.CommandCase.GET_MODEL_SIZE =>
+ val modelRefId = mlCommand.getGetModelSize.getModelRef.getId
+ val model = mlCache.get(modelRefId)
+ val modelSize = model.asInstanceOf[Model[_]].estimatedSize
+ proto.MlCommandResult
+ .newBuilder()
+ .setParam(LiteralValueProtoConverter.toLiteralProto(modelSize))
+ .build()
+
case other => throw MlUnsupportedException(s"$other not supported")
}
}
- def transformMLRelation(relation: proto.MlRelation, sessionHolder: SessionHolder): DataFrame = {
+ def wrapHandler(
+ originHandler: () => Any,
+ reqProto: com.google.protobuf.GeneratedMessage): Any = {
+ val threadId = Thread.currentThread().getId
+ val startTime = System.currentTimeMillis()
+ handlerExecutionStartTimeMap.put(threadId, startTime)
+ try {
+ originHandler()
+ } catch {
+ case e: InterruptedException =>
+ val stackTrace = {
+ val sw = new StringWriter()
+ val pw = new PrintWriter(sw)
+ e.printStackTrace(pw)
+ sw.toString
+ }
+ val execTime = (System.currentTimeMillis() - startTime) / (60 * 1000)
+ throw SparkException.internalError(
+ s"The Spark Connect ML handler thread is interrupted after executing for " +
+ s"$execTime minutes.\nThe request proto message is:\n${reqProto.toString}\n, " +
+ s"the current stack trace is:\n$stackTrace\n")
+ } finally {
+ handlerExecutionStartTimeMap.remove(threadId)
+ }
+ }
+
+ def handleMlCommand(
+ sessionHolder: SessionHolder,
+ mlCommand: proto.MlCommand): proto.MlCommandResult = {
+ wrapHandler(() => _handleMlCommand(sessionHolder, mlCommand), mlCommand)
+ .asInstanceOf[proto.MlCommandResult]
+ }
+
+ private def createModelSummary(
+ sessionHolder: SessionHolder,
+ createSummaryCmd: proto.MlCommand.CreateSummary): proto.MlCommandResult =
+ sessionHolder.mlCache.synchronized {
+ val refId = createSummaryCmd.getModelRef.getId
+ val model = sessionHolder.mlCache.get(refId).asInstanceOf[HasTrainingSummary[_]]
+ val isCreated = if (!model.hasSummary) {
+ val dataset = MLUtils.parseRelationProto(createSummaryCmd.getDataset, sessionHolder)
+ val modelPath = sessionHolder.mlCache.getModelOffloadingPath(refId)
+ val summaryPath = modelPath.resolve("summary").toString
+ model.loadSummary(summaryPath, dataset)
+ true
+ } else {
+ false
+ }
+ proto.MlCommandResult
+ .newBuilder()
+ .setParam(LiteralValueProtoConverter.toLiteralProto(isCreated))
+ .build()
+ }
+
+ def _transformMLRelation(
+ relation: proto.MlRelation,
+ sessionHolder: SessionHolder): DataFrame = {
relation.getMlTypeCase match {
// Ml transform
case proto.MlRelation.MlTypeCase.TRANSFORM =>
@@ -433,13 +546,36 @@ private[connect] object MLHandler extends Logging {
// Get the attribute from a cached object which could be a model or summary
case proto.MlRelation.MlTypeCase.FETCH =>
- val helper = AttributeHelper(
- sessionHolder,
- relation.getFetch.getObjRef.getId,
- relation.getFetch.getMethodsList.asScala.toArray)
+ val objRefId = relation.getFetch.getObjRef.getId
+ val methods = relation.getFetch.getMethodsList.asScala.toArray
+ val obj = sessionHolder.mlCache.get(objRefId)
+ sessionHolder.mlCache.synchronized {
+ if (obj != null && obj.isInstanceOf[HasTrainingSummary[_]]
+ && methods(0).getMethod == "summary"
+ && !obj.asInstanceOf[HasTrainingSummary[_]].hasSummary) {
+
+ if (relation.hasModelSummaryDataset) {
+ val dataset =
+ MLUtils.parseRelationProto(relation.getModelSummaryDataset, sessionHolder)
+ val modelPath = sessionHolder.mlCache.getModelOffloadingPath(objRefId)
+ val summaryPath = modelPath.resolve("summary").toString
+ obj.asInstanceOf[HasTrainingSummary[_]].loadSummary(summaryPath, dataset)
+ } else {
+ // For old Spark client backward compatibility.
+ throw MLModelSummaryLostException(objRefId)
+ }
+ }
+ }
+
+ val helper = AttributeHelper(sessionHolder, objRefId, methods)
helper.getAttribute.asInstanceOf[DataFrame]
case other => throw MlUnsupportedException(s"$other not supported")
}
}
+
+ def transformMLRelation(relation: proto.MlRelation, sessionHolder: SessionHolder): DataFrame = {
+ wrapHandler(() => _transformMLRelation(relation, sessionHolder), relation)
+ .asInstanceOf[DataFrame]
+ }
}
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLUtils.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLUtils.scala
index 0ab9105637291..b06ddaadc4da5 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLUtils.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLUtils.scala
@@ -40,7 +40,7 @@ import org.apache.spark.ml.tree.{DecisionTreeModel, TreeEnsembleModel}
import org.apache.spark.ml.util.{ConnectHelper, HasTrainingSummary, Identifiable, MLReader, MLWritable}
import org.apache.spark.sql.DataFrame
import org.apache.spark.sql.classic.Dataset
-import org.apache.spark.sql.connect.common.LiteralValueProtoConverter
+import org.apache.spark.sql.connect.common.{LiteralValueProtoConverter, ProtoSpecializedArray}
import org.apache.spark.sql.connect.planner.SparkConnectPlanner
import org.apache.spark.sql.connect.plugin.SparkConnectPluginRegistry
import org.apache.spark.sql.connect.service.SessionHolder
@@ -73,39 +73,19 @@ private[ml] object MLUtils {
.toMap
}
- private def parseInts(ints: proto.Ints): Array[Int] = {
- val size = ints.getValuesCount
- val values = Array.ofDim[Int](size)
- var i = 0
- while (i < size) {
- values(i) = ints.getValues(i)
- i += 1
- }
- values
- }
-
- private def parseDoubles(doubles: proto.Doubles): Array[Double] = {
- val size = doubles.getValuesCount
- val values = Array.ofDim[Double](size)
- var i = 0
- while (i < size) {
- values(i) = doubles.getValues(i)
- i += 1
- }
- values
- }
-
def deserializeVector(s: proto.Expression.Literal.Struct): Vector = {
assert(s.getElementsCount == 4)
s.getElements(0).getByte match {
case 0 =>
val size = s.getElements(1).getInteger
- val indices = parseInts(s.getElements(2).getSpecializedArray.getInts)
- val values = parseDoubles(s.getElements(3).getSpecializedArray.getDoubles)
+ val indices = ProtoSpecializedArray.toArray(s.getElements(2).getSpecializedArray.getInts)
+ val values =
+ ProtoSpecializedArray.toArray(s.getElements(3).getSpecializedArray.getDoubles)
Vectors.sparse(size, indices, values)
case 1 =>
- val values = parseDoubles(s.getElements(3).getSpecializedArray.getDoubles)
+ val values =
+ ProtoSpecializedArray.toArray(s.getElements(3).getSpecializedArray.getDoubles)
Vectors.dense(values)
case o => throw MlUnsupportedException(s"Unknown Vector type $o")
@@ -118,16 +98,19 @@ private[ml] object MLUtils {
case 0 =>
val numRows = s.getElements(1).getInteger
val numCols = s.getElements(2).getInteger
- val colPtrs = parseInts(s.getElements(3).getSpecializedArray.getInts)
- val rowIndices = parseInts(s.getElements(4).getSpecializedArray.getInts)
- val values = parseDoubles(s.getElements(5).getSpecializedArray.getDoubles)
+ val colPtrs = ProtoSpecializedArray.toArray(s.getElements(3).getSpecializedArray.getInts)
+ val rowIndices =
+ ProtoSpecializedArray.toArray(s.getElements(4).getSpecializedArray.getInts)
+ val values =
+ ProtoSpecializedArray.toArray(s.getElements(5).getSpecializedArray.getDoubles)
val isTransposed = s.getElements(6).getBoolean
new SparseMatrix(numRows, numCols, colPtrs, rowIndices, values, isTransposed)
case 1 =>
val numRows = s.getElements(1).getInteger
val numCols = s.getElements(2).getInteger
- val values = parseDoubles(s.getElements(5).getSpecializedArray.getDoubles)
+ val values =
+ ProtoSpecializedArray.toArray(s.getElements(5).getSpecializedArray.getDoubles)
val isTransposed = s.getElements(6).getBoolean
new DenseMatrix(numRows, numCols, values, isTransposed)
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/DataflowGraphRegistry.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/DataflowGraphRegistry.scala
index 4402dde04f3c8..e0c7beb43001d 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/DataflowGraphRegistry.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/DataflowGraphRegistry.scala
@@ -28,10 +28,7 @@ import org.apache.spark.sql.pipelines.graph.GraphRegistrationContext
* PipelinesHandler when CreateDataflowGraph is called, and the PipelinesHandler also supports
* attaching flows/datasets to a graph.
*/
-// TODO(SPARK-51727): Currently DataflowGraphRegistry is a singleton, but it should instead be
-// scoped to a single SparkSession for proper isolation between pipelines that are run on the
-// same cluster.
-object DataflowGraphRegistry {
+class DataflowGraphRegistry {
private val dataflowGraphs = new ConcurrentHashMap[String, GraphRegistrationContext]()
@@ -55,7 +52,7 @@ object DataflowGraphRegistry {
/** Retrieves the graph for a given id, and throws if the id could not be found. */
def getDataflowGraphOrThrow(dataflowGraphId: String): GraphRegistrationContext =
- DataflowGraphRegistry.getDataflowGraph(dataflowGraphId).getOrElse {
+ getDataflowGraph(dataflowGraphId).getOrElse {
throw new SparkException(
errorClass = "DATAFLOW_GRAPH_NOT_FOUND",
messageParameters = Map("graphId" -> dataflowGraphId),
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/PipelinesHandler.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/PipelinesHandler.scala
index 92cb5bcac4ee0..b595ba2c501ed 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/PipelinesHandler.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/PipelinesHandler.scala
@@ -26,14 +26,13 @@ import org.apache.spark.connect.proto
import org.apache.spark.connect.proto.{ExecutePlanResponse, PipelineCommandResult, Relation}
import org.apache.spark.internal.Logging
import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.classic.SparkSession
import org.apache.spark.sql.connect.common.DataTypeProtoConverter
import org.apache.spark.sql.connect.service.SessionHolder
import org.apache.spark.sql.pipelines.Language.Python
-import org.apache.spark.sql.pipelines.QueryOriginType
import org.apache.spark.sql.pipelines.common.RunState.{CANCELED, FAILED}
-import org.apache.spark.sql.pipelines.graph.{FlowAnalysis, GraphIdentifierManager, IdentifierHelper, PipelineUpdateContextImpl, QueryContext, QueryOrigin, SqlGraphRegistrationContext, Table, TemporaryView, UnresolvedFlow}
+import org.apache.spark.sql.pipelines.graph.{AllTables, FlowAnalysis, GraphIdentifierManager, GraphRegistrationContext, IdentifierHelper, NoTables, PipelineUpdateContextImpl, QueryContext, QueryOrigin, QueryOriginType, SomeTables, SqlGraphRegistrationContext, Table, TableFilter, TemporaryView, UnresolvedFlow}
import org.apache.spark.sql.pipelines.logging.{PipelineEvent, RunProgress}
import org.apache.spark.sql.types.StructType
@@ -67,7 +66,7 @@ private[connect] object PipelinesHandler extends Logging {
cmd.getCommandTypeCase match {
case proto.PipelineCommand.CommandTypeCase.CREATE_DATAFLOW_GRAPH =>
val createdGraphId =
- createDataflowGraph(cmd.getCreateDataflowGraph, sessionHolder.session)
+ createDataflowGraph(cmd.getCreateDataflowGraph, sessionHolder)
PipelineCommandResult
.newBuilder()
.setCreateDataflowGraphResult(
@@ -77,15 +76,16 @@ private[connect] object PipelinesHandler extends Logging {
.build()
case proto.PipelineCommand.CommandTypeCase.DROP_DATAFLOW_GRAPH =>
logInfo(s"Drop pipeline cmd received: $cmd")
- DataflowGraphRegistry.dropDataflowGraph(cmd.getDropDataflowGraph.getDataflowGraphId)
+ sessionHolder.dataflowGraphRegistry
+ .dropDataflowGraph(cmd.getDropDataflowGraph.getDataflowGraphId)
defaultResponse
case proto.PipelineCommand.CommandTypeCase.DEFINE_DATASET =>
logInfo(s"Define pipelines dataset cmd received: $cmd")
- defineDataset(cmd.getDefineDataset, sessionHolder.session)
+ defineDataset(cmd.getDefineDataset, sessionHolder)
defaultResponse
case proto.PipelineCommand.CommandTypeCase.DEFINE_FLOW =>
logInfo(s"Define pipelines flow cmd received: $cmd")
- defineFlow(cmd.getDefineFlow, transformRelationFunc, sessionHolder.session)
+ defineFlow(cmd.getDefineFlow, transformRelationFunc, sessionHolder)
defaultResponse
case proto.PipelineCommand.CommandTypeCase.START_RUN =>
logInfo(s"Start pipeline cmd received: $cmd")
@@ -93,7 +93,7 @@ private[connect] object PipelinesHandler extends Logging {
defaultResponse
case proto.PipelineCommand.CommandTypeCase.DEFINE_SQL_GRAPH_ELEMENTS =>
logInfo(s"Register sql datasets cmd received: $cmd")
- defineSqlGraphElements(cmd.getDefineSqlGraphElements, sessionHolder.session)
+ defineSqlGraphElements(cmd.getDefineSqlGraphElements, sessionHolder)
defaultResponse
case other => throw new UnsupportedOperationException(s"$other not supported")
}
@@ -101,24 +101,24 @@ private[connect] object PipelinesHandler extends Logging {
private def createDataflowGraph(
cmd: proto.PipelineCommand.CreateDataflowGraph,
- spark: SparkSession): String = {
+ sessionHolder: SessionHolder): String = {
val defaultCatalog = Option
.when(cmd.hasDefaultCatalog)(cmd.getDefaultCatalog)
.getOrElse {
logInfo(s"No default catalog was supplied. Falling back to the current catalog.")
- spark.catalog.currentCatalog()
+ sessionHolder.session.catalog.currentCatalog()
}
val defaultDatabase = Option
.when(cmd.hasDefaultDatabase)(cmd.getDefaultDatabase)
.getOrElse {
logInfo(s"No default database was supplied. Falling back to the current database.")
- spark.catalog.currentDatabase
+ sessionHolder.session.catalog.currentDatabase
}
val defaultSqlConf = cmd.getSqlConfMap.asScala.toMap
- DataflowGraphRegistry.createDataflowGraph(
+ sessionHolder.dataflowGraphRegistry.createDataflowGraph(
defaultCatalog = defaultCatalog,
defaultDatabase = defaultDatabase,
defaultSqlConf = defaultSqlConf)
@@ -126,24 +126,31 @@ private[connect] object PipelinesHandler extends Logging {
private def defineSqlGraphElements(
cmd: proto.PipelineCommand.DefineSqlGraphElements,
- session: SparkSession): Unit = {
+ sessionHolder: SessionHolder): Unit = {
val dataflowGraphId = cmd.getDataflowGraphId
- val graphElementRegistry = DataflowGraphRegistry.getDataflowGraphOrThrow(dataflowGraphId)
+ val graphElementRegistry =
+ sessionHolder.dataflowGraphRegistry.getDataflowGraphOrThrow(dataflowGraphId)
val sqlGraphElementRegistrationContext = new SqlGraphRegistrationContext(graphElementRegistry)
- sqlGraphElementRegistrationContext.processSqlFile(cmd.getSqlText, cmd.getSqlFilePath, session)
+ sqlGraphElementRegistrationContext.processSqlFile(
+ cmd.getSqlText,
+ cmd.getSqlFilePath,
+ sessionHolder.session)
}
private def defineDataset(
dataset: proto.PipelineCommand.DefineDataset,
- sparkSession: SparkSession): Unit = {
+ sessionHolder: SessionHolder): Unit = {
val dataflowGraphId = dataset.getDataflowGraphId
- val graphElementRegistry = DataflowGraphRegistry.getDataflowGraphOrThrow(dataflowGraphId)
+ val graphElementRegistry =
+ sessionHolder.dataflowGraphRegistry.getDataflowGraphOrThrow(dataflowGraphId)
dataset.getDatasetType match {
case proto.DatasetType.MATERIALIZED_VIEW | proto.DatasetType.TABLE =>
val tableIdentifier =
- GraphIdentifierManager.parseTableIdentifier(dataset.getDatasetName, sparkSession)
+ GraphIdentifierManager.parseTableIdentifier(
+ dataset.getDatasetName,
+ sessionHolder.session)
graphElementRegistry.registerTable(
Table(
identifier = tableIdentifier,
@@ -161,10 +168,12 @@ private[connect] object PipelinesHandler extends Logging {
language = Option(Python())),
format = Option.when(dataset.hasFormat)(dataset.getFormat),
normalizedPath = None,
- isStreamingTableOpt = None))
+ isStreamingTable = dataset.getDatasetType == proto.DatasetType.TABLE))
case proto.DatasetType.TEMPORARY_VIEW =>
val viewIdentifier =
- GraphIdentifierManager.parseTableIdentifier(dataset.getDatasetName, sparkSession)
+ GraphIdentifierManager.parseTableIdentifier(
+ dataset.getDatasetName,
+ sessionHolder.session)
graphElementRegistry.registerView(
TemporaryView(
@@ -183,14 +192,15 @@ private[connect] object PipelinesHandler extends Logging {
private def defineFlow(
flow: proto.PipelineCommand.DefineFlow,
transformRelationFunc: Relation => LogicalPlan,
- sparkSession: SparkSession): Unit = {
+ sessionHolder: SessionHolder): Unit = {
val dataflowGraphId = flow.getDataflowGraphId
- val graphElementRegistry = DataflowGraphRegistry.getDataflowGraphOrThrow(dataflowGraphId)
+ val graphElementRegistry =
+ sessionHolder.dataflowGraphRegistry.getDataflowGraphOrThrow(dataflowGraphId)
val isImplicitFlow = flow.getFlowName == flow.getTargetDatasetName
val flowIdentifier = GraphIdentifierManager
- .parseTableIdentifier(name = flow.getFlowName, spark = sparkSession)
+ .parseTableIdentifier(name = flow.getFlowName, spark = sessionHolder.session)
// If the flow is not an implicit flow (i.e. one defined as part of dataset creation), then
// it must be a single-part identifier.
@@ -204,15 +214,14 @@ private[connect] object PipelinesHandler extends Logging {
new UnresolvedFlow(
identifier = flowIdentifier,
destinationIdentifier = GraphIdentifierManager
- .parseTableIdentifier(name = flow.getTargetDatasetName, spark = sparkSession),
+ .parseTableIdentifier(name = flow.getTargetDatasetName, spark = sessionHolder.session),
func =
- FlowAnalysis.createFlowFunctionFromLogicalPlan(transformRelationFunc(flow.getPlan)),
+ FlowAnalysis.createFlowFunctionFromLogicalPlan(transformRelationFunc(flow.getRelation)),
sqlConf = flow.getSqlConfMap.asScala.toMap,
- once = flow.getOnce,
+ once = false,
queryContext = QueryContext(
Option(graphElementRegistry.defaultCatalog),
Option(graphElementRegistry.defaultDatabase)),
- comment = None,
origin = QueryOrigin(
objectType = Option(QueryOriginType.Flow.toString),
objectName = Option(flowIdentifier.unquotedString),
@@ -224,9 +233,12 @@ private[connect] object PipelinesHandler extends Logging {
responseObserver: StreamObserver[ExecutePlanResponse],
sessionHolder: SessionHolder): Unit = {
val dataflowGraphId = cmd.getDataflowGraphId
- val graphElementRegistry = DataflowGraphRegistry.getDataflowGraphOrThrow(dataflowGraphId)
+ val graphElementRegistry =
+ sessionHolder.dataflowGraphRegistry.getDataflowGraphOrThrow(dataflowGraphId)
+ val tableFiltersResult = createTableFilters(cmd, graphElementRegistry, sessionHolder)
+
// We will use this variable to store the run failure event if it occurs. This will be set
- // by the event callback that is executed when an event is added to the PipelineRunEventBuffer.
+ // by the event callback.
@volatile var runFailureEvent = Option.empty[PipelineEvent]
// Define a callback which will stream logs back to the SparkConnect client when an internal
// pipeline event is emitted during pipeline execution. We choose to pass a callback rather the
@@ -280,10 +292,17 @@ private[connect] object PipelinesHandler extends Logging {
.build())
}
}
- val pipelineUpdateContext =
- new PipelineUpdateContextImpl(graphElementRegistry.toDataflowGraph, eventCallback)
+ val pipelineUpdateContext = new PipelineUpdateContextImpl(
+ graphElementRegistry.toDataflowGraph,
+ eventCallback,
+ tableFiltersResult.refresh,
+ tableFiltersResult.fullRefresh)
sessionHolder.cachePipelineExecution(dataflowGraphId, pipelineUpdateContext)
- pipelineUpdateContext.pipelineExecution.runPipeline()
+ if (cmd.getDry) {
+ pipelineUpdateContext.pipelineExecution.dryRunPipeline()
+ } else {
+ pipelineUpdateContext.pipelineExecution.runPipeline()
+ }
// Rethrow any exceptions that caused the pipeline run to fail so that the exception is
// propagated back to the SC client / CLI.
@@ -291,4 +310,87 @@ private[connect] object PipelinesHandler extends Logging {
throw event.error.get
}
}
+
+ /**
+ * Creates the table filters for the full refresh and refresh operations based on the StartRun
+ * command user provided. Also validates the command parameters to ensure that they are
+ * consistent and do not conflict with each other.
+ *
+ * If `fullRefreshAll` is true, create `AllTables` filter for full refresh.
+ *
+ * If `fullRefreshTables` and `refreshTables` are both empty, create `AllTables` filter for
+ * refresh as a default behavior.
+ *
+ * If both non-empty, verifies that there is no overlap and creates SomeTables filters for both.
+ *
+ * If one non-empty and the other empty, create `SomeTables` filter for the non-empty one, and
+ * `NoTables` filter for the empty one.
+ */
+ private def createTableFilters(
+ startRunCommand: proto.PipelineCommand.StartRun,
+ graphElementRegistry: GraphRegistrationContext,
+ sessionHolder: SessionHolder): TableFilters = {
+ // Convert table names to fully qualified TableIdentifier objects
+ def parseTableNames(tableNames: Seq[String]): Set[TableIdentifier] = {
+ tableNames.map { name =>
+ GraphIdentifierManager
+ .parseAndQualifyTableIdentifier(
+ rawTableIdentifier =
+ GraphIdentifierManager.parseTableIdentifier(name, sessionHolder.session),
+ currentCatalog = Some(graphElementRegistry.defaultCatalog),
+ currentDatabase = Some(graphElementRegistry.defaultDatabase))
+ .identifier
+ }.toSet
+ }
+
+ val fullRefreshTables = startRunCommand.getFullRefreshSelectionList.asScala.toSeq
+ val fullRefreshAll = startRunCommand.getFullRefreshAll
+ val refreshTables = startRunCommand.getRefreshSelectionList.asScala.toSeq
+
+ if (refreshTables.nonEmpty && fullRefreshAll) {
+ throw new IllegalArgumentException(
+ "Cannot specify a subset to refresh when full refresh all is set to true.")
+ }
+
+ if (fullRefreshTables.nonEmpty && fullRefreshAll) {
+ throw new IllegalArgumentException(
+ "Cannot specify a subset to full refresh when full refresh all is set to true.")
+ }
+ val refreshTableNames = parseTableNames(refreshTables)
+ val fullRefreshTableNames = parseTableNames(fullRefreshTables)
+
+ if (refreshTables.nonEmpty && fullRefreshTables.nonEmpty) {
+ // check if there is an intersection between the subset
+ val intersection = refreshTableNames.intersect(fullRefreshTableNames)
+ if (intersection.nonEmpty) {
+ throw new IllegalArgumentException(
+ "Datasets specified for refresh and full refresh cannot overlap: " +
+ s"${intersection.mkString(", ")}")
+ }
+ }
+
+ if (fullRefreshAll) {
+ return TableFilters(fullRefresh = AllTables, refresh = NoTables)
+ }
+
+ (fullRefreshTables, refreshTables) match {
+ case (Nil, Nil) =>
+ // If both are empty, we default to refreshing all tables
+ TableFilters(fullRefresh = NoTables, refresh = AllTables)
+ case (_, Nil) =>
+ TableFilters(fullRefresh = SomeTables(fullRefreshTableNames), refresh = NoTables)
+ case (Nil, _) =>
+ TableFilters(fullRefresh = NoTables, refresh = SomeTables(refreshTableNames))
+ case (_, _) =>
+ // If both are specified, we create filters for both after validation
+ TableFilters(
+ fullRefresh = SomeTables(fullRefreshTableNames),
+ refresh = SomeTables(refreshTableNames))
+ }
+ }
+
+ /**
+ * A case class to hold the table filters for full refresh and refresh operations.
+ */
+ private case class TableFilters(fullRefresh: TableFilter, refresh: TableFilter)
}
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/LiteralExpressionProtoConverter.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/LiteralExpressionProtoConverter.scala
index 50f413399befd..e1d30240f867e 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/LiteralExpressionProtoConverter.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/LiteralExpressionProtoConverter.scala
@@ -97,21 +97,35 @@ object LiteralExpressionProtoConverter {
case proto.Expression.Literal.LiteralTypeCase.DAY_TIME_INTERVAL =>
expressions.Literal(lit.getDayTimeInterval, DayTimeIntervalType())
+ case proto.Expression.Literal.LiteralTypeCase.TIME =>
+ var precision = TimeType.DEFAULT_PRECISION
+ if (lit.getTime.hasPrecision) {
+ precision = lit.getTime.getPrecision
+ }
+ expressions.Literal(lit.getTime.getNano, TimeType(precision))
+
case proto.Expression.Literal.LiteralTypeCase.ARRAY =>
- expressions.Literal.create(
- LiteralValueProtoConverter.toCatalystArray(lit.getArray),
- ArrayType(DataTypeProtoConverter.toCatalystType(lit.getArray.getElementType)))
+ val arrayData = LiteralValueProtoConverter.toCatalystArray(lit.getArray)
+ val dataType = DataTypeProtoConverter.toCatalystType(
+ proto.DataType.newBuilder
+ .setArray(LiteralValueProtoConverter.getProtoArrayType(lit.getArray))
+ .build())
+ expressions.Literal.create(arrayData, dataType)
case proto.Expression.Literal.LiteralTypeCase.MAP =>
- expressions.Literal.create(
- LiteralValueProtoConverter.toCatalystMap(lit.getMap),
- MapType(
- DataTypeProtoConverter.toCatalystType(lit.getMap.getKeyType),
- DataTypeProtoConverter.toCatalystType(lit.getMap.getValueType)))
+ val mapData = LiteralValueProtoConverter.toCatalystMap(lit.getMap)
+ val dataType = DataTypeProtoConverter.toCatalystType(
+ proto.DataType.newBuilder
+ .setMap(LiteralValueProtoConverter.getProtoMapType(lit.getMap))
+ .build())
+ expressions.Literal.create(mapData, dataType)
case proto.Expression.Literal.LiteralTypeCase.STRUCT =>
- val dataType = DataTypeProtoConverter.toCatalystType(lit.getStruct.getStructType)
val structData = LiteralValueProtoConverter.toCatalystStruct(lit.getStruct)
+ val dataType = DataTypeProtoConverter.toCatalystType(
+ proto.DataType.newBuilder
+ .setStruct(LiteralValueProtoConverter.getProtoStructType(lit.getStruct))
+ .build())
val convert = CatalystTypeConverters.createToCatalystConverter(dataType)
expressions.Literal(convert(structData), dataType)
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala
index bcd643a30253f..c0b1fd01616a4 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala
@@ -17,8 +17,7 @@
package org.apache.spark.sql.connect.planner
-import java.util.Properties
-import java.util.UUID
+import java.util.{HashMap, Properties, UUID}
import scala.collection.mutable
import scala.jdk.CollectionConverters._
@@ -26,11 +25,10 @@ import scala.util.Try
import scala.util.control.NonFatal
import com.google.common.base.Throwables
-import com.google.common.collect.{Lists, Maps}
+import com.google.common.collect.Lists
import com.google.protobuf.{Any => ProtoAny, ByteString}
import io.grpc.{Context, Status, StatusRuntimeException}
import io.grpc.stub.StreamObserver
-import org.apache.commons.lang3.exception.ExceptionUtils
import org.apache.spark.{SparkClassNotFoundException, SparkEnv, SparkException, TaskContext}
import org.apache.spark.annotation.{DeveloperApi, Since}
@@ -41,7 +39,7 @@ import org.apache.spark.connect.proto.ExecutePlanResponse.SqlCommandResult
import org.apache.spark.connect.proto.Parse.ParseFormat
import org.apache.spark.connect.proto.StreamingQueryManagerCommandResult.StreamingQueryInstance
import org.apache.spark.connect.proto.WriteStreamOperationStart.TriggerCase
-import org.apache.spark.internal.{Logging, LogKeys, MDC}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.internal.LogKeys.{DATAFRAME_ID, SESSION_ID}
import org.apache.spark.resource.{ExecutorResourceRequest, ResourceProfile, TaskResourceProfile, TaskResourceRequest}
import org.apache.spark.sql.{Column, Encoders, ForeachWriter, Observation, Row}
@@ -59,7 +57,7 @@ import org.apache.spark.sql.catalyst.streaming.InternalOutputModes
import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, Origin, TreePattern}
import org.apache.spark.sql.catalyst.types.DataTypeUtils
import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, CharVarcharUtils}
-import org.apache.spark.sql.classic.{Catalog, Dataset, MergeIntoWriter, RelationalGroupedDataset, SparkSession, TypedAggUtils, UserDefinedFunctionUtils}
+import org.apache.spark.sql.classic.{Catalog, DataFrameWriter, Dataset, MergeIntoWriter, RelationalGroupedDataset, SparkSession, TypedAggUtils, UserDefinedFunctionUtils}
import org.apache.spark.sql.classic.ClassicConversions._
import org.apache.spark.sql.connect.client.arrow.ArrowSerializer
import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, ForeachWriterPacket, LiteralValueProtoConverter, StorageLevelProtoConverter, StreamingListenerPacket, UdfPacket}
@@ -79,8 +77,8 @@ import org.apache.spark.sql.execution.datasources.v2.python.UserDefinedPythonDat
import org.apache.spark.sql.execution.python.{UserDefinedPythonFunction, UserDefinedPythonTableFunction}
import org.apache.spark.sql.execution.python.streaming.PythonForeachWriter
import org.apache.spark.sql.execution.stat.StatFunctions
-import org.apache.spark.sql.execution.streaming.GroupStateImpl.groupStateTimeoutFromString
-import org.apache.spark.sql.execution.streaming.StreamingQueryWrapper
+import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.GroupStateImpl.groupStateTimeoutFromString
+import org.apache.spark.sql.execution.streaming.runtime.StreamingQueryWrapper
import org.apache.spark.sql.expressions.{Aggregator, ReduceAggregator, SparkUserDefinedFunction, UserDefinedAggregator, UserDefinedFunction}
import org.apache.spark.sql.streaming.{GroupStateTimeout, OutputMode, StatefulProcessor, StatefulProcessorWithInitialState, StreamingQuery, StreamingQueryListener, StreamingQueryProgress, Trigger}
import org.apache.spark.sql.types._
@@ -1191,7 +1189,7 @@ class SparkConnectPlanner(
SimplePythonFunction(
command = fun.getCommand.toByteArray.toImmutableArraySeq,
// Empty environment variables
- envVars = Maps.newHashMap(),
+ envVars = new HashMap[String, String](),
pythonIncludes = sessionHolder.artifactManager.getPythonIncludes.asJava,
pythonExec = pythonExec,
pythonVer = fun.getPythonVer,
@@ -1205,7 +1203,7 @@ class SparkConnectPlanner(
SimplePythonFunction(
command = ds.getCommand.toByteArray.toImmutableArraySeq,
// Empty environment variables
- envVars = Maps.newHashMap(),
+ envVars = new HashMap[String, String](),
pythonIncludes = sessionHolder.artifactManager.getPythonIncludes.asJava,
pythonExec = pythonExec,
pythonVer = ds.getPythonVer,
@@ -1443,9 +1441,8 @@ class SparkConnectPlanner(
}
if (rel.hasData) {
- val (rows, structType) = ArrowConverters.fromBatchWithSchemaIterator(
- Iterator(rel.getData.toByteArray),
- TaskContext.get())
+ val (rows, structType) =
+ ArrowConverters.fromIPCStream(rel.getData.toByteArray, TaskContext.get())
if (structType == null) {
throw InvalidInputErrors.inputDataForLocalRelationNoSchema()
}
@@ -2034,7 +2031,7 @@ class SparkConnectPlanner(
SimplePythonFunction(
command = fun.getCommand.toByteArray.toImmutableArraySeq,
// Empty environment variables
- envVars = Maps.newHashMap(),
+ envVars = new HashMap[String, String](),
pythonExec = pythonExec,
// Merge the user specified includes with the includes managed by the artifact manager.
pythonIncludes = (fun.getAdditionalIncludesList.asScala.toSeq ++
@@ -2646,9 +2643,29 @@ class SparkConnectPlanner(
process(command, new MockObserver())
}
+ def transformCommand(command: proto.Command): Option[QueryPlanningTracker => LogicalPlan] = {
+ command.getCommandTypeCase match {
+ case proto.Command.CommandTypeCase.WRITE_OPERATION =>
+ Some(transformWriteOperation(command.getWriteOperation))
+ case proto.Command.CommandTypeCase.WRITE_OPERATION_V2 =>
+ Some(transformWriteOperationV2(command.getWriteOperationV2))
+ case proto.Command.CommandTypeCase.MERGE_INTO_TABLE_COMMAND =>
+ Some(transformMergeIntoTableCommand(command.getMergeIntoTableCommand))
+ case proto.Command.CommandTypeCase.CREATE_DATAFRAME_VIEW =>
+ Some(_ => transformCreateViewCommand(command.getCreateDataframeView))
+ case _ =>
+ None
+ }
+ }
+
def process(
command: proto.Command,
responseObserver: StreamObserver[ExecutePlanResponse]): Unit = {
+ val transformerOpt = transformCommand(command)
+ if (transformerOpt.isDefined) {
+ transformAndRunCommand(transformerOpt.get)
+ return
+ }
command.getCommandTypeCase match {
case proto.Command.CommandTypeCase.REGISTER_FUNCTION =>
handleRegisterUserDefinedFunction(command.getRegisterFunction)
@@ -2656,12 +2673,6 @@ class SparkConnectPlanner(
handleRegisterUserDefinedTableFunction(command.getRegisterTableFunction)
case proto.Command.CommandTypeCase.REGISTER_DATA_SOURCE =>
handleRegisterUserDefinedDataSource(command.getRegisterDataSource)
- case proto.Command.CommandTypeCase.WRITE_OPERATION =>
- handleWriteOperation(command.getWriteOperation)
- case proto.Command.CommandTypeCase.CREATE_DATAFRAME_VIEW =>
- handleCreateViewCommand(command.getCreateDataframeView)
- case proto.Command.CommandTypeCase.WRITE_OPERATION_V2 =>
- handleWriteOperationV2(command.getWriteOperationV2)
case proto.Command.CommandTypeCase.EXTENSION =>
handleCommandPlugin(command.getExtension)
case proto.Command.CommandTypeCase.SQL_COMMAND =>
@@ -2689,8 +2700,6 @@ class SparkConnectPlanner(
handleCheckpointCommand(command.getCheckpointCommand, responseObserver)
case proto.Command.CommandTypeCase.REMOVE_CACHED_REMOTE_RELATION_COMMAND =>
handleRemoveCachedRemoteRelationCommand(command.getRemoveCachedRemoteRelationCommand)
- case proto.Command.CommandTypeCase.MERGE_INTO_TABLE_COMMAND =>
- handleMergeIntoTableCommand(command.getMergeIntoTableCommand)
case proto.Command.CommandTypeCase.ML_COMMAND =>
handleMlCommand(command.getMlCommand, responseObserver)
case proto.Command.CommandTypeCase.PIPELINE_COMMAND =>
@@ -3050,7 +3059,8 @@ class SparkConnectPlanner(
executeHolder.eventsManager.postFinished()
}
- private def handleCreateViewCommand(createView: proto.CreateDataFrameViewCommand): Unit = {
+ private def transformCreateViewCommand(
+ createView: proto.CreateDataFrameViewCommand): LogicalPlan = {
val viewType = if (createView.getIsGlobal) GlobalTempView else LocalTempView
val tableIdentifier =
@@ -3061,7 +3071,7 @@ class SparkConnectPlanner(
throw QueryCompilationErrors.invalidViewNameError(createView.getName)
}
- val plan = CreateViewCommand(
+ CreateViewCommand(
name = tableIdentifier,
userSpecifiedColumns = Nil,
comment = None,
@@ -3072,14 +3082,10 @@ class SparkConnectPlanner(
allowExisting = false,
replace = createView.getReplace,
viewType = viewType)
-
- val tracker = executeHolder.eventsManager.createQueryPlanningTracker()
- Dataset.ofRows(session, plan, tracker).queryExecution.commandExecuted
- executeHolder.eventsManager.postFinished()
}
/**
- * Transforms the write operation and executes it.
+ * Transforms the write operation.
*
* The input write operation contains a reference to the input plan and transforms it to the
* corresponding logical plan. Afterwards, creates the DataFrameWriter and translates the
@@ -3087,14 +3093,14 @@ class SparkConnectPlanner(
*
* @param writeOperation
*/
- private def handleWriteOperation(writeOperation: proto.WriteOperation): Unit = {
+ private def transformWriteOperation(writeOperation: proto.WriteOperation)(
+ tracker: QueryPlanningTracker): LogicalPlan = {
// Transform the input plan into the logical plan.
val plan = transformRelation(writeOperation.getInput)
// And create a Dataset from the plan.
- val tracker = executeHolder.eventsManager.createQueryPlanningTracker()
val dataset = Dataset.ofRows(session, plan, tracker)
- val w = dataset.write
+ val w = dataset.write.asInstanceOf[DataFrameWriter[_]]
if (writeOperation.getMode != proto.WriteOperation.SaveMode.SAVE_MODE_UNSPECIFIED) {
w.mode(SaveModeConverter.toSaveMode(writeOperation.getMode))
}
@@ -3129,25 +3135,32 @@ class SparkConnectPlanner(
}
writeOperation.getSaveTypeCase match {
- case proto.WriteOperation.SaveTypeCase.SAVETYPE_NOT_SET => w.save()
- case proto.WriteOperation.SaveTypeCase.PATH => w.save(writeOperation.getPath)
+ case proto.WriteOperation.SaveTypeCase.SAVETYPE_NOT_SET => w.saveCommand(None)
+ case proto.WriteOperation.SaveTypeCase.PATH =>
+ w.saveCommand(Some(writeOperation.getPath))
case proto.WriteOperation.SaveTypeCase.TABLE =>
val tableName = writeOperation.getTable.getTableName
writeOperation.getTable.getSaveMethod match {
case proto.WriteOperation.SaveTable.TableSaveMethod.TABLE_SAVE_METHOD_SAVE_AS_TABLE =>
- w.saveAsTable(tableName)
+ w.saveAsTableCommand(tableName)
case proto.WriteOperation.SaveTable.TableSaveMethod.TABLE_SAVE_METHOD_INSERT_INTO =>
- w.insertInto(tableName)
+ w.insertIntoCommand(tableName)
case other => throw InvalidInputErrors.invalidEnum(other)
}
case other =>
throw InvalidInputErrors.invalidOneOfField(other, writeOperation.getDescriptorForType)
}
+ }
+
+ private def transformAndRunCommand(transformer: QueryPlanningTracker => LogicalPlan): Unit = {
+ val tracker = executeHolder.eventsManager.createQueryPlanningTracker()
+ val qe = new QueryExecution(session, transformer(tracker), tracker)
+ qe.assertCommandExecuted()
executeHolder.eventsManager.postFinished()
}
/**
- * Transforms the write operation and executes it.
+ * Transforms the write operation.
*
* The input write operation contains a reference to the input plan and transforms it to the
* corresponding logical plan. Afterwards, creates the DataFrameWriter and translates the
@@ -3155,11 +3168,11 @@ class SparkConnectPlanner(
*
* @param writeOperation
*/
- private def handleWriteOperationV2(writeOperation: proto.WriteOperationV2): Unit = {
+ private def transformWriteOperationV2(writeOperation: proto.WriteOperationV2)(
+ tracker: QueryPlanningTracker): LogicalPlan = {
// Transform the input plan into the logical plan.
val plan = transformRelation(writeOperation.getInput)
// And create a Dataset from the plan.
- val tracker = executeHolder.eventsManager.createQueryPlanningTracker()
val dataset = Dataset.ofRows(session, plan, tracker)
val w = dataset.writeTo(table = writeOperation.getTableName)
@@ -3190,32 +3203,28 @@ class SparkConnectPlanner(
writeOperation.getMode match {
case proto.WriteOperationV2.Mode.MODE_CREATE =>
if (writeOperation.hasProvider) {
- w.using(writeOperation.getProvider).create()
- } else {
- w.create()
+ w.using(writeOperation.getProvider)
}
+ w.createCommand()
case proto.WriteOperationV2.Mode.MODE_OVERWRITE =>
- w.overwrite(Column(transformExpression(writeOperation.getOverwriteCondition)))
+ w.overwriteCommand(Column(transformExpression(writeOperation.getOverwriteCondition)))
case proto.WriteOperationV2.Mode.MODE_OVERWRITE_PARTITIONS =>
- w.overwritePartitions()
+ w.overwritePartitionsCommand()
case proto.WriteOperationV2.Mode.MODE_APPEND =>
- w.append()
+ w.appendCommand()
case proto.WriteOperationV2.Mode.MODE_REPLACE =>
if (writeOperation.hasProvider) {
- w.using(writeOperation.getProvider).replace()
- } else {
- w.replace()
+ w.using(writeOperation.getProvider)
}
+ w.replaceCommand(orCreate = false)
case proto.WriteOperationV2.Mode.MODE_CREATE_OR_REPLACE =>
if (writeOperation.hasProvider) {
- w.using(writeOperation.getProvider).createOrReplace()
- } else {
- w.createOrReplace()
+ w.using(writeOperation.getProvider)
}
+ w.replaceCommand(orCreate = true)
case other =>
throw InvalidInputErrors.invalidEnum(other)
}
- executeHolder.eventsManager.postFinished()
}
private def handleWriteStreamOperationStart(
@@ -3472,7 +3481,7 @@ class SparkConnectPlanner(
.setExceptionMessage(e.toString())
.setErrorClass(e.getCondition)
- val stackTrace = Option(ExceptionUtils.getStackTrace(e))
+ val stackTrace = Option(Utils.stackTraceToString(e))
stackTrace.foreach { s =>
exception_builder.setStackTrace(s)
}
@@ -3745,7 +3754,8 @@ class SparkConnectPlanner(
executeHolder.eventsManager.postFinished()
}
- private def handleMergeIntoTableCommand(cmd: proto.MergeIntoTableCommand): Unit = {
+ private def transformMergeIntoTableCommand(cmd: proto.MergeIntoTableCommand)(
+ tracker: QueryPlanningTracker): LogicalPlan = {
def transformActions(actions: java.util.List[proto.Expression]): Seq[MergeAction] =
actions.asScala.map(transformExpression).map(_.asInstanceOf[MergeAction]).toSeq
@@ -3753,7 +3763,7 @@ class SparkConnectPlanner(
val notMatchedActions = transformActions(cmd.getNotMatchedActionsList)
val notMatchedBySourceActions = transformActions(cmd.getNotMatchedBySourceActionsList)
- val sourceDs = Dataset.ofRows(session, transformRelation(cmd.getSourceTablePlan))
+ val sourceDs = Dataset.ofRows(session, transformRelation(cmd.getSourceTablePlan), tracker)
val mergeInto = sourceDs
.mergeInto(cmd.getTargetTableName, Column(transformExpression(cmd.getMergeCondition)))
.asInstanceOf[MergeIntoWriter[Row]]
@@ -3763,8 +3773,7 @@ class SparkConnectPlanner(
if (cmd.getWithSchemaEvolution) {
mergeInto.withSchemaEvolution()
}
- mergeInto.merge()
- executeHolder.eventsManager.postFinished()
+ mergeInto.mergeCommand()
}
private val emptyLocalRelation = LocalRelation(
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala
index 70f18a1a9c1a0..04312a35a3b4b 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala
@@ -24,7 +24,7 @@ import io.grpc.stub.StreamObserver
import org.apache.spark.connect.proto.ExecutePlanResponse
import org.apache.spark.connect.proto.StreamingQueryListenerBusCommand
import org.apache.spark.connect.proto.StreamingQueryListenerEventsResult
-import org.apache.spark.internal.{Logging, LogKeys, MDC}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.sql.connect.service.ExecuteHolder
/**
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingForeachBatchHelper.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingForeachBatchHelper.scala
index 5a44607a4c7d9..a4da5ea998386 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingForeachBatchHelper.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingForeachBatchHelper.scala
@@ -26,7 +26,7 @@ import scala.util.control.NonFatal
import org.apache.spark.SparkException
import org.apache.spark.api.python.{PythonException, PythonWorkerUtils, SimplePythonFunction, SpecialLengths, StreamingPythonRunner}
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{DATAFRAME_ID, PYTHON_EXEC, QUERY_ID, RUN_ID_STRING, SESSION_ID, USER_ID}
import org.apache.spark.sql.{DataFrame, Dataset}
import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, AgnosticEncoders}
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingQueryListenerHelper.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingQueryListenerHelper.scala
index 42c090d43f065..f994ada920ec2 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingQueryListenerHelper.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingQueryListenerHelper.scala
@@ -21,7 +21,7 @@ import java.io.EOFException
import org.apache.spark.SparkException
import org.apache.spark.api.python.{PythonException, PythonWorkerUtils, SimplePythonFunction, SpecialLengths, StreamingPythonRunner}
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.FUNCTION_NAME
import org.apache.spark.sql.connect.config.Connect
import org.apache.spark.sql.connect.service.{SessionHolder, SparkConnectService}
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/LoggingInterceptor.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/LoggingInterceptor.scala
index a071579692fb1..695034954e981 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/LoggingInterceptor.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/LoggingInterceptor.scala
@@ -28,7 +28,7 @@ import io.grpc.ServerCall
import io.grpc.ServerCallHandler
import io.grpc.ServerInterceptor
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{DESCRIPTION, MESSAGE}
/**
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala
index ada322fd859c5..be59439daefbb 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala
@@ -31,13 +31,14 @@ import com.google.common.cache.{Cache, CacheBuilder}
import org.apache.spark.{SparkEnv, SparkException, SparkSQLException}
import org.apache.spark.api.python.PythonFunction.PythonAccumulator
import org.apache.spark.connect.proto
-import org.apache.spark.internal.{Logging, LogKeys, MDC}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.sql.DataFrame
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.classic.SparkSession
import org.apache.spark.sql.connect.common.InvalidPlanInput
import org.apache.spark.sql.connect.config.Connect
import org.apache.spark.sql.connect.ml.MLCache
+import org.apache.spark.sql.connect.pipelines.DataflowGraphRegistry
import org.apache.spark.sql.connect.planner.PythonStreamingQueryListener
import org.apache.spark.sql.connect.planner.StreamingForeachBatchHelper
import org.apache.spark.sql.connect.service.SessionHolder.{ERROR_CACHE_SIZE, ERROR_CACHE_TIMEOUT_SEC}
@@ -125,6 +126,9 @@ case class SessionHolder(userId: String, sessionId: String, session: SparkSessio
private lazy val pipelineExecutions =
new ConcurrentHashMap[String, PipelineUpdateContext]()
+ // Registry for dataflow graphs specific to this session
+ private[connect] lazy val dataflowGraphRegistry = new DataflowGraphRegistry()
+
// Handles Python process clean up for streaming queries. Initialized on first use in a query.
private[connect] lazy val streamingForeachBatchRunnerCleanerCache =
new StreamingForeachBatchHelper.CleanerCache(this)
@@ -320,6 +324,9 @@ case class SessionHolder(userId: String, sessionId: String, session: SparkSessio
// Stops all pipeline execution and clears the pipeline execution cache
removeAllPipelineExecutions()
+ // Clean up dataflow graphs
+ dataflowGraphRegistry.dropAllDataflowGraphs()
+
// if there is a server side listener, clean up related resources
if (streamingServersideListenerHolder.isServerSideListenerRegistered) {
streamingServersideListenerHolder.cleanUp()
@@ -514,6 +521,11 @@ case class SessionHolder(userId: String, sessionId: String, session: SparkSessio
// We only cache plans that have a plan ID.
val hasPlanId = rel.hasCommon && rel.getCommon.hasPlanId
+ // Always cache a `Read.DataSource` to avoid re-analyzing the same `DataSource` twice.
+ lazy val alwaysCacheDataSourceReadsEnabled = Option(session)
+ .forall(_.conf.get(Connect.CONNECT_ALWAYS_CACHE_DATA_SOURCE_READS_ENABLED, true))
+ lazy val isDataSourceRead = rel.hasRead && rel.getRead.hasDataSource
+
def getPlanCache(rel: proto.Relation): Option[LogicalPlan] =
planCache match {
case Some(cache) if planCacheEnabled && hasPlanId =>
@@ -535,7 +547,7 @@ case class SessionHolder(userId: String, sessionId: String, session: SparkSessio
getPlanCache(rel)
.getOrElse({
val plan = transform(rel)
- if (cachePlan) {
+ if (cachePlan || (alwaysCacheDataSourceReadsEnabled && isDataSourceRead)) {
putPlanCache(rel, plan)
}
plan
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAddArtifactsHandler.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAddArtifactsHandler.scala
index 3ba79402e99ef..becd7d855133d 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAddArtifactsHandler.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAddArtifactsHandler.scala
@@ -26,6 +26,7 @@ import scala.util.control.NonFatal
import com.google.common.io.CountingOutputStream
import io.grpc.stub.StreamObserver
+import org.apache.spark.SparkRuntimeException
import org.apache.spark.connect.proto
import org.apache.spark.connect.proto.{AddArtifactsRequest, AddArtifactsResponse}
import org.apache.spark.connect.proto.AddArtifactsResponse.ArtifactSummary
@@ -112,19 +113,32 @@ class SparkConnectAddArtifactsHandler(val responseObserver: StreamObserver[AddAr
* @return
*/
protected def flushStagedArtifacts(): Seq[ArtifactSummary] = {
+ val failedArtifactExceptions = mutable.ListBuffer[SparkRuntimeException]()
+
// Non-lazy transformation when using Buffer.
- stagedArtifacts.map { artifact =>
- // We do not store artifacts that fail the CRC. The failure is reported in the artifact
- // summary and it is up to the client to decide whether to retry sending the artifact.
- if (artifact.getCrcStatus.contains(true)) {
- if (artifact.path.startsWith(ArtifactManager.forwardToFSPrefix + File.separator)) {
- holder.artifactManager.uploadArtifactToFs(artifact.path, artifact.stagedPath)
- } else {
- addStagedArtifactToArtifactManager(artifact)
+ val summaries = stagedArtifacts.map { artifact =>
+ try {
+ // We do not store artifacts that fail the CRC. The failure is reported in the artifact
+ // summary and it is up to the client to decide whether to retry sending the artifact.
+ if (artifact.getCrcStatus.contains(true)) {
+ if (artifact.path.startsWith(ArtifactManager.forwardToFSPrefix + File.separator)) {
+ holder.artifactManager.uploadArtifactToFs(artifact.path, artifact.stagedPath)
+ } else {
+ addStagedArtifactToArtifactManager(artifact)
+ }
}
+ } catch {
+ case e: SparkRuntimeException if e.getCondition == "ARTIFACT_ALREADY_EXISTS" =>
+ failedArtifactExceptions += e
}
artifact.summary()
}.toSeq
+
+ if (failedArtifactExceptions.nonEmpty) {
+ throw ArtifactUtils.mergeExceptionsWithSuppressed(failedArtifactExceptions.toSeq)
+ }
+
+ summaries
}
protected def cleanUpStagedArtifacts(): Unit = Utils.deleteRecursively(stagingDir.toFile)
@@ -216,6 +230,7 @@ class SparkConnectAddArtifactsHandler(val responseObserver: StreamObserver[AddAr
private val fileOut = Files.newOutputStream(stagedPath)
private val countingOut = new CountingOutputStream(fileOut)
private val checksumOut = new CheckedOutputStream(countingOut, new CRC32)
+ private val overallChecksum = new CRC32()
private val builder = ArtifactSummary.newBuilder().setName(name)
private var artifactSummary: ArtifactSummary = _
@@ -227,6 +242,8 @@ class SparkConnectAddArtifactsHandler(val responseObserver: StreamObserver[AddAr
def getCrcStatus: Option[Boolean] = Option(isCrcSuccess)
+ def getCrc: Long = overallChecksum.getValue
+
def write(dataChunk: proto.AddArtifactsRequest.ArtifactChunk): Unit = {
try dataChunk.getData.writeTo(checksumOut)
catch {
@@ -234,6 +251,8 @@ class SparkConnectAddArtifactsHandler(val responseObserver: StreamObserver[AddAr
close()
throw e
}
+
+ overallChecksum.update(dataChunk.getData.toByteArray)
updateCrc(checksumOut.getChecksum.getValue == dataChunk.getCrc)
checksumOut.getChecksum.reset()
}
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectExecutionManager.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectExecutionManager.scala
index c8f4a1bc977f8..35c4073fe93c9 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectExecutionManager.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectExecutionManager.scala
@@ -30,7 +30,7 @@ import io.grpc.stub.StreamObserver
import org.apache.spark.{SparkEnv, SparkSQLException}
import org.apache.spark.connect.proto
-import org.apache.spark.internal.{Logging, LogKeys, MDC}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.sql.catalyst.util.DateTimeConstants.NANOS_PER_MILLIS
import org.apache.spark.sql.connect.config.Connect.{CONNECT_EXECUTE_MANAGER_ABANDONED_TOMBSTONES_SIZE, CONNECT_EXECUTE_MANAGER_DETACHED_TIMEOUT, CONNECT_EXECUTE_MANAGER_MAINTENANCE_INTERVAL}
import org.apache.spark.sql.connect.execution.ExecuteGrpcResponseSender
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectListenerBusListener.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectListenerBusListener.scala
index 8fbcf3218a003..91fe395f520d8 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectListenerBusListener.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectListenerBusListener.scala
@@ -29,7 +29,7 @@ import org.apache.spark.connect.proto.ExecutePlanResponse
import org.apache.spark.connect.proto.StreamingQueryEventType
import org.apache.spark.connect.proto.StreamingQueryListenerEvent
import org.apache.spark.connect.proto.StreamingQueryListenerEventsResult
-import org.apache.spark.internal.{Logging, LogKeys, MDC}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.sql.connect.execution.ExecuteResponseObserver
import org.apache.spark.sql.streaming.StreamingQueryListener
import org.apache.spark.util.ArrayImplicits._
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectServer.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectServer.scala
index b2c4d1abb17b4..5b1034a4a27b7 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectServer.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectServer.scala
@@ -17,7 +17,7 @@
package org.apache.spark.sql.connect.service
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{HOST, PORT}
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.internal.SQLConf
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala
index 262cc954f8f9d..cc1cb95b66c46 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala
@@ -29,13 +29,12 @@ import io.grpc.netty.NettyServerBuilder
import io.grpc.protobuf.ProtoUtils
import io.grpc.protobuf.services.ProtoReflectionService
import io.grpc.stub.StreamObserver
-import org.apache.commons.lang3.StringUtils
import org.apache.spark.{SparkContext, SparkEnv}
import org.apache.spark.connect.proto
import org.apache.spark.connect.proto.{AddArtifactsRequest, AddArtifactsResponse, SparkConnectServiceGrpc}
import org.apache.spark.connect.proto.SparkConnectServiceGrpc.AsyncService
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.HOST
import org.apache.spark.internal.config.UI.UI_ENABLED
import org.apache.spark.scheduler.{LiveListenerBus, SparkListenerEvent}
@@ -504,7 +503,7 @@ object SparkConnectService extends Logging {
}
def extractErrorMessage(st: Throwable): String = {
- val message = StringUtils.abbreviate(st.getMessage, 2048)
+ val message = Utils.abbreviate(st.getMessage, 2048)
convertNullString(message)
}
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManager.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManager.scala
index 572d760187e9d..1c3cfd67f132c 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManager.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManager.scala
@@ -28,7 +28,7 @@ import scala.util.control.NonFatal
import com.google.common.cache.CacheBuilder
import org.apache.spark.{SparkEnv, SparkSQLException}
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{INTERVAL, SESSION_HOLD_INFO}
import org.apache.spark.sql.classic.SparkSession
import org.apache.spark.sql.connect.config.Connect.{CONNECT_SESSION_MANAGER_CLOSED_SESSIONS_TOMBSTONES_SIZE, CONNECT_SESSION_MANAGER_DEFAULT_SESSION_TIMEOUT, CONNECT_SESSION_MANAGER_MAINTENANCE_INTERVAL}
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectStreamingQueryCache.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectStreamingQueryCache.scala
index beff193f6701f..8ff13f5afe191 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectStreamingQueryCache.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectStreamingQueryCache.scala
@@ -25,7 +25,7 @@ import scala.concurrent.{ExecutionContext, Future}
import scala.concurrent.duration.{Duration, DurationInt, FiniteDuration}
import scala.util.control.NonFatal
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{DURATION, NEW_VALUE, OLD_VALUE, QUERY_CACHE_VALUE, QUERY_ID, QUERY_RUN_ID, SESSION_ID}
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.streaming.StreamingQuery
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ui/SparkConnectServerListener.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ui/SparkConnectServerListener.scala
index 3a93bbae3f2b8..98dccc6c9a6c8 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ui/SparkConnectServerListener.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ui/SparkConnectServerListener.scala
@@ -23,7 +23,7 @@ import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer
import org.apache.spark.{SparkConf, SparkContext, SparkEnv}
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{OP_ID, SESSION_ID}
import org.apache.spark.internal.config.Status.LIVE_ENTITY_UPDATE_PERIOD
import org.apache.spark.scheduler._
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/ErrorUtils.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/ErrorUtils.scala
index 837d4a4d3ee78..76f91ee710590 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/ErrorUtils.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/ErrorUtils.scala
@@ -30,20 +30,19 @@ import com.google.rpc.{Code => RPCCode, ErrorInfo, Status => RPCStatus}
import io.grpc.Status
import io.grpc.protobuf.StatusProto
import io.grpc.stub.StreamObserver
-import org.apache.commons.lang3.StringUtils
-import org.apache.commons.lang3.exception.ExceptionUtils
import org.json4s.JsonDSL._
import org.json4s.jackson.JsonMethods
import org.apache.spark.{QueryContextType, SparkEnv, SparkException, SparkThrowable}
import org.apache.spark.api.python.PythonException
import org.apache.spark.connect.proto.FetchErrorDetailsResponse
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{OP_TYPE, SESSION_ID, USER_ID}
import org.apache.spark.sql.connect.config.Connect
import org.apache.spark.sql.connect.service.{ExecuteEventsManager, SessionHolder, SessionKey, SparkConnectService}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.util.ArrayImplicits._
+import org.apache.spark.util.Utils
private[connect] object ErrorUtils extends Logging {
@@ -217,7 +216,7 @@ private[connect] object ErrorUtils extends Logging {
.put(errorId, st)
}
- lazy val stackTrace = Option(ExceptionUtils.getStackTrace(st))
+ lazy val stackTrace = Option(Utils.stackTraceToString(st))
val stackTraceEnabled = sessionHolderOpt.exists(
_.session.sessionState.conf.getConf(SQLConf.PYSPARK_JVM_STACKTRACE_ENABLED))
val withStackTrace =
@@ -225,7 +224,7 @@ private[connect] object ErrorUtils extends Logging {
val maxSize = Math.min(
SparkEnv.get.conf.get(Connect.CONNECT_JVM_STACK_TRACE_MAX_SIZE),
maxMetadataSize)
- errorInfo.putMetadata("stackTrace", StringUtils.abbreviate(stackTrace.get, maxSize.toInt))
+ errorInfo.putMetadata("stackTrace", Utils.abbreviate(stackTrace.get, maxSize.toInt))
} else {
errorInfo
}
@@ -297,7 +296,7 @@ private[connect] object ErrorUtils extends Logging {
e,
Status.UNKNOWN
.withCause(e)
- .withDescription(StringUtils.abbreviate(e.getMessage, 2048))
+ .withDescription(Utils.abbreviate(e.getMessage, 2048))
.asRuntimeException())
}
partial
diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/ProtoToParsedPlanTestSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/ProtoToParsedPlanTestSuite.scala
index 29ad97ad9fbe8..02c5683e02765 100644
--- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/ProtoToParsedPlanTestSuite.scala
+++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/ProtoToParsedPlanTestSuite.scala
@@ -27,7 +27,6 @@ import scala.util.{Failure, Success, Try}
import org.apache.spark.{SparkConf, SparkFunSuite}
import org.apache.spark.connect.proto
import org.apache.spark.internal.LogKeys.PATH
-import org.apache.spark.internal.MDC
import org.apache.spark.sql.catalyst.{catalog, QueryPlanningTracker}
import org.apache.spark.sql.catalyst.analysis.{caseSensitiveResolution, Analyzer, FunctionRegistry, Resolver, TableFunctionRegistry}
import org.apache.spark.sql.catalyst.catalog.SessionCatalog
@@ -38,6 +37,7 @@ import org.apache.spark.sql.connect.config.Connect
import org.apache.spark.sql.connect.planner.SparkConnectPlanner
import org.apache.spark.sql.connector.catalog.{CatalogManager, Column, Identifier, InMemoryCatalog}
import org.apache.spark.sql.connector.expressions.Transform
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.LongType
import org.apache.spark.sql.util.CaseInsensitiveStringMap
@@ -198,7 +198,9 @@ class ProtoToParsedPlanTestSuite
}
Helper.execute(catalystPlan)
}
- val actual = removeMemoryAddress(normalizeExprIds(finalAnalyzedPlan).treeString)
+ val actual = withSQLConf(SQLConf.MAX_TO_STRING_FIELDS.key -> Int.MaxValue.toString) {
+ removeMemoryAddress(normalizeExprIds(finalAnalyzedPlan).treeString)
+ }
val goldenFile = goldenFilePath.resolve(relativePath).getParent.resolve(name + ".explain")
Try(readGoldenFile(goldenFile)) match {
case Success(expected) if expected == actual => // Test passes.
diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/messages/AbbreviateSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/messages/AbbreviateSuite.scala
index 94d92a264d20c..41271a874cc8d 100644
--- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/messages/AbbreviateSuite.scala
+++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/messages/AbbreviateSuite.scala
@@ -27,7 +27,7 @@ import org.apache.spark.sql.connect.common.{ProtoDataTypes, ProtoUtils}
class AbbreviateSuite extends SparkFunSuite {
test("truncate string: simple SQL text") {
- val message = proto.SQL.newBuilder().setQuery("x" * 1024).build()
+ val message = proto.SQL.newBuilder().setQuery("x".repeat(1024)).build()
Seq(1, 16, 256, 512, 1024, 2048).foreach { threshold =>
val truncated = ProtoUtils.abbreviate(message, threshold)
@@ -47,7 +47,7 @@ class AbbreviateSuite extends SparkFunSuite {
.setSql(
proto.SQL
.newBuilder()
- .setQuery("x" * 1024)
+ .setQuery("x".repeat(1024))
.build())
.build()
val drop = proto.Relation
@@ -214,7 +214,10 @@ class AbbreviateSuite extends SparkFunSuite {
test("truncate map") {
val read = proto.Read.NamedTable
.newBuilder()
- .putAllOptions(Map("k1" * 4096 -> "v1" * 4096, "k2" * 4096 -> "v2" * 4096).asJava)
+ .putAllOptions(
+ Map(
+ "k1".repeat(4096) -> "v1".repeat(4096),
+ "k2".repeat(4096) -> "v2".repeat(4096)).asJava)
.build()
val threshold = 1024
@@ -237,14 +240,14 @@ class AbbreviateSuite extends SparkFunSuite {
.newBuilder()
.setUnresolvedAttribute(proto.Expression.UnresolvedAttribute
.newBuilder()
- .setUnparsedIdentifier("v1" * 4096)
+ .setUnparsedIdentifier("v1".repeat(4096))
.build())
.build(),
"k2" -> proto.Expression
.newBuilder()
.setUnresolvedAttribute(proto.Expression.UnresolvedAttribute
.newBuilder()
- .setUnparsedIdentifier("v2" * 4096)
+ .setUnparsedIdentifier("v2".repeat(4096))
.build())
.build()).asJava)
.build()
@@ -272,7 +275,7 @@ class AbbreviateSuite extends SparkFunSuite {
.newBuilder()
.setQuery(
// Level 5.
- "x" * (threshold + 32))
+ "x".repeat(threshold + 32))
.build())
.build()
diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PipelineEventStreamSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PipelineEventStreamSuite.scala
index 100aa2e3b63a7..83862545a723e 100644
--- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PipelineEventStreamSuite.scala
+++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PipelineEventStreamSuite.scala
@@ -70,14 +70,67 @@ class PipelineEventStreamSuite extends SparkDeclarativePipelinesServerTest {
}
}
- test("check error events from stream") {
+ test("flow resolution failure") {
+ val dryOptions = Seq(true, false)
+
+ dryOptions.foreach { dry =>
+ withRawBlockingStub { implicit stub =>
+ val graphId = createDataflowGraph
+ val pipeline = new TestPipelineDefinition(graphId) {
+ createTable(
+ name = "a",
+ datasetType = proto.DatasetType.MATERIALIZED_VIEW,
+ sql = Some("SELECT * FROM unknown_table"))
+ createTable(
+ name = "b",
+ datasetType = proto.DatasetType.TABLE,
+ sql = Some("SELECT * FROM STREAM a"))
+ }
+ registerPipelineDatasets(pipeline)
+
+ val capturedEvents = new ArrayBuffer[PipelineEvent]()
+ withClient { client =>
+ val startRunRequest = buildStartRunPlan(
+ proto.PipelineCommand.StartRun
+ .newBuilder()
+ .setDataflowGraphId(graphId)
+ .setDry(dry)
+ .build())
+ val ex = intercept[AnalysisException] {
+ val responseIterator = client.execute(startRunRequest)
+ while (responseIterator.hasNext) {
+ val response = responseIterator.next()
+ if (response.hasPipelineEventResult) {
+ capturedEvents.append(response.getPipelineEventResult.getEvent)
+ }
+ }
+ }
+ // (?s) enables wildcard matching on newline characters
+ val runFailureErrorMsg = "(?s).*Failed to resolve flows in the pipeline.*".r
+ assert(runFailureErrorMsg.matches(ex.getMessage))
+ val expectedLogPatterns = Set(
+ "(?s).*Failed to resolve flow.*Failed to read dataset 'spark_catalog.default.a'.*".r,
+ "(?s).*Failed to resolve flow.*[TABLE_OR_VIEW_NOT_FOUND].*".r)
+ expectedLogPatterns.foreach { logPattern =>
+ assert(
+ capturedEvents.exists(e => logPattern.matches(e.getMessage)),
+ s"Did not receive expected event matching pattern: $logPattern")
+ }
+ // Ensure that the error causing the run failure is not surfaced to the user twice
+ assert(capturedEvents.forall(e => !runFailureErrorMsg.matches(e.getMessage)))
+ }
+ }
+ }
+ }
+
+ test("successful dry run") {
withRawBlockingStub { implicit stub =>
val graphId = createDataflowGraph
val pipeline = new TestPipelineDefinition(graphId) {
createTable(
name = "a",
datasetType = proto.DatasetType.MATERIALIZED_VIEW,
- sql = Some("SELECT * FROM unknown_table"))
+ sql = Some("SELECT * FROM RANGE(5)"))
createTable(
name = "b",
datasetType = proto.DatasetType.TABLE,
@@ -88,29 +141,29 @@ class PipelineEventStreamSuite extends SparkDeclarativePipelinesServerTest {
val capturedEvents = new ArrayBuffer[PipelineEvent]()
withClient { client =>
val startRunRequest = buildStartRunPlan(
- proto.PipelineCommand.StartRun.newBuilder().setDataflowGraphId(graphId).build())
- val ex = intercept[AnalysisException] {
- val responseIterator = client.execute(startRunRequest)
- while (responseIterator.hasNext) {
- val response = responseIterator.next()
- if (response.hasPipelineEventResult) {
- capturedEvents.append(response.getPipelineEventResult.getEvent)
- }
+ proto.PipelineCommand.StartRun
+ .newBuilder()
+ .setDataflowGraphId(graphId)
+ .setDry(true)
+ .build())
+ val responseIterator = client.execute(startRunRequest)
+ while (responseIterator.hasNext) {
+ val response = responseIterator.next()
+ if (response.hasPipelineEventResult) {
+ capturedEvents.append(response.getPipelineEventResult.getEvent)
}
}
- // (?s) enables wildcard matching on newline characters
- val runFailureErrorMsg = "(?s).*Failed to resolve flows in the pipeline.*".r
- assert(runFailureErrorMsg.matches(ex.getMessage))
- val expectedLogPatterns = Set(
- "(?s).*Failed to resolve flow.*Failed to read dataset 'spark_catalog.default.a'.*".r,
- "(?s).*Failed to resolve flow.*[TABLE_OR_VIEW_NOT_FOUND].*".r)
- expectedLogPatterns.foreach { logPattern =>
+ val expectedEventMessages = Set("Run is COMPLETED")
+ expectedEventMessages.foreach { eventMessage =>
assert(
- capturedEvents.exists(e => logPattern.matches(e.getMessage)),
- s"Did not receive expected event matching pattern: $logPattern")
+ capturedEvents.exists(e => e.getMessage.contains(eventMessage)),
+ s"Did not receive expected event: $eventMessage")
}
- // Ensure that the error causing the run failure is not surfaced to the user twice
- assert(capturedEvents.forall(e => !runFailureErrorMsg.matches(e.getMessage)))
+ }
+
+ // No flows should be started in dry run mode
+ capturedEvents.foreach { event =>
+ assert(!event.getMessage.contains("is QUEUED"))
}
}
}
diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PipelineRefreshFunctionalSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PipelineRefreshFunctionalSuite.scala
new file mode 100644
index 0000000000000..794932544d5f9
--- /dev/null
+++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PipelineRefreshFunctionalSuite.scala
@@ -0,0 +1,353 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.connect.pipelines
+
+import scala.collection.mutable.ArrayBuffer
+import scala.jdk.CollectionConverters._
+
+import org.apache.spark.connect.proto.{DatasetType, PipelineCommand, PipelineEvent}
+import org.apache.spark.sql.QueryTest
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.connect.service.{SessionKey, SparkConnectService}
+import org.apache.spark.sql.pipelines.utils.{EventVerificationTestHelpers, TestPipelineUpdateContextMixin}
+
+/**
+ * Comprehensive test suite that validates pipeline refresh functionality by running actual
+ * pipelines with different refresh parameters and validating the results.
+ */
+class PipelineRefreshFunctionalSuite
+ extends SparkDeclarativePipelinesServerTest
+ with TestPipelineUpdateContextMixin
+ with EventVerificationTestHelpers {
+
+ private val externalSourceTable = TableIdentifier(
+ catalog = Some("spark_catalog"),
+ database = Some("default"),
+ table = "source_data")
+
+ override def beforeEach(): Unit = {
+ super.beforeEach()
+ // Create source table to simulate streaming updates
+ spark.sql(s"CREATE TABLE $externalSourceTable AS SELECT * FROM RANGE(1, 2)")
+ }
+
+ override def afterEach(): Unit = {
+ super.afterEach()
+ // Clean up the source table after each test
+ spark.sql(s"DROP TABLE IF EXISTS $externalSourceTable")
+ }
+
+ private def createTestPipeline(graphId: String): TestPipelineDefinition = {
+ new TestPipelineDefinition(graphId) {
+ // Create tables that depend on the mv
+ createTable(
+ name = "a",
+ datasetType = DatasetType.TABLE,
+ sql = Some(s"SELECT id FROM STREAM $externalSourceTable"))
+ createTable(
+ name = "b",
+ datasetType = DatasetType.TABLE,
+ sql = Some(s"SELECT id FROM STREAM $externalSourceTable"))
+ createTable(
+ name = "mv",
+ datasetType = DatasetType.MATERIALIZED_VIEW,
+ sql = Some(s"SELECT id FROM a"))
+ }
+ }
+
+ /**
+ * Helper method to run refresh tests with common setup and verification logic. This reduces
+ * code duplication across the refresh test cases.
+ */
+ private def runRefreshTest(
+ refreshConfigBuilder: String => Option[PipelineCommand.StartRun] = _ => None,
+ expectedContentAfterRefresh: Map[String, Set[Map[String, Any]]],
+ eventValidation: Option[ArrayBuffer[PipelineEvent] => Unit] = None): Unit = {
+ withRawBlockingStub { implicit stub =>
+ val graphId = createDataflowGraph
+ val pipeline = createTestPipeline(graphId)
+ registerPipelineDatasets(pipeline)
+
+ // First run to populate tables
+ startPipelineAndWaitForCompletion(graphId)
+
+ // combine above into a map for verification
+ val initialContent = Map(
+ "spark_catalog.default.a" -> Set(Map("id" -> 1)),
+ "spark_catalog.default.b" -> Set(Map("id" -> 1)),
+ "spark_catalog.default.mv" -> Set(Map("id" -> 1)))
+ // Verify initial content
+ initialContent.foreach { case (tableName, expectedRows) =>
+ checkTableContent(tableName, expectedRows)
+ }
+ // Clear cached pipeline execution before starting new run
+ SparkConnectService.sessionManager
+ .getIsolatedSessionIfPresent(SessionKey(defaultUserId, defaultSessionId))
+ .foreach(_.removeAllPipelineExecutions())
+
+ // Replace source data to simulate a streaming update
+ spark.sql(
+ "INSERT OVERWRITE TABLE spark_catalog.default.source_data " +
+ "SELECT * FROM VALUES (2), (3) AS t(id)")
+
+ // Run with specified refresh configuration
+ val capturedEvents = refreshConfigBuilder(graphId) match {
+ case Some(startRun) => startPipelineAndWaitForCompletion(startRun)
+ case None => startPipelineAndWaitForCompletion(graphId)
+ }
+
+ // Additional validation if provided
+ eventValidation.foreach(_(capturedEvents))
+
+ // Verify final content with checkTableContent
+ expectedContentAfterRefresh.foreach { case (tableName, expectedRows) =>
+ checkTableContent(tableName, expectedRows)
+ }
+ }
+ }
+
+ test("pipeline runs selective full_refresh") {
+ runRefreshTest(
+ refreshConfigBuilder = { graphId =>
+ Some(
+ PipelineCommand.StartRun
+ .newBuilder()
+ .setDataflowGraphId(graphId)
+ .addAllFullRefreshSelection(List("a").asJava)
+ .build())
+ },
+ expectedContentAfterRefresh = Map(
+ "spark_catalog.default.a" -> Set(
+ Map("id" -> 2), // a is fully refreshed and only contains the new values
+ Map("id" -> 3)),
+ "spark_catalog.default.b" -> Set(
+ Map("id" -> 1) // b is not refreshed, so it retains the old value
+ ),
+ "spark_catalog.default.mv" -> Set(
+ Map("id" -> 1) // mv is not refreshed, so it retains the old value
+ )),
+ eventValidation = Some { capturedEvents =>
+ // assert that table_b is excluded
+ assert(
+ capturedEvents.exists(
+ _.getMessage.contains(s"Flow \'spark_catalog.default.b\' is EXCLUDED.")))
+ // assert that table_a ran to completion
+ assert(
+ capturedEvents.exists(
+ _.getMessage.contains(s"Flow spark_catalog.default.a has COMPLETED.")))
+ // assert that mv is excluded
+ assert(
+ capturedEvents.exists(
+ _.getMessage.contains(s"Flow \'spark_catalog.default.mv\' is EXCLUDED.")))
+ // Verify completion event
+ assert(capturedEvents.exists(_.getMessage.contains("Run is COMPLETED")))
+ })
+ }
+
+ test("pipeline runs selective full_refresh and selective refresh") {
+ runRefreshTest(
+ refreshConfigBuilder = { graphId =>
+ Some(
+ PipelineCommand.StartRun
+ .newBuilder()
+ .setDataflowGraphId(graphId)
+ .addAllFullRefreshSelection(Seq("a", "mv").asJava)
+ .addRefreshSelection("b")
+ .build())
+ },
+ expectedContentAfterRefresh = Map(
+ "spark_catalog.default.a" -> Set(
+ Map("id" -> 2), // a is fully refreshed and only contains the new values
+ Map("id" -> 3)),
+ "spark_catalog.default.b" -> Set(
+ Map("id" -> 1), // b is refreshed, so it retains the old value and adds the new ones
+ Map("id" -> 2),
+ Map("id" -> 3)),
+ "spark_catalog.default.mv" -> Set(
+ Map("id" -> 2), // mv is fully refreshed and only contains the new values
+ Map("id" -> 3))))
+ }
+
+ test("pipeline runs refresh by default") {
+ runRefreshTest(expectedContentAfterRefresh =
+ Map(
+ "spark_catalog.default.a" -> Set(
+ Map(
+ "id" -> 1
+ ), // a is refreshed by default, retains the old value and adds the new ones
+ Map("id" -> 2),
+ Map("id" -> 3)),
+ "spark_catalog.default.b" -> Set(
+ Map(
+ "id" -> 1
+ ), // b is refreshed by default, retains the old value and adds the new ones
+ Map("id" -> 2),
+ Map("id" -> 3)),
+ "spark_catalog.default.mv" -> Set(
+ Map("id" -> 1),
+ Map("id" -> 2), // mv is refreshed from table a, retains all values
+ Map("id" -> 3))))
+ }
+
+ test("pipeline runs full refresh all") {
+ runRefreshTest(
+ refreshConfigBuilder = { graphId =>
+ Some(
+ PipelineCommand.StartRun
+ .newBuilder()
+ .setDataflowGraphId(graphId)
+ .setFullRefreshAll(true)
+ .build())
+ },
+ // full refresh all causes all tables to lose the initial value
+ // and only contain the new values after the source data is updated
+ expectedContentAfterRefresh = Map(
+ "spark_catalog.default.a" -> Set(Map("id" -> 2), Map("id" -> 3)),
+ "spark_catalog.default.b" -> Set(Map("id" -> 2), Map("id" -> 3)),
+ "spark_catalog.default.mv" -> Set(Map("id" -> 2), Map("id" -> 3))))
+ }
+
+ test("validation: cannot specify subset refresh when full_refresh_all is true") {
+ withRawBlockingStub { implicit stub =>
+ val graphId = createDataflowGraph
+ val pipeline = createTestPipeline(graphId)
+ registerPipelineDatasets(pipeline)
+
+ val startRun = PipelineCommand.StartRun
+ .newBuilder()
+ .setDataflowGraphId(graphId)
+ .setFullRefreshAll(true)
+ .addRefreshSelection("a")
+ .build()
+
+ val exception = intercept[IllegalArgumentException] {
+ startPipelineAndWaitForCompletion(startRun)
+ }
+ assert(
+ exception.getMessage.contains(
+ "Cannot specify a subset to refresh when full refresh all is set to true"))
+ }
+ }
+
+ test("validation: cannot specify subset full_refresh when full_refresh_all is true") {
+ withRawBlockingStub { implicit stub =>
+ val graphId = createDataflowGraph
+ val pipeline = createTestPipeline(graphId)
+ registerPipelineDatasets(pipeline)
+
+ val startRun = PipelineCommand.StartRun
+ .newBuilder()
+ .setDataflowGraphId(graphId)
+ .setFullRefreshAll(true)
+ .addFullRefreshSelection("a")
+ .build()
+
+ val exception = intercept[IllegalArgumentException] {
+ startPipelineAndWaitForCompletion(startRun)
+ }
+ assert(
+ exception.getMessage.contains(
+ "Cannot specify a subset to full refresh when full refresh all is set to true"))
+ }
+ }
+
+ test("validation: refresh and full_refresh cannot overlap") {
+ withRawBlockingStub { implicit stub =>
+ val graphId = createDataflowGraph
+ val pipeline = createTestPipeline(graphId)
+ registerPipelineDatasets(pipeline)
+
+ val startRun = PipelineCommand.StartRun
+ .newBuilder()
+ .setDataflowGraphId(graphId)
+ .addRefreshSelection("a")
+ .addFullRefreshSelection("a")
+ .build()
+
+ val exception = intercept[IllegalArgumentException] {
+ startPipelineAndWaitForCompletion(startRun)
+ }
+ assert(
+ exception.getMessage.contains(
+ "Datasets specified for refresh and full refresh cannot overlap"))
+ assert(exception.getMessage.contains("a"))
+ }
+ }
+
+ test("validation: multiple overlapping tables in refresh and full_refresh not allowed") {
+ withRawBlockingStub { implicit stub =>
+ val graphId = createDataflowGraph
+ val pipeline = createTestPipeline(graphId)
+ registerPipelineDatasets(pipeline)
+
+ val startRun = PipelineCommand.StartRun
+ .newBuilder()
+ .setDataflowGraphId(graphId)
+ .addRefreshSelection("a")
+ .addRefreshSelection("b")
+ .addFullRefreshSelection("a")
+ .build()
+
+ val exception = intercept[IllegalArgumentException] {
+ startPipelineAndWaitForCompletion(startRun)
+ }
+ assert(
+ exception.getMessage.contains(
+ "Datasets specified for refresh and full refresh cannot overlap"))
+ assert(exception.getMessage.contains("a"))
+ }
+ }
+
+ test("validation: fully qualified table names in validation") {
+ withRawBlockingStub { implicit stub =>
+ val graphId = createDataflowGraph
+ val pipeline = createTestPipeline(graphId)
+ registerPipelineDatasets(pipeline)
+
+ val startRun = PipelineCommand.StartRun
+ .newBuilder()
+ .setDataflowGraphId(graphId)
+ .addRefreshSelection("spark_catalog.default.a")
+ .addFullRefreshSelection("a") // This should be treated as the same table
+ .build()
+
+ val exception = intercept[IllegalArgumentException] {
+ startPipelineAndWaitForCompletion(startRun)
+ }
+ assert(
+ exception.getMessage.contains(
+ "Datasets specified for refresh and full refresh cannot overlap"))
+ }
+ }
+
+ private def checkTableContent[A <: Map[String, Any]](
+ name: String,
+ expectedContent: Set[A]): Unit = {
+ spark.catalog.refreshTable(name) // clear cache for the table
+ val df = spark.table(name)
+ QueryTest.checkAnswer(
+ df,
+ expectedContent
+ .map(row => {
+ // Convert each row to a Row object
+ org.apache.spark.sql.Row.fromSeq(row.values.toSeq)
+ })
+ .toSeq
+ .asJava)
+ }
+}
diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PythonPipelineSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PythonPipelineSuite.scala
index 21f2857090182..0895463037f2a 100644
--- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PythonPipelineSuite.scala
+++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/PythonPipelineSuite.scala
@@ -20,6 +20,7 @@ package org.apache.spark.sql.connect.pipelines
import java.io.{BufferedReader, InputStreamReader}
import java.nio.charset.StandardCharsets
import java.nio.file.Paths
+import java.util.UUID
import java.util.concurrent.TimeUnit
import scala.collection.mutable.ArrayBuffer
@@ -28,7 +29,8 @@ import scala.util.Try
import org.apache.spark.api.python.PythonUtils
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.pipelines.graph.DataflowGraph
+import org.apache.spark.sql.connect.service.SparkConnectService
+import org.apache.spark.sql.pipelines.graph.{DataflowGraph, PipelineUpdateContextImpl}
import org.apache.spark.sql.pipelines.utils.{EventVerificationTestHelpers, TestPipelineUpdateContextMixin}
/**
@@ -42,10 +44,12 @@ class PythonPipelineSuite
def buildGraph(pythonText: String): DataflowGraph = {
val indentedPythonText = pythonText.linesIterator.map(" " + _).mkString("\n")
+ // create a unique identifier to allow identifying the session and dataflow graph
+ val customSessionIdentifier = UUID.randomUUID().toString
val pythonCode =
s"""
|from pyspark.sql import SparkSession
- |from pyspark import pipelines as sdp
+ |from pyspark import pipelines as dp
|from pyspark.pipelines.spark_connect_graph_element_registry import (
| SparkConnectGraphElementRegistry,
|)
@@ -57,6 +61,7 @@ class PythonPipelineSuite
|spark = SparkSession.builder \\
| .remote("sc://localhost:$serverPort") \\
| .config("spark.connect.grpc.channel.timeout", "5s") \\
+ | .config("spark.custom.identifier", "$customSessionIdentifier") \\
| .create()
|
|dataflow_graph_id = create_dataflow_graph(
@@ -78,8 +83,17 @@ class PythonPipelineSuite
throw new RuntimeException(
s"Python process failed with exit code $exitCode. Output: ${output.mkString("\n")}")
}
+ val activeSessions = SparkConnectService.sessionManager.listActiveSessions
- val dataflowGraphContexts = DataflowGraphRegistry.getAllDataflowGraphs
+ // get the session holder by finding the session with the custom UUID set in the conf
+ val sessionHolder = activeSessions
+ .map(info => SparkConnectService.sessionManager.getIsolatedSession(info.key, None))
+ .find(_.session.conf.get("spark.custom.identifier") == customSessionIdentifier)
+ .getOrElse(
+ throw new RuntimeException(s"Session with identifier $customSessionIdentifier not found"))
+
+ // get all dataflow graphs from the session holder
+ val dataflowGraphContexts = sessionHolder.dataflowGraphRegistry.getAllDataflowGraphs
assert(dataflowGraphContexts.size == 1)
dataflowGraphContexts.head.toDataflowGraph
@@ -91,9 +105,9 @@ class PythonPipelineSuite
test("basic") {
val graph = buildGraph("""
- |@sdp.table
+ |@dp.table
|def table1():
- | return spark.range(10)
+ | return spark.readStream.format("rate").load()
|""".stripMargin)
.resolve()
.validate()
@@ -104,19 +118,19 @@ class PythonPipelineSuite
test("basic with inverted topological order") {
// This graph is purposefully in the wrong topological order to test the topological sort
val graph = buildGraph("""
- |@sdp.table()
+ |@dp.table()
|def b():
| return spark.readStream.table("a")
|
- |@sdp.table()
+ |@dp.table()
|def c():
| return spark.readStream.table("a")
|
- |@sdp.table()
+ |@dp.materialized_view()
|def d():
| return spark.read.table("a")
|
- |@sdp.table()
+ |@dp.materialized_view()
|def a():
| return spark.range(5)
|""".stripMargin)
@@ -127,11 +141,11 @@ class PythonPipelineSuite
test("flows") {
val graph = buildGraph("""
- |@sdp.table()
+ |@dp.table()
|def a():
| return spark.readStream.format("rate").load()
|
- |@sdp.append_flow(target = "a")
+ |@dp.append_flow(target = "a")
|def supplement():
| return spark.readStream.format("rate").load()
|""".stripMargin).resolve().validate()
@@ -146,15 +160,15 @@ class PythonPipelineSuite
test("referencing internal datasets") {
val graph = buildGraph("""
- |@sdp.materialized_view
+ |@dp.materialized_view
|def src():
| return spark.range(5)
|
- |@sdp.materialized_view
+ |@dp.materialized_view
|def a():
| return spark.read.table("src")
|
- |@sdp.table
+ |@dp.table
|def b():
| return spark.readStream.table("src")
|""".stripMargin).resolve().validate()
@@ -177,15 +191,15 @@ class PythonPipelineSuite
test("referencing external datasets") {
sql("CREATE TABLE spark_catalog.default.src AS SELECT * FROM RANGE(5)")
val graph = buildGraph("""
- |@sdp.table
+ |@dp.materialized_view
|def a():
| return spark.read.table("spark_catalog.default.src")
|
- |@sdp.table
+ |@dp.materialized_view
|def b():
| return spark.table("spark_catalog.default.src")
|
- |@sdp.table
+ |@dp.table
|def c():
| return spark.readStream.table("spark_catalog.default.src")
|""".stripMargin).resolve().validate()
@@ -204,15 +218,15 @@ class PythonPipelineSuite
test("referencing internal datasets failed") {
val graph = buildGraph("""
- |@sdp.table
+ |@dp.table
|def a():
| return spark.read.table("src")
|
- |@sdp.table
+ |@dp.table
|def b():
| return spark.table("src")
|
- |@sdp.table
+ |@dp.table
|def c():
| return spark.readStream.table("src")
|""".stripMargin).resolve()
@@ -226,15 +240,15 @@ class PythonPipelineSuite
test("referencing external datasets failed") {
val graph = buildGraph("""
- |@sdp.table
+ |@dp.table
|def a():
| return spark.read.table("spark_catalog.default.src")
|
- |@sdp.table
+ |@dp.materialized_view
|def b():
| return spark.table("spark_catalog.default.src")
|
- |@sdp.table
+ |@dp.materialized_view
|def c():
| return spark.readStream.table("spark_catalog.default.src")
|""".stripMargin).resolve()
@@ -246,11 +260,11 @@ class PythonPipelineSuite
test("create dataset with the same name will fail") {
val ex = intercept[AnalysisException] {
buildGraph(s"""
- |@sdp.materialized_view
+ |@dp.materialized_view
|def a():
| return spark.range(1)
|
- |@sdp.materialized_view(name = "a")
+ |@dp.materialized_view(name = "a")
|def b():
| return spark.range(1)
|""".stripMargin)
@@ -260,19 +274,19 @@ class PythonPipelineSuite
test("create datasets with fully/partially qualified names") {
val graph = buildGraph(s"""
- |@sdp.table
+ |@dp.table
|def mv_1():
| return spark.range(5)
|
- |@sdp.table(name = "schema_a.mv_2")
+ |@dp.table(name = "schema_a.mv_2")
|def irrelevant_1():
| return spark.range(5)
|
- |@sdp.table(name = "st_1")
+ |@dp.table(name = "st_1")
|def irrelevant_2():
| return spark.readStream.format("rate").load()
|
- |@sdp.table(name = "schema_b.st_2")
+ |@dp.table(name = "schema_b.st_2")
|def irrelevant_3():
| return spark.readStream.format("rate").load()
|""".stripMargin).resolve()
@@ -319,11 +333,11 @@ class PythonPipelineSuite
test("create datasets with three part names") {
val graphTry = Try {
buildGraph(s"""
- |@sdp.table(name = "some_catalog.some_schema.mv")
+ |@dp.table(name = "some_catalog.some_schema.mv")
|def irrelevant_1():
| return spark.range(5)
|
- |@sdp.table(name = "some_catalog.some_schema.st")
+ |@dp.table(name = "some_catalog.some_schema.st")
|def irrelevant_2():
| return spark.readStream.format("rate").load()
|""".stripMargin).resolve()
@@ -339,24 +353,28 @@ class PythonPipelineSuite
TableIdentifier("st", Some("some_schema"), Some("some_catalog"))))
}
- test("view works") {
+ test("temporary views works") {
+ // A table is defined since pipeline with only temporary views is invalid.
val graph = buildGraph(s"""
- |@sdp.temporary_view
+ |@dp.table
+ |def mv_1():
+ | return spark.range(5)
+ |@dp.temporary_view
|def view_1():
| return spark.range(5)
|
- |@sdp.temporary_view(name= "view_2")
+ |@dp.temporary_view(name= "view_2")
|def irrelevant_1():
| return spark.read.table("view_1")
|
- |@sdp.temporary_view(name= "view_3")
+ |@dp.temporary_view(name= "view_3")
|def irrelevant_2():
| return spark.read.table("view_1")
|""".stripMargin).resolve()
// views are temporary views, so they're not fully qualified.
- assert(graph.tables.isEmpty)
assert(
- graph.flows.map(_.identifier.unquotedString).toSet == Set("view_1", "view_2", "view_3"))
+ Set("view_1", "view_2", "view_3").subsetOf(
+ graph.flows.map(_.identifier.unquotedString).toSet))
// dependencies are correctly resolved view_2 reading from view_1
assert(
graph.resolvedFlow(TableIdentifier("view_2")).inputs.contains(TableIdentifier("view_1")))
@@ -367,11 +385,11 @@ class PythonPipelineSuite
test("create named flow with multipart name will fail") {
val ex = intercept[RuntimeException] {
buildGraph(s"""
- |@sdp.table
+ |@dp.table
|def src():
| return spark.readStream.table("src0")
|
- |@sdp.append_flow(name ="some_schema.some_flow", target = "src")
+ |@dp.append_flow(name ="some_schema.some_flow", target = "src")
|def some_flow():
| return spark.readStream.format("rate").load()
|""".stripMargin)
@@ -381,11 +399,11 @@ class PythonPipelineSuite
test("create flow with multipart target and no explicit name succeeds") {
val graph = buildGraph("""
- |@sdp.table()
+ |@dp.table()
|def a():
| return spark.readStream.format("rate").load()
|
- |@sdp.append_flow(target = "default.a")
+ |@dp.append_flow(target = "default.a")
|def supplement():
| return spark.readStream.format("rate").load()
|""".stripMargin).resolve().validate()
@@ -400,11 +418,11 @@ class PythonPipelineSuite
test("create named flow with multipart target succeeds") {
val graph = buildGraph("""
- |@sdp.table()
+ |@dp.table()
|def a():
| return spark.readStream.format("rate").load()
|
- |@sdp.append_flow(target = "default.a", name = "something")
+ |@dp.append_flow(target = "default.a", name = "something")
|def supplement():
| return spark.readStream.format("rate").load()
|""".stripMargin)
@@ -416,6 +434,103 @@ class PythonPipelineSuite
.map(_.identifier) == Seq(graphIdentifier("a"), graphIdentifier("something")))
}
+ test("groupby and rollup works with internal datasets, referencing with (col, str)") {
+ val graph = buildGraph("""
+ from pyspark.sql.functions import col, sum, count
+
+ @dp.materialized_view
+ def src():
+ return spark.range(3)
+
+ @dp.materialized_view
+ def groupby_with_col_result():
+ return spark.read.table("src").groupBy(col("id")).agg(
+ sum("id").alias("sum_id"),
+ count("*").alias("cnt")
+ )
+
+ @dp.materialized_view
+ def groupby_with_str_result():
+ return spark.read.table("src").groupBy("id").agg(
+ sum("id").alias("sum_id"),
+ count("*").alias("cnt")
+ )
+
+ @dp.materialized_view
+ def rollup_with_col_result():
+ return spark.read.table("src").rollup(col("id")).agg(
+ sum("id").alias("sum_id"),
+ count("*").alias("cnt")
+ )
+
+ @dp.materialized_view
+ def rollup_with_str_result():
+ return spark.read.table("src").rollup("id").agg(
+ sum("id").alias("sum_id"),
+ count("*").alias("cnt")
+ )
+ """)
+
+ val updateContext = new PipelineUpdateContextImpl(graph, _ => ())
+ updateContext.pipelineExecution.runPipeline()
+ updateContext.pipelineExecution.awaitCompletion()
+
+ val groupbyDfs =
+ Seq(spark.table("groupby_with_col_result"), spark.table("groupby_with_str_result"))
+
+ val rollupDfs =
+ Seq(spark.table("rollup_with_col_result"), spark.table("rollup_with_str_result"))
+
+ // groupBy: each variant should have exactly one row per id [0,1,2]
+ groupbyDfs.foreach { df =>
+ assert(df.select("id").collect().map(_.getLong(0)).toSet == Set(0L, 1L, 2L))
+ }
+
+ // rollup: each variant should have groupBy rows + one total row
+ rollupDfs.foreach { df =>
+ assert(df.count() == 3 + 1) // 3 ids + 1 total
+ val totalRow = df.filter("id IS NULL").collect().head
+ assert(totalRow.getLong(1) == 3L && totalRow.getLong(2) == 3L)
+ }
+ }
+
+ test("create pipeline without table will throw RUN_EMPTY_PIPELINE exception") {
+ checkError(
+ exception = intercept[AnalysisException] {
+ buildGraph(s"""
+ |spark.range(1)
+ |""".stripMargin)
+ },
+ condition = "RUN_EMPTY_PIPELINE",
+ parameters = Map.empty)
+ }
+
+ test("create pipeline with only temp view will throw RUN_EMPTY_PIPELINE exception") {
+ checkError(
+ exception = intercept[AnalysisException] {
+ buildGraph(s"""
+ |@dp.temporary_view
+ |def view_1():
+ | return spark.range(5)
+ |""".stripMargin)
+ },
+ condition = "RUN_EMPTY_PIPELINE",
+ parameters = Map.empty)
+ }
+
+ test("create pipeline with only flow will throw RUN_EMPTY_PIPELINE exception") {
+ checkError(
+ exception = intercept[AnalysisException] {
+ buildGraph(s"""
+ |@dp.append_flow(target = "a")
+ |def flow():
+ | return spark.range(5)
+ |""".stripMargin)
+ },
+ condition = "RUN_EMPTY_PIPELINE",
+ parameters = Map.empty)
+ }
+
/**
* Executes Python code in a separate process and returns the exit code.
*
diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/SparkDeclarativePipelinesServerSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/SparkDeclarativePipelinesServerSuite.scala
index 6bd31fba8ec26..ef5da0c014ee1 100644
--- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/SparkDeclarativePipelinesServerSuite.scala
+++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/SparkDeclarativePipelinesServerSuite.scala
@@ -17,10 +17,13 @@
package org.apache.spark.sql.connect.pipelines
+import java.util.UUID
+
import org.apache.spark.connect.proto
import org.apache.spark.connect.proto.{DatasetType, Expression, PipelineCommand, Relation, UnresolvedTableValuedFunction}
import org.apache.spark.connect.proto.PipelineCommand.{DefineDataset, DefineFlow}
import org.apache.spark.internal.Logging
+import org.apache.spark.sql.connect.service.{SessionKey, SparkConnectService}
class SparkDeclarativePipelinesServerSuite
extends SparkDeclarativePipelinesServerTest
@@ -41,8 +44,7 @@ class SparkDeclarativePipelinesServerSuite
.newBuilder()
.build())).getPipelineCommandResult.getCreateDataflowGraphResult.getDataflowGraphId
val definition =
- DataflowGraphRegistry
- .getDataflowGraphOrThrow(graphId)
+ getDefaultSessionHolder.dataflowGraphRegistry.getDataflowGraphOrThrow(graphId)
assert(definition.defaultDatabase == "test_db")
}
}
@@ -93,7 +95,7 @@ class SparkDeclarativePipelinesServerSuite
.setDataflowGraphId(graphId)
.setFlowName("mv")
.setTargetDatasetName("mv")
- .setPlan(
+ .setRelation(
Relation
.newBuilder()
.setUnresolvedTableValuedFunction(
@@ -115,8 +117,7 @@ class SparkDeclarativePipelinesServerSuite
|""".stripMargin)
val definition =
- DataflowGraphRegistry
- .getDataflowGraphOrThrow(graphId)
+ getDefaultSessionHolder.dataflowGraphRegistry.getDataflowGraphOrThrow(graphId)
val graph = definition.toDataflowGraph.resolve()
@@ -161,8 +162,7 @@ class SparkDeclarativePipelinesServerSuite
}
val definition =
- DataflowGraphRegistry
- .getDataflowGraphOrThrow(graphId)
+ getDefaultSessionHolder.dataflowGraphRegistry.getDataflowGraphOrThrow(graphId)
registerPipelineDatasets(pipeline)
val graph = definition.toDataflowGraph
@@ -206,7 +206,7 @@ class SparkDeclarativePipelinesServerSuite
sql = Some("SELECT * FROM STREAM tableA"))
createTable(
name = "tableC",
- datasetType = DatasetType.TABLE,
+ datasetType = DatasetType.MATERIALIZED_VIEW,
sql = Some("SELECT * FROM tableB"))
}
@@ -238,7 +238,7 @@ class SparkDeclarativePipelinesServerSuite
createView(name = "viewC", sql = "SELECT * FROM curr.tableB")
createTable(
name = "other.tableD",
- datasetType = proto.DatasetType.TABLE,
+ datasetType = proto.DatasetType.MATERIALIZED_VIEW,
sql = Some("SELECT * FROM viewC"))
}
@@ -251,4 +251,239 @@ class SparkDeclarativePipelinesServerSuite
assert(spark.table("spark_catalog.other.tableD").count() == 5)
}
}
+
+ test("dataflow graphs are session-specific") {
+ withRawBlockingStub { implicit stub =>
+ // Create a dataflow graph in the default session
+ val graphId1 = createDataflowGraph
+
+ // Register a dataset in the default session
+ sendPlan(
+ buildPlanFromPipelineCommand(
+ PipelineCommand
+ .newBuilder()
+ .setDefineDataset(
+ DefineDataset
+ .newBuilder()
+ .setDataflowGraphId(graphId1)
+ .setDatasetName("session1_table")
+ .setDatasetType(DatasetType.MATERIALIZED_VIEW))
+ .build()))
+
+ // Verify the graph exists in the default session
+ assert(getDefaultSessionHolder.dataflowGraphRegistry.getAllDataflowGraphs.size == 1)
+ }
+
+ // Create a second session with different user/session ID
+ val newSessionId = UUID.randomUUID().toString
+ val newSessionUserId = "session2_user"
+
+ withRawBlockingStub { implicit stub =>
+ // Override the test context to use different session
+ val newSessionExecuteRequest = buildExecutePlanRequest(
+ buildCreateDataflowGraphPlan(
+ proto.PipelineCommand.CreateDataflowGraph
+ .newBuilder()
+ .setDefaultCatalog("spark_catalog")
+ .setDefaultDatabase("default")
+ .build())).toBuilder
+ .setUserContext(proto.UserContext
+ .newBuilder()
+ .setUserId(newSessionUserId)
+ .build())
+ .setSessionId(newSessionId)
+ .build()
+
+ val response = stub.executePlan(newSessionExecuteRequest)
+ val graphId2 =
+ response.next().getPipelineCommandResult.getCreateDataflowGraphResult.getDataflowGraphId
+
+ // Register a different dataset in second session
+ val session2DefineRequest = buildExecutePlanRequest(
+ buildPlanFromPipelineCommand(
+ PipelineCommand
+ .newBuilder()
+ .setDefineDataset(
+ DefineDataset
+ .newBuilder()
+ .setDataflowGraphId(graphId2)
+ .setDatasetName("session2_table")
+ .setDatasetType(DatasetType.MATERIALIZED_VIEW))
+ .build())).toBuilder
+ .setUserContext(proto.UserContext
+ .newBuilder()
+ .setUserId(newSessionUserId)
+ .build())
+ .setSessionId(newSessionId)
+ .build()
+
+ stub.executePlan(session2DefineRequest).next()
+
+ // Verify session isolation - each session should only see its own graphs
+ val newSessionHolder = SparkConnectService.sessionManager
+ .getIsolatedSession(SessionKey(newSessionUserId, newSessionId), None)
+
+ val defaultSessionGraphs =
+ getDefaultSessionHolder.dataflowGraphRegistry.getAllDataflowGraphs
+ val newSessionGraphs = newSessionHolder.dataflowGraphRegistry.getAllDataflowGraphs
+
+ assert(defaultSessionGraphs.size == 1)
+ assert(newSessionGraphs.size == 1)
+
+ assert(
+ defaultSessionGraphs.head.toDataflowGraph.tables
+ .exists(_.identifier.table == "session1_table"),
+ "Session 1 should have its own table")
+ assert(
+ newSessionGraphs.head.toDataflowGraph.tables
+ .exists(_.identifier.table == "session2_table"),
+ "Session 2 should have its own table")
+ }
+ }
+
+ test("dataflow graphs are cleaned up when session is closed") {
+ val testUserId = "test_user"
+ val testSessionId = UUID.randomUUID().toString
+
+ // Create a session and dataflow graph
+ withRawBlockingStub { implicit stub =>
+ val createGraphRequest = buildExecutePlanRequest(
+ buildCreateDataflowGraphPlan(
+ proto.PipelineCommand.CreateDataflowGraph
+ .newBuilder()
+ .setDefaultCatalog("spark_catalog")
+ .setDefaultDatabase("default")
+ .build())).toBuilder
+ .setUserContext(proto.UserContext
+ .newBuilder()
+ .setUserId(testUserId)
+ .build())
+ .setSessionId(testSessionId)
+ .build()
+
+ val response = stub.executePlan(createGraphRequest)
+ val graphId =
+ response.next().getPipelineCommandResult.getCreateDataflowGraphResult.getDataflowGraphId
+
+ // Register a dataset
+ val defineRequest = buildExecutePlanRequest(
+ buildPlanFromPipelineCommand(
+ PipelineCommand
+ .newBuilder()
+ .setDefineDataset(
+ DefineDataset
+ .newBuilder()
+ .setDataflowGraphId(graphId)
+ .setDatasetName("test_table")
+ .setDatasetType(DatasetType.MATERIALIZED_VIEW))
+ .build())).toBuilder
+ .setUserContext(proto.UserContext
+ .newBuilder()
+ .setUserId(testUserId)
+ .build())
+ .setSessionId(testSessionId)
+ .build()
+
+ stub.executePlan(defineRequest).next()
+
+ // Verify the graph exists
+ val sessionHolder = SparkConnectService.sessionManager
+ .getIsolatedSessionIfPresent(SessionKey(testUserId, testSessionId))
+ .get
+
+ val graphsBefore = sessionHolder.dataflowGraphRegistry.getAllDataflowGraphs
+ assert(graphsBefore.size == 1)
+
+ // Close the session
+ SparkConnectService.sessionManager.closeSession(SessionKey(testUserId, testSessionId))
+
+ // Verify the session is no longer available
+ val sessionAfterClose = SparkConnectService.sessionManager
+ .getIsolatedSessionIfPresent(SessionKey(testUserId, testSessionId))
+
+ assert(sessionAfterClose.isEmpty, "Session should be cleaned up after close")
+ // Verify the graph is removed
+ val graphsAfter = sessionHolder.dataflowGraphRegistry.getAllDataflowGraphs
+ assert(graphsAfter.isEmpty, "Graph should be removed after session close")
+ }
+ }
+
+ test("multiple dataflow graphs can exist in the same session") {
+ withRawBlockingStub { implicit stub =>
+ // Create two dataflow graphs in the same session
+ val graphId1 = createDataflowGraph
+ val graphId2 = createDataflowGraph
+
+ // Register datasets in both graphs
+ sendPlan(
+ buildPlanFromPipelineCommand(
+ PipelineCommand
+ .newBuilder()
+ .setDefineDataset(
+ DefineDataset
+ .newBuilder()
+ .setDataflowGraphId(graphId1)
+ .setDatasetName("graph1_table")
+ .setDatasetType(DatasetType.MATERIALIZED_VIEW))
+ .build()))
+
+ sendPlan(
+ buildPlanFromPipelineCommand(
+ PipelineCommand
+ .newBuilder()
+ .setDefineDataset(
+ DefineDataset
+ .newBuilder()
+ .setDataflowGraphId(graphId2)
+ .setDatasetName("graph2_table")
+ .setDatasetType(DatasetType.MATERIALIZED_VIEW))
+ .build()))
+
+ // Verify both graphs exist in the session
+ val sessionHolder = getDefaultSessionHolder
+ val graph1 = sessionHolder.dataflowGraphRegistry.getDataflowGraphOrThrow(graphId1)
+ val graph2 = sessionHolder.dataflowGraphRegistry.getDataflowGraphOrThrow(graphId2)
+ // Check that both graphs have their datasets registered
+ assert(graph1.toDataflowGraph.tables.exists(_.identifier.table == "graph1_table"))
+ assert(graph2.toDataflowGraph.tables.exists(_.identifier.table == "graph2_table"))
+ }
+ }
+
+ test("dropping a dataflow graph removes it from session") {
+ withRawBlockingStub { implicit stub =>
+ val graphId = createDataflowGraph
+
+ // Register a dataset
+ sendPlan(
+ buildPlanFromPipelineCommand(
+ PipelineCommand
+ .newBuilder()
+ .setDefineDataset(
+ DefineDataset
+ .newBuilder()
+ .setDataflowGraphId(graphId)
+ .setDatasetName("test_table")
+ .setDatasetType(DatasetType.MATERIALIZED_VIEW))
+ .build()))
+
+ // Verify the graph exists
+ val sessionHolder = getDefaultSessionHolder
+ val graphsBefore = sessionHolder.dataflowGraphRegistry.getAllDataflowGraphs
+ assert(graphsBefore.size == 1)
+
+ // Drop the graph
+ sendPlan(
+ buildPlanFromPipelineCommand(
+ PipelineCommand
+ .newBuilder()
+ .setDropDataflowGraph(PipelineCommand.DropDataflowGraph
+ .newBuilder()
+ .setDataflowGraphId(graphId))
+ .build()))
+
+ // Verify the graph is removed
+ val graphsAfter = sessionHolder.dataflowGraphRegistry.getAllDataflowGraphs
+ assert(graphsAfter.isEmpty, "Graph should be removed after drop")
+ }
+ }
}
diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/SparkDeclarativePipelinesServerTest.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/SparkDeclarativePipelinesServerTest.scala
index ea4cc5f3aba55..a31883677f92a 100644
--- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/SparkDeclarativePipelinesServerTest.scala
+++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/SparkDeclarativePipelinesServerTest.scala
@@ -17,10 +17,13 @@
package org.apache.spark.sql.connect.pipelines
+import scala.collection.mutable.ArrayBuffer
+
import org.apache.spark.connect.{proto => sc}
+import org.apache.spark.connect.proto.{PipelineCommand, PipelineEvent}
import org.apache.spark.sql.connect.{SparkConnectServerTest, SparkConnectTestUtils}
import org.apache.spark.sql.connect.planner.SparkConnectPlanner
-import org.apache.spark.sql.connect.service.{SessionKey, SparkConnectService}
+import org.apache.spark.sql.connect.service.{SessionHolder, SessionKey, SparkConnectService}
import org.apache.spark.sql.pipelines.utils.PipelineTest
class SparkDeclarativePipelinesServerTest extends SparkConnectServerTest {
@@ -28,12 +31,20 @@ class SparkDeclarativePipelinesServerTest extends SparkConnectServerTest {
override def afterEach(): Unit = {
SparkConnectService.sessionManager
.getIsolatedSessionIfPresent(SessionKey(defaultUserId, defaultSessionId))
- .foreach(_.removeAllPipelineExecutions())
- DataflowGraphRegistry.dropAllDataflowGraphs()
+ .foreach(s => {
+ s.removeAllPipelineExecutions()
+ s.dataflowGraphRegistry.dropAllDataflowGraphs()
+ })
PipelineTest.cleanupMetastore(spark)
super.afterEach()
}
+ // Helper method to get the session holder
+ protected def getDefaultSessionHolder: SessionHolder = {
+ SparkConnectService.sessionManager
+ .getIsolatedSession(SessionKey(defaultUserId, defaultSessionId), None)
+ }
+
def buildPlanFromPipelineCommand(command: sc.PipelineCommand): sc.Plan = {
sc.Plan
.newBuilder()
@@ -125,15 +136,27 @@ class SparkDeclarativePipelinesServerTest extends SparkConnectServerTest {
def createPlanner(): SparkConnectPlanner =
new SparkConnectPlanner(SparkConnectTestUtils.createDummySessionHolder(spark))
- def startPipelineAndWaitForCompletion(graphId: String): Unit = {
+ def startPipelineAndWaitForCompletion(graphId: String): ArrayBuffer[PipelineEvent] = {
+ val defaultStartRunCommand =
+ PipelineCommand.StartRun.newBuilder().setDataflowGraphId(graphId).build()
+ startPipelineAndWaitForCompletion(defaultStartRunCommand)
+ }
+
+ def startPipelineAndWaitForCompletion(
+ startRunCommand: PipelineCommand.StartRun): ArrayBuffer[PipelineEvent] = {
withClient { client =>
- val startRunRequest = buildStartRunPlan(
- sc.PipelineCommand.StartRun.newBuilder().setDataflowGraphId(graphId).build())
+ val capturedEvents = new ArrayBuffer[PipelineEvent]()
+ val startRunRequest = buildStartRunPlan(startRunCommand)
val responseIterator = client.execute(startRunRequest)
// The response iterator will be closed when the pipeline is completed.
while (responseIterator.hasNext) {
- responseIterator.next()
+ val response = responseIterator.next()
+ if (response.hasPipelineEventResult) {
+ capturedEvents.append(response.getPipelineEventResult.getEvent)
+ }
}
+ return capturedEvents
}
+ ArrayBuffer.empty[PipelineEvent]
}
}
diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/TestPipelineDefinition.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/TestPipelineDefinition.scala
index 3efc71ba7d259..c31aec0b7a5e6 100644
--- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/TestPipelineDefinition.scala
+++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/pipelines/TestPipelineDefinition.scala
@@ -58,9 +58,8 @@ class TestPipelineDefinition(graphId: String) {
.setDataflowGraphId(graphId)
.setFlowName(name)
.setTargetDatasetName(name)
- .setPlan(q)
+ .setRelation(q)
.putAllSqlConf(sparkConf.asJava)
- .setOnce(false)
.build()
}
}
@@ -98,9 +97,8 @@ class TestPipelineDefinition(graphId: String) {
.setDataflowGraphId(graphId)
.setFlowName(name)
.setTargetDatasetName(name)
- .setPlan(query)
+ .setRelation(query)
.putAllSqlConf(sparkConf.asJava)
- .setOnce(false)
.build()
}
@@ -125,9 +123,8 @@ class TestPipelineDefinition(graphId: String) {
.setDataflowGraphId(graphId)
.setFlowName(name)
.setTargetDatasetName(destinationName)
- .setPlan(query)
+ .setRelation(query)
.putAllSqlConf(sparkConf.asJava)
- .setOnce(once)
.build()
}
}
diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/LiteralExpressionProtoConverterSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/LiteralExpressionProtoConverterSuite.scala
index 79ef8decb310a..0af181e4be1a7 100644
--- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/LiteralExpressionProtoConverterSuite.scala
+++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/LiteralExpressionProtoConverterSuite.scala
@@ -19,15 +19,244 @@ package org.apache.spark.sql.connect.planner
import org.scalatest.funsuite.AnyFunSuite // scalastyle:ignore funsuite
+import org.apache.spark.connect.proto
import org.apache.spark.sql.connect.common.LiteralValueProtoConverter
+import org.apache.spark.sql.connect.common.LiteralValueProtoConverter.ToLiteralProtoOptions
+import org.apache.spark.sql.connect.planner.LiteralExpressionProtoConverter
+import org.apache.spark.sql.types._
class LiteralExpressionProtoConverterSuite extends AnyFunSuite { // scalastyle:ignore funsuite
+ private def toLiteralProto(v: Any): proto.Expression.Literal = {
+ LiteralValueProtoConverter
+ .toLiteralProtoWithOptions(
+ v,
+ None,
+ ToLiteralProtoOptions(useDeprecatedDataTypeFields = false))
+ }
+
+ private def toLiteralProto(v: Any, t: DataType): proto.Expression.Literal = {
+ LiteralValueProtoConverter
+ .toLiteralProtoWithOptions(
+ v,
+ Some(t),
+ ToLiteralProtoOptions(useDeprecatedDataTypeFields = false))
+ }
+
test("basic proto value and catalyst value conversion") {
val values = Array(null, true, 1.toByte, 1.toShort, 1, 1L, 1.1d, 1.1f, "spark")
for (v <- values) {
- assertResult(v)(
- LiteralValueProtoConverter.toCatalystValue(LiteralValueProtoConverter.toLiteralProto(v)))
+ assertResult(v)(LiteralValueProtoConverter.toCatalystValue(toLiteralProto(v)))
+ }
+ }
+
+ Seq(
+ (
+ (1, "string", true),
+ StructType(
+ Seq(
+ StructField("a", IntegerType),
+ StructField("b", StringType),
+ StructField("c", BooleanType)))),
+ (
+ Array((1, "string", true), (2, "string", false), (3, "string", true)),
+ ArrayType(
+ StructType(
+ Seq(
+ StructField("a", IntegerType),
+ StructField("b", StringType),
+ StructField("c", BooleanType))))),
+ (
+ (1, (2, 3)),
+ StructType(
+ Seq(
+ StructField("a", IntegerType),
+ StructField(
+ "b",
+ StructType(
+ Seq(StructField("c", IntegerType), StructField("d", IntegerType)))))))).zipWithIndex
+ .foreach { case ((v, t), idx) =>
+ test(s"complex proto value and catalyst value conversion #$idx") {
+ assertResult(v)(
+ LiteralValueProtoConverter.toCatalystValue(
+ LiteralValueProtoConverter.toLiteralProtoWithOptions(
+ v,
+ Some(t),
+ ToLiteralProtoOptions(useDeprecatedDataTypeFields = false))))
+ }
+
+ test(s"complex proto value and catalyst value conversion #$idx - backward compatibility") {
+ assertResult(v)(
+ LiteralValueProtoConverter.toCatalystValue(
+ LiteralValueProtoConverter.toLiteralProtoWithOptions(
+ v,
+ Some(t),
+ ToLiteralProtoOptions(useDeprecatedDataTypeFields = true))))
+ }
}
+
+ test("backward compatibility for array literal proto") {
+ // Test the old way of defining arrays with elementType field and elements
+ val arrayProto = proto.Expression.Literal.Array
+ .newBuilder()
+ .setElementType(
+ proto.DataType
+ .newBuilder()
+ .setInteger(proto.DataType.Integer.newBuilder())
+ .build())
+ .addElements(toLiteralProto(1))
+ .addElements(toLiteralProto(2))
+ .addElements(toLiteralProto(3))
+ .build()
+
+ val literalProto = proto.Expression.Literal.newBuilder().setArray(arrayProto).build()
+ val literal = LiteralExpressionProtoConverter.toCatalystExpression(literalProto)
+ assert(literal.dataType.isInstanceOf[ArrayType])
+ assert(literal.dataType.asInstanceOf[ArrayType].elementType == IntegerType)
+ // The containsNull field is always set to true when using the old way of defining arrays.
+ assert(literal.dataType.asInstanceOf[ArrayType].containsNull)
+
+ val arrayData = literal.value.asInstanceOf[org.apache.spark.sql.catalyst.util.ArrayData]
+ assert(arrayData.numElements() == 3)
+ assert(arrayData.getInt(0) == 1)
+ assert(arrayData.getInt(1) == 2)
+ assert(arrayData.getInt(2) == 3)
+ }
+
+ test("backward compatibility for map literal proto") {
+ // Test the old way of defining maps with keyType and valueType fields
+ val mapProto = proto.Expression.Literal.Map
+ .newBuilder()
+ .setKeyType(
+ proto.DataType
+ .newBuilder()
+ .setString(proto.DataType.String.newBuilder())
+ .build())
+ .setValueType(
+ proto.DataType
+ .newBuilder()
+ .setInteger(proto.DataType.Integer.newBuilder())
+ .build())
+ .addKeys(toLiteralProto("a"))
+ .addKeys(toLiteralProto("b"))
+ .addValues(toLiteralProto(1))
+ .addValues(toLiteralProto(2))
+ .build()
+
+ val literalProto = proto.Expression.Literal.newBuilder().setMap(mapProto).build()
+ val literal = LiteralExpressionProtoConverter.toCatalystExpression(literalProto)
+ assert(literal.dataType.isInstanceOf[MapType])
+ assert(literal.dataType.asInstanceOf[MapType].keyType == StringType)
+ assert(literal.dataType.asInstanceOf[MapType].valueType == IntegerType)
+ // The valueContainsNull field is always set to true when using the old way of defining maps.
+ assert(literal.dataType.asInstanceOf[MapType].valueContainsNull)
+
+ val mapData = literal.value.asInstanceOf[org.apache.spark.sql.catalyst.util.MapData]
+ assert(mapData.numElements() == 2)
+ val keys = mapData.keyArray()
+ val values = mapData.valueArray()
+ assert(keys.getUTF8String(0).toString == "a")
+ assert(values.getInt(0) == 1)
+ assert(keys.getUTF8String(1).toString == "b")
+ assert(values.getInt(1) == 2)
+ }
+
+ test("backward compatibility for struct literal proto") {
+ // Test the old way of defining structs with structType field and elements
+ val structTypeProto = proto.DataType.Struct
+ .newBuilder()
+ .addFields(
+ proto.DataType.StructField
+ .newBuilder()
+ .setName("a")
+ .setDataType(proto.DataType
+ .newBuilder()
+ .setInteger(proto.DataType.Integer.newBuilder())
+ .build())
+ .setNullable(true)
+ .build())
+ .addFields(
+ proto.DataType.StructField
+ .newBuilder()
+ .setName("b")
+ .setDataType(proto.DataType
+ .newBuilder()
+ .setString(proto.DataType.String.newBuilder())
+ .build())
+ .setNullable(false)
+ .build())
+ .build()
+
+ val structProto = proto.Expression.Literal.Struct
+ .newBuilder()
+ .setStructType(proto.DataType.newBuilder().setStruct(structTypeProto).build())
+ .addElements(LiteralValueProtoConverter.toLiteralProto(1))
+ .addElements(LiteralValueProtoConverter.toLiteralProto("test"))
+ .build()
+
+ val result = LiteralValueProtoConverter.toCatalystStruct(structProto)
+ val resultType = LiteralValueProtoConverter.getProtoStructType(structProto)
+
+ // Verify the result is a tuple with correct values
+ assert(result.isInstanceOf[Product])
+ val product = result.asInstanceOf[Product]
+ assert(product.productArity == 2)
+ assert(product.productElement(0) == 1)
+ assert(product.productElement(1) == "test")
+
+ // Verify the returned struct type matches the original
+ assert(resultType.getFieldsCount == 2)
+ assert(resultType.getFields(0).getName == "a")
+ assert(resultType.getFields(0).getDataType.hasInteger)
+ assert(resultType.getFields(0).getNullable)
+ assert(resultType.getFields(1).getName == "b")
+ assert(resultType.getFields(1).getDataType.hasString)
+ assert(!resultType.getFields(1).getNullable)
+ }
+
+ test("data types of struct fields are not set for inferable types") {
+ val literalProto = toLiteralProto(
+ (1, 2.0, true, (1, 2)),
+ StructType(
+ Seq(
+ StructField("a", IntegerType),
+ StructField("b", DoubleType),
+ StructField("c", BooleanType),
+ StructField(
+ "d",
+ StructType(Seq(StructField("e", IntegerType), StructField("f", IntegerType)))))))
+ assert(!literalProto.getStruct.getDataTypeStruct.getFieldsList.get(0).hasDataType)
+ assert(!literalProto.getStruct.getDataTypeStruct.getFieldsList.get(1).hasDataType)
+ assert(!literalProto.getStruct.getDataTypeStruct.getFieldsList.get(2).hasDataType)
+ assert(!literalProto.getStruct.getDataTypeStruct.getFieldsList.get(3).hasDataType)
+ }
+
+ test("data types of struct fields are set for non-inferable types") {
+ val literalProto = toLiteralProto(
+ ("string", Decimal(1)),
+ StructType(Seq(StructField("a", StringType), StructField("b", DecimalType(10, 2)))))
+ assert(literalProto.getStruct.getDataTypeStruct.getFieldsList.get(0).hasDataType)
+ assert(literalProto.getStruct.getDataTypeStruct.getFieldsList.get(1).hasDataType)
+ }
+
+ test("nullable and metadata fields are set for struct literal proto") {
+ val literalProto = toLiteralProto(
+ ("string", Decimal(1)),
+ StructType(Seq(
+ StructField("a", StringType, nullable = true, Metadata.fromJson("""{"key": "value"}""")),
+ StructField("b", DecimalType(10, 2), nullable = false))))
+ val structFields = literalProto.getStruct.getDataTypeStruct.getFieldsList
+ assert(structFields.get(0).getNullable)
+ assert(structFields.get(0).hasMetadata)
+ assert(structFields.get(0).getMetadata == """{"key":"value"}""")
+ assert(!structFields.get(1).getNullable)
+ assert(!structFields.get(1).hasMetadata)
+
+ val structTypeProto = LiteralValueProtoConverter.getProtoStructType(literalProto.getStruct)
+ assert(structTypeProto.getFieldsList.get(0).getNullable)
+ assert(structTypeProto.getFieldsList.get(0).hasMetadata)
+ assert(structTypeProto.getFieldsList.get(0).getMetadata == """{"key":"value"}""")
+ assert(!structTypeProto.getFieldsList.get(1).getNullable)
+ assert(!structTypeProto.getFieldsList.get(1).hasMetadata)
}
}
diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectPlannerSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectPlannerSuite.scala
index 126c4d7a3cfe5..16cdd7da82799 100644
--- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectPlannerSuite.scala
+++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectPlannerSuite.scala
@@ -38,7 +38,7 @@ import org.apache.spark.sql.connect.common.InvalidPlanInput
import org.apache.spark.sql.connect.common.LiteralValueProtoConverter.toLiteralProto
import org.apache.spark.sql.execution.arrow.ArrowConverters
import org.apache.spark.sql.test.SharedSparkSession
-import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType}
+import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType, TimeType}
import org.apache.spark.unsafe.types.UTF8String
/**
@@ -961,4 +961,49 @@ class SparkConnectPlannerSuite extends SparkFunSuite with SparkConnectPlanTest {
assert(plan.aggregateExpressions.forall(aggregateExpression =>
!aggregateExpression.containsPattern(TreePattern.UNRESOLVED_ORDINAL)))
}
+
+ test("Time literal") {
+ val project = proto.Project.newBuilder
+ .addExpressions(
+ proto.Expression.newBuilder
+ .setLiteral(proto.Expression.Literal.newBuilder.setTime(
+ proto.Expression.Literal.newBuilder.getTimeBuilder
+ .setNano(86399999999999L)
+ .setPrecision(TimeType.MIN_PRECISION)))
+ .build())
+ .addExpressions(
+ proto.Expression.newBuilder
+ .setLiteral(
+ proto.Expression.Literal.newBuilder.setTime(
+ proto.Expression.Literal.newBuilder.getTimeBuilder
+ .setNano(86399999999999L)
+ .setPrecision(TimeType.MAX_PRECISION)))
+ .build())
+ .addExpressions(
+ proto.Expression.newBuilder
+ .setLiteral(
+ proto.Expression.Literal.newBuilder.setTime(
+ proto.Expression.Literal.newBuilder.getTimeBuilder
+ .setNano(86399999999999L)
+ .setPrecision(TimeType.DEFAULT_PRECISION)))
+ .build())
+ .addExpressions(proto.Expression.newBuilder
+ .setLiteral(proto.Expression.Literal.newBuilder.setTime(
+ proto.Expression.Literal.newBuilder.getTimeBuilder.setNano(86399999999999L)))
+ .build())
+ .build()
+
+ val logical = transform(proto.Relation.newBuilder.setProject(project).build())
+ val df = Dataset.ofRows(spark, logical)
+ assertResult(df.schema.fields(0).dataType)(TimeType(TimeType.MIN_PRECISION))
+ assertResult(df.schema.fields(1).dataType)(TimeType(TimeType.MAX_PRECISION))
+ assertResult(df.schema.fields(2).dataType)(TimeType(TimeType.DEFAULT_PRECISION))
+ assertResult(df.schema.fields(3).dataType)(TimeType(TimeType.DEFAULT_PRECISION))
+ assertResult(df.collect()(0).toString)(
+ InternalRow(
+ "23:59:59.999999999",
+ "23:59:59.999999999",
+ "23:59:59.999999999",
+ "23:59:59.999999999").toString)
+ }
}
diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala
index 5c43715d2dd13..7b734f93e595e 100644
--- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala
+++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala
@@ -40,7 +40,7 @@ import org.apache.spark.sql.connect.dsl.MockRemoteSession
import org.apache.spark.sql.connect.dsl.commands._
import org.apache.spark.sql.connect.dsl.expressions._
import org.apache.spark.sql.connect.dsl.plans._
-import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryTableCatalog, TableCatalog}
+import org.apache.spark.sql.connector.catalog.{Column => ColumnV2, Identifier, InMemoryTableCatalog, TableCatalog}
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.CatalogHelper
import org.apache.spark.sql.execution.arrow.ArrowConverters
import org.apache.spark.sql.functions._
@@ -822,7 +822,9 @@ class SparkConnectProtoSuite extends PlanTest with SparkConnectPlanTest {
.asTableCatalog
.loadTable(Identifier.of(Array(), "table_name"))
assert(table.name === "testcat.table_name")
- assert(table.schema === new StructType().add("id", LongType).add("data", StringType))
+ assert(
+ table.columns sameElements
+ Array(ColumnV2.create("id", LongType), ColumnV2.create("data", StringType)))
assert(table.partitioning.isEmpty)
assert(table.properties === (Map("provider" -> "foo") ++ defaultOwnership).asJava)
}
diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectServiceSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectServiceSuite.scala
index 4fb57b736185a..c5567e0d218c3 100644
--- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectServiceSuite.scala
+++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectServiceSuite.scala
@@ -920,7 +920,11 @@ class SparkConnectServiceSuite
}
def onError(throwable: Throwable): Unit = {
assert(executeHolder.eventsManager.hasCanceled.isEmpty)
- assert(executeHolder.eventsManager.hasError.isDefined)
+ Eventually.eventually(EVENT_WAIT_TIMEOUT) {
+ assert(
+ executeHolder.eventsManager.hasError.isDefined,
+ s"Error has not been recorded in events manager within $EVENT_WAIT_TIMEOUT")
+ }
}
def onCompleted(producedRowCount: Option[Long] = None): Unit = {
assert(executeHolder.eventsManager.getProducedRowCount == producedRowCount)
diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/AddArtifactsHandlerSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/AddArtifactsHandlerSuite.scala
index a158ca9fad8ce..6cc5daadfddd7 100644
--- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/AddArtifactsHandlerSuite.scala
+++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/AddArtifactsHandlerSuite.scala
@@ -32,6 +32,7 @@ import io.grpc.StatusRuntimeException
import io.grpc.protobuf.StatusProto
import io.grpc.stub.StreamObserver
+import org.apache.spark.SparkRuntimeException
import org.apache.spark.connect.proto
import org.apache.spark.connect.proto.{AddArtifactsRequest, AddArtifactsResponse}
import org.apache.spark.sql.connect.ResourceHelper
@@ -43,6 +44,7 @@ class AddArtifactsHandlerSuite extends SharedSparkSession with ResourceHelper {
private val CHUNK_SIZE: Int = 32 * 1024
private val sessionId = UUID.randomUUID.toString()
+ private val sessionKey = SessionKey("c1", sessionId)
class DummyStreamObserver(p: Promise[AddArtifactsResponse])
extends StreamObserver[AddArtifactsResponse] {
@@ -51,17 +53,31 @@ class AddArtifactsHandlerSuite extends SharedSparkSession with ResourceHelper {
override def onCompleted(): Unit = {}
}
- class TestAddArtifactsHandler(responseObserver: StreamObserver[AddArtifactsResponse])
+ class TestAddArtifactsHandler(
+ responseObserver: StreamObserver[AddArtifactsResponse],
+ throwIfArtifactExists: Boolean = false)
extends SparkConnectAddArtifactsHandler(responseObserver) {
// Stop the staged artifacts from being automatically deleted
override protected def cleanUpStagedArtifacts(): Unit = {}
private val finalArtifacts = mutable.Buffer.empty[String]
+ private val artifactChecksums: mutable.Map[String, Long] = mutable.Map.empty
// Record the artifacts that are sent out for final processing.
override protected def addStagedArtifactToArtifactManager(artifact: StagedArtifact): Unit = {
+ // Throw if artifact already exists and has different checksum
+ // This mocks the behavior of ArtifactManager.addArtifact without comparing the entire file
+ if (throwIfArtifactExists
+ && finalArtifacts.contains(artifact.name)
+ && artifact.getCrc != artifactChecksums(artifact.name)) {
+ throw new SparkRuntimeException(
+ "ARTIFACT_ALREADY_EXISTS",
+ Map("normalizedRemoteRelativePath" -> artifact.name))
+ }
+
finalArtifacts.append(artifact.name)
+ artifactChecksums += (artifact.name -> artifact.getCrc)
}
def getFinalArtifacts: Seq[String] = finalArtifacts.toSeq
@@ -418,4 +434,79 @@ class AddArtifactsHandlerSuite extends SharedSparkSession with ResourceHelper {
}
}
+ def addSingleChunkArtifact(
+ handler: SparkConnectAddArtifactsHandler,
+ sessionKey: SessionKey,
+ name: String,
+ artifactPath: Path): Unit = {
+ val dataChunks = getDataChunks(artifactPath)
+ assert(dataChunks.size == 1)
+ val bytes = dataChunks.head
+ val context = proto.UserContext
+ .newBuilder()
+ .setUserId(sessionKey.userId)
+ .build()
+ val fileNameNoExtension = artifactPath.getFileName.toString.split('.').head
+ val singleChunkArtifact = proto.AddArtifactsRequest.SingleChunkArtifact
+ .newBuilder()
+ .setName(name)
+ .setData(
+ proto.AddArtifactsRequest.ArtifactChunk
+ .newBuilder()
+ .setData(bytes)
+ .setCrc(getCrcValues(crcPath.resolve(fileNameNoExtension + ".txt")).head)
+ .build())
+ .build()
+
+ val singleChunkArtifactRequest = AddArtifactsRequest
+ .newBuilder()
+ .setSessionId(sessionKey.sessionId)
+ .setUserContext(context)
+ .setBatch(
+ proto.AddArtifactsRequest.Batch.newBuilder().addArtifacts(singleChunkArtifact).build())
+ .build()
+
+ handler.onNext(singleChunkArtifactRequest)
+ }
+
+ test("All artifacts are added, even if some fail") {
+ val promise = Promise[AddArtifactsResponse]()
+ val handler =
+ new TestAddArtifactsHandler(new DummyStreamObserver(promise), throwIfArtifactExists = true)
+ try {
+ val name1 = "jars/dummy1.jar"
+ val name2 = "jars/dummy2.jar"
+ val name3 = "jars/dummy3.jar"
+
+ val artifactPath1 = inputFilePath.resolve("smallClassFile.class")
+ val artifactPath2 = inputFilePath.resolve("smallJar.jar")
+
+ assume(artifactPath1.toFile.exists)
+ addSingleChunkArtifact(handler, sessionKey, name1, artifactPath1)
+ addSingleChunkArtifact(handler, sessionKey, name3, artifactPath1)
+
+ val e = intercept[StatusRuntimeException] {
+ addSingleChunkArtifact(handler, sessionKey, name1, artifactPath2)
+ addSingleChunkArtifact(handler, sessionKey, name2, artifactPath1)
+ addSingleChunkArtifact(handler, sessionKey, name3, artifactPath2)
+ handler.onCompleted()
+ }
+
+ // Both artifacts should be added, despite exception
+ assert(handler.getFinalArtifacts.contains(name1))
+ assert(handler.getFinalArtifacts.contains(name2))
+ assert(handler.getFinalArtifacts.contains(name3))
+
+ assert(e.getStatus.getCode == Code.INTERNAL)
+ val statusProto = StatusProto.fromThrowable(e)
+ assert(statusProto.getDetailsCount == 1)
+ val details = statusProto.getDetails(0)
+ val info = details.unpack(classOf[ErrorInfo])
+
+ assert(e.getMessage.contains("ARTIFACT_ALREADY_EXISTS"))
+ assert(info.getMetadataMap().get("messageParameters").contains(name1))
+ } finally {
+ handler.forceCleanUp()
+ }
+ }
}
diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/ExecuteEventsManagerSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/ExecuteEventsManagerSuite.scala
index 66dd3680d93a2..a17c76ae95286 100644
--- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/ExecuteEventsManagerSuite.scala
+++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/ExecuteEventsManagerSuite.scala
@@ -24,7 +24,7 @@ import scala.util.matching.Regex
import org.mockito.Mockito._
import org.scalatestplus.mockito.MockitoSugar
-import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite}
+import org.apache.spark.{SparkContext, SparkFunSuite}
import org.apache.spark.connect.proto
import org.apache.spark.connect.proto.{ExecutePlanRequest, Plan, UserContext}
import org.apache.spark.scheduler.LiveListenerBus
@@ -51,7 +51,6 @@ class ExecuteEventsManagerSuite
val DEFAULT_SESSION_ID = UUID.randomUUID.toString
val DEFAULT_QUERY_ID = UUID.randomUUID.toString
val DEFAULT_CLIENT_TYPE = "clientType"
- val jsonProtocol = new JsonProtocol(new SparkConf())
test("SPARK-43923: post started") {
val events = setupEvents(ExecuteStatus.Pending)
@@ -72,8 +71,8 @@ class ExecuteEventsManagerSuite
.post(expectedEvent)
assert(
- jsonProtocol
- .sparkEventFromJson(jsonProtocol.sparkEventToJsonString(expectedEvent))
+ JsonProtocol
+ .sparkEventFromJson(JsonProtocol.sparkEventToJsonString(expectedEvent))
.isInstanceOf[SparkListenerConnectOperationStarted])
}
@@ -91,8 +90,8 @@ class ExecuteEventsManagerSuite
.post(expectedEvent)
assert(
- jsonProtocol
- .sparkEventFromJson(jsonProtocol.sparkEventToJsonString(expectedEvent))
+ JsonProtocol
+ .sparkEventFromJson(JsonProtocol.sparkEventToJsonString(expectedEvent))
.isInstanceOf[SparkListenerConnectOperationAnalyzed])
}
@@ -118,8 +117,8 @@ class ExecuteEventsManagerSuite
.post(expectedEvent)
assert(
- jsonProtocol
- .sparkEventFromJson(jsonProtocol.sparkEventToJsonString(expectedEvent))
+ JsonProtocol
+ .sparkEventFromJson(JsonProtocol.sparkEventToJsonString(expectedEvent))
.isInstanceOf[SparkListenerConnectOperationReadyForExecution])
}
@@ -134,8 +133,8 @@ class ExecuteEventsManagerSuite
.post(expectedEvent)
assert(
- jsonProtocol
- .sparkEventFromJson(jsonProtocol.sparkEventToJsonString(expectedEvent))
+ JsonProtocol
+ .sparkEventFromJson(JsonProtocol.sparkEventToJsonString(expectedEvent))
.isInstanceOf[SparkListenerConnectOperationCanceled])
}
@@ -152,8 +151,8 @@ class ExecuteEventsManagerSuite
.post(expectedEvent)
assert(
- jsonProtocol
- .sparkEventFromJson(jsonProtocol.sparkEventToJsonString(expectedEvent))
+ JsonProtocol
+ .sparkEventFromJson(JsonProtocol.sparkEventToJsonString(expectedEvent))
.isInstanceOf[SparkListenerConnectOperationFailed])
}
@@ -168,8 +167,8 @@ class ExecuteEventsManagerSuite
.post(expectedEvent)
assert(
- jsonProtocol
- .sparkEventFromJson(jsonProtocol.sparkEventToJsonString(expectedEvent))
+ JsonProtocol
+ .sparkEventFromJson(JsonProtocol.sparkEventToJsonString(expectedEvent))
.isInstanceOf[SparkListenerConnectOperationFinished])
}
@@ -209,8 +208,8 @@ class ExecuteEventsManagerSuite
.post(expectedEvent)
assert(
- jsonProtocol
- .sparkEventFromJson(jsonProtocol.sparkEventToJsonString(expectedEvent))
+ JsonProtocol
+ .sparkEventFromJson(JsonProtocol.sparkEventToJsonString(expectedEvent))
.isInstanceOf[SparkListenerConnectOperationClosed])
}
diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionHolderSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionHolderSuite.scala
index 159b29ce6b8e7..a110b0164f199 100644
--- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionHolderSuite.scala
+++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionHolderSuite.scala
@@ -401,19 +401,23 @@ class SparkConnectSessionHolderSuite extends SharedSparkSession {
test("Test session plan cache - disabled") {
val sessionHolder = SparkConnectTestUtils.createDummySessionHolder(spark)
// Disable plan cache of the session
- sessionHolder.session.conf.set(Connect.CONNECT_SESSION_PLAN_CACHE_ENABLED.key, false)
- val planner = new SparkConnectPlanner(sessionHolder)
+ try {
+ sessionHolder.session.conf.set(Connect.CONNECT_SESSION_PLAN_CACHE_ENABLED.key, false)
+ val planner = new SparkConnectPlanner(sessionHolder)
- val query = buildRelation("select 1")
+ val query = buildRelation("select 1")
- // If cachePlan is false, the cache is still empty.
- // Although the cache is created as cache size is greater than zero, it won't be used.
- planner.transformRelation(query, cachePlan = false)
- assertPlanCache(sessionHolder, Some(Set()))
+ // If cachePlan is false, the cache is still empty.
+ // Although the cache is created as cache size is greater than zero, it won't be used.
+ planner.transformRelation(query, cachePlan = false)
+ assertPlanCache(sessionHolder, Some(Set()))
- // Even if we specify "cachePlan = true", the cache is still empty.
- planner.transformRelation(query, cachePlan = true)
- assertPlanCache(sessionHolder, Some(Set()))
+ // Even if we specify "cachePlan = true", the cache is still empty.
+ planner.transformRelation(query, cachePlan = true)
+ assertPlanCache(sessionHolder, Some(Set()))
+ } finally {
+ sessionHolder.session.conf.set(Connect.CONNECT_SESSION_PLAN_CACHE_ENABLED, true)
+ }
}
test("Test duplicate operation IDs") {
@@ -440,4 +444,40 @@ class SparkConnectSessionHolderSuite extends SharedSparkSession {
sessionHolder.getPipelineExecution(graphId).isEmpty,
"pipeline execution was not removed")
}
+
+ gridTest("Actively cache data source reads")(Seq(true, false)) { enabled =>
+ val sessionHolder = SparkConnectTestUtils.createDummySessionHolder(spark)
+ val planner = new SparkConnectPlanner(sessionHolder)
+
+ val dataSourceRead = proto.Relation
+ .newBuilder()
+ .setRead(
+ proto.Read
+ .newBuilder()
+ .setDataSource(proto.Read.DataSource
+ .newBuilder()
+ .setSchema("col int")))
+ .setCommon(proto.RelationCommon.newBuilder().setPlanId(Random.nextLong()).build())
+ .build()
+ val dataSourceReadJoin = proto.Relation
+ .newBuilder()
+ .setJoin(
+ proto.Join
+ .newBuilder()
+ .setLeft(dataSourceRead)
+ .setRight(dataSourceRead)
+ .setJoinType(proto.Join.JoinType.JOIN_TYPE_CROSS))
+ .setCommon(proto.RelationCommon.newBuilder().setPlanId(Random.nextLong()).build())
+ .build()
+
+ sessionHolder.session.conf
+ .set(Connect.CONNECT_ALWAYS_CACHE_DATA_SOURCE_READS_ENABLED, enabled)
+ planner.transformRelation(dataSourceReadJoin, cachePlan = true)
+ val expected = if (enabled) {
+ Set(dataSourceReadJoin, dataSourceRead)
+ } else {
+ Set(dataSourceReadJoin)
+ }
+ assertPlanCache(sessionHolder, Some(expected))
+ }
}
diff --git a/sql/core/benchmarks/AggregateBenchmark-jdk21-results.txt b/sql/core/benchmarks/AggregateBenchmark-jdk21-results.txt
index 186593aff6bbc..86de9e21ade3f 100644
--- a/sql/core/benchmarks/AggregateBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/AggregateBenchmark-jdk21-results.txt
@@ -2,147 +2,147 @@
aggregate without grouping
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
agg w/o group: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-agg w/o group wholestage off 35437 35937 707 59.2 16.9 1.0X
-agg w/o group wholestage on 2857 2862 6 734.0 1.4 12.4X
+agg w/o group wholestage off 35386 35792 574 59.3 16.9 1.0X
+agg w/o group wholestage on 2850 2855 4 735.8 1.4 12.4X
================================================================================================
stat functions
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
stddev: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-stddev wholestage off 4410 4433 34 23.8 42.1 1.0X
-stddev wholestage on 982 987 4 106.8 9.4 4.5X
+stddev wholestage off 4380 4400 27 23.9 41.8 1.0X
+stddev wholestage on 982 989 7 106.8 9.4 4.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
kurtosis: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-kurtosis wholestage off 22289 22520 326 4.7 212.6 1.0X
-kurtosis wholestage on 988 996 6 106.2 9.4 22.6X
+kurtosis wholestage off 21731 21746 20 4.8 207.2 1.0X
+kurtosis wholestage on 990 998 6 105.9 9.4 21.9X
================================================================================================
aggregate with linear keys
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Aggregate w keys: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-codegen = F 6846 6857 16 12.3 81.6 1.0X
-codegen = T, hashmap = F 4326 4415 102 19.4 51.6 1.6X
-codegen = T, row-based hashmap = T 1180 1194 12 71.1 14.1 5.8X
-codegen = T, vectorized hashmap = T 804 843 39 104.3 9.6 8.5X
+codegen = F 6421 6468 66 13.1 76.5 1.0X
+codegen = T, hashmap = F 4050 4075 23 20.7 48.3 1.6X
+codegen = T, row-based hashmap = T 1229 1239 10 68.2 14.7 5.2X
+codegen = T, vectorized hashmap = T 812 824 12 103.3 9.7 7.9X
================================================================================================
aggregate with randomized keys
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Aggregate w keys: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-codegen = F 7511 7796 403 11.2 89.5 1.0X
-codegen = T, hashmap = F 4800 4868 67 17.5 57.2 1.6X
-codegen = T, row-based hashmap = T 1661 1677 11 50.5 19.8 4.5X
-codegen = T, vectorized hashmap = T 1077 1159 121 77.9 12.8 7.0X
+codegen = F 7268 7292 33 11.5 86.6 1.0X
+codegen = T, hashmap = F 4630 4645 18 18.1 55.2 1.6X
+codegen = T, row-based hashmap = T 1641 1645 5 51.1 19.6 4.4X
+codegen = T, vectorized hashmap = T 1066 1115 73 78.7 12.7 6.8X
================================================================================================
aggregate with string key
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Aggregate w string key: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-codegen = F 2438 2457 26 8.6 116.3 1.0X
-codegen = T, hashmap = F 1762 1770 7 11.9 84.0 1.4X
-codegen = T, row-based hashmap = T 1028 1044 11 20.4 49.0 2.4X
-codegen = T, vectorized hashmap = T 803 817 18 26.1 38.3 3.0X
+codegen = F 2524 2549 35 8.3 120.3 1.0X
+codegen = T, hashmap = F 1515 1540 41 13.8 72.2 1.7X
+codegen = T, row-based hashmap = T 1059 1068 6 19.8 50.5 2.4X
+codegen = T, vectorized hashmap = T 823 825 3 25.5 39.2 3.1X
================================================================================================
aggregate with decimal key
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Aggregate w decimal key: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-codegen = F 2272 2276 6 9.2 108.3 1.0X
-codegen = T, hashmap = F 1343 1356 18 15.6 64.1 1.7X
-codegen = T, row-based hashmap = T 456 481 19 46.0 21.7 5.0X
-codegen = T, vectorized hashmap = T 333 336 4 63.0 15.9 6.8X
+codegen = F 2169 2174 6 9.7 103.4 1.0X
+codegen = T, hashmap = F 1349 1363 20 15.5 64.3 1.6X
+codegen = T, row-based hashmap = T 470 483 9 44.6 22.4 4.6X
+codegen = T, vectorized hashmap = T 313 317 2 67.0 14.9 6.9X
================================================================================================
aggregate with multiple key types
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Aggregate w multiple keys: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-codegen = F 4329 4360 43 4.8 206.4 1.0X
-codegen = T, hashmap = F 2337 2347 14 9.0 111.5 1.9X
-codegen = T, row-based hashmap = T 1749 1752 4 12.0 83.4 2.5X
-codegen = T, vectorized hashmap = T 1628 1632 6 12.9 77.6 2.7X
+codegen = F 4283 4287 7 4.9 204.2 1.0X
+codegen = T, hashmap = F 2291 2301 13 9.2 109.3 1.9X
+codegen = T, row-based hashmap = T 1750 1756 7 12.0 83.5 2.4X
+codegen = T, vectorized hashmap = T 1584 1595 15 13.2 75.6 2.7X
================================================================================================
max function bytecode size of wholestagecodegen
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
max function bytecode size: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-codegen = F 359 370 9 1.8 548.0 1.0X
-codegen = T, hugeMethodLimit = 10000 142 160 16 4.6 216.6 2.5X
-codegen = T, hugeMethodLimit = 1500 136 146 11 4.8 207.3 2.6X
+codegen = F 365 391 24 1.8 557.4 1.0X
+codegen = T, hugeMethodLimit = 10000 141 155 13 4.6 215.6 2.6X
+codegen = T, hugeMethodLimit = 1500 134 146 20 4.9 205.0 2.7X
================================================================================================
cube
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
cube: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-cube wholestage off 2070 2141 100 2.5 394.8 1.0X
-cube wholestage on 1094 1113 12 4.8 208.6 1.9X
+cube wholestage off 2067 2088 31 2.5 394.2 1.0X
+cube wholestage on 1072 1084 9 4.9 204.4 1.9X
================================================================================================
hash and BytesToBytesMap
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
BytesToBytesMap: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-UnsafeRowhash 146 147 2 143.2 7.0 1.0X
-murmur3 hash 54 54 0 390.9 2.6 2.7X
-fast hash 24 24 0 882.5 1.1 6.2X
-arrayEqual 137 137 0 153.2 6.5 1.1X
-Java HashMap (Long) 60 70 7 346.9 2.9 2.4X
-Java HashMap (two ints) 81 84 2 259.2 3.9 1.8X
-Java HashMap (UnsafeRow) 531 533 1 39.5 25.3 0.3X
-LongToUnsafeRowMap (opt=false) 351 352 1 59.7 16.7 0.4X
-LongToUnsafeRowMap (opt=true) 74 76 1 283.3 3.5 2.0X
-BytesToBytesMap (off Heap) 413 417 5 50.8 19.7 0.4X
-BytesToBytesMap (on Heap) 428 433 5 49.0 20.4 0.3X
-Aggregate HashMap 30 31 2 689.0 1.5 4.8X
+UnsafeRowhash 146 147 1 143.2 7.0 1.0X
+murmur3 hash 54 54 0 390.8 2.6 2.7X
+fast hash 24 24 0 882.6 1.1 6.2X
+arrayEqual 137 137 0 153.1 6.5 1.1X
+Java HashMap (Long) 62 65 3 335.8 3.0 2.3X
+Java HashMap (two ints) 86 90 4 242.5 4.1 1.7X
+Java HashMap (UnsafeRow) 509 511 4 41.2 24.3 0.3X
+LongToUnsafeRowMap (opt=false) 345 346 1 60.8 16.4 0.4X
+LongToUnsafeRowMap (opt=true) 85 86 1 246.3 4.1 1.7X
+BytesToBytesMap (off Heap) 437 439 1 48.0 20.8 0.3X
+BytesToBytesMap (on Heap) 437 438 1 48.0 20.8 0.3X
+Aggregate HashMap 30 31 1 690.8 1.4 4.8X
diff --git a/sql/core/benchmarks/AggregateBenchmark-results.txt b/sql/core/benchmarks/AggregateBenchmark-results.txt
index 68c605d5c4a16..27b6d7d37c505 100644
--- a/sql/core/benchmarks/AggregateBenchmark-results.txt
+++ b/sql/core/benchmarks/AggregateBenchmark-results.txt
@@ -2,147 +2,147 @@
aggregate without grouping
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
agg w/o group: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-agg w/o group wholestage off 38366 39019 923 54.7 18.3 1.0X
-agg w/o group wholestage on 3369 3383 9 622.5 1.6 11.4X
+agg w/o group wholestage off 38902 39364 653 53.9 18.5 1.0X
+agg w/o group wholestage on 3377 3381 4 621.0 1.6 11.5X
================================================================================================
stat functions
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
stddev: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-stddev wholestage off 4539 4551 17 23.1 43.3 1.0X
-stddev wholestage on 981 986 4 106.8 9.4 4.6X
+stddev wholestage off 4528 4536 12 23.2 43.2 1.0X
+stddev wholestage on 979 992 9 107.2 9.3 4.6X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
kurtosis: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-kurtosis wholestage off 22063 22102 56 4.8 210.4 1.0X
-kurtosis wholestage on 985 994 7 106.4 9.4 22.4X
+kurtosis wholestage off 23508 23553 64 4.5 224.2 1.0X
+kurtosis wholestage on 995 1000 5 105.4 9.5 23.6X
================================================================================================
aggregate with linear keys
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Aggregate w keys: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-codegen = F 6868 6923 78 12.2 81.9 1.0X
-codegen = T, hashmap = F 4329 4358 35 19.4 51.6 1.6X
-codegen = T, row-based hashmap = T 1203 1222 19 69.7 14.3 5.7X
-codegen = T, vectorized hashmap = T 828 847 34 101.3 9.9 8.3X
+codegen = F 7037 7162 177 11.9 83.9 1.0X
+codegen = T, hashmap = F 4256 4318 57 19.7 50.7 1.7X
+codegen = T, row-based hashmap = T 1236 1256 20 67.9 14.7 5.7X
+codegen = T, vectorized hashmap = T 817 840 19 102.7 9.7 8.6X
================================================================================================
aggregate with randomized keys
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Aggregate w keys: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-codegen = F 7623 7658 49 11.0 90.9 1.0X
-codegen = T, hashmap = F 4942 4971 28 17.0 58.9 1.5X
-codegen = T, row-based hashmap = T 1706 1729 17 49.2 20.3 4.5X
-codegen = T, vectorized hashmap = T 1042 1079 31 80.5 12.4 7.3X
+codegen = F 7713 7764 73 10.9 91.9 1.0X
+codegen = T, hashmap = F 4964 4975 18 16.9 59.2 1.6X
+codegen = T, row-based hashmap = T 1689 1711 20 49.7 20.1 4.6X
+codegen = T, vectorized hashmap = T 1044 1086 34 80.3 12.4 7.4X
================================================================================================
aggregate with string key
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Aggregate w string key: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-codegen = F 2493 2520 39 8.4 118.9 1.0X
-codegen = T, hashmap = F 1527 1538 11 13.7 72.8 1.6X
-codegen = T, row-based hashmap = T 1015 1025 12 20.7 48.4 2.5X
-codegen = T, vectorized hashmap = T 821 831 11 25.5 39.2 3.0X
+codegen = F 2526 2539 18 8.3 120.5 1.0X
+codegen = T, hashmap = F 1567 1575 11 13.4 74.7 1.6X
+codegen = T, row-based hashmap = T 1024 1028 3 20.5 48.8 2.5X
+codegen = T, vectorized hashmap = T 825 829 5 25.4 39.3 3.1X
================================================================================================
aggregate with decimal key
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Aggregate w decimal key: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-codegen = F 2122 2170 68 9.9 101.2 1.0X
-codegen = T, hashmap = F 1377 1393 22 15.2 65.7 1.5X
-codegen = T, row-based hashmap = T 437 442 4 48.0 20.9 4.9X
-codegen = T, vectorized hashmap = T 310 317 3 67.6 14.8 6.8X
+codegen = F 2110 2114 6 9.9 100.6 1.0X
+codegen = T, hashmap = F 1449 1450 2 14.5 69.1 1.5X
+codegen = T, row-based hashmap = T 448 458 13 46.8 21.4 4.7X
+codegen = T, vectorized hashmap = T 311 319 5 67.5 14.8 6.8X
================================================================================================
aggregate with multiple key types
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Aggregate w multiple keys: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-codegen = F 4196 4197 2 5.0 200.1 1.0X
-codegen = T, hashmap = F 2482 2493 14 8.4 118.4 1.7X
-codegen = T, row-based hashmap = T 1727 1740 18 12.1 82.4 2.4X
-codegen = T, vectorized hashmap = T 1648 1671 32 12.7 78.6 2.5X
+codegen = F 4264 4286 31 4.9 203.3 1.0X
+codegen = T, hashmap = F 2383 2406 32 8.8 113.6 1.8X
+codegen = T, row-based hashmap = T 1873 1876 4 11.2 89.3 2.3X
+codegen = T, vectorized hashmap = T 1674 1681 10 12.5 79.8 2.5X
================================================================================================
max function bytecode size of wholestagecodegen
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
max function bytecode size: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-codegen = F 359 388 25 1.8 548.0 1.0X
-codegen = T, hugeMethodLimit = 10000 147 166 16 4.5 223.6 2.5X
-codegen = T, hugeMethodLimit = 1500 135 146 13 4.8 206.5 2.7X
+codegen = F 379 411 21 1.7 577.6 1.0X
+codegen = T, hugeMethodLimit = 10000 137 160 20 4.8 209.4 2.8X
+codegen = T, hugeMethodLimit = 1500 135 156 15 4.9 205.7 2.8X
================================================================================================
cube
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
cube: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-cube wholestage off 2183 2214 44 2.4 416.4 1.0X
-cube wholestage on 1139 1148 7 4.6 217.3 1.9X
+cube wholestage off 2099 2117 27 2.5 400.3 1.0X
+cube wholestage on 1157 1164 5 4.5 220.7 1.8X
================================================================================================
hash and BytesToBytesMap
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
BytesToBytesMap: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-UnsafeRowhash 204 205 1 102.9 9.7 1.0X
-murmur3 hash 69 70 1 304.0 3.3 3.0X
-fast hash 72 73 7 292.8 3.4 2.8X
-arrayEqual 144 144 0 145.8 6.9 1.4X
-Java HashMap (Long) 65 69 3 320.7 3.1 3.1X
-Java HashMap (two ints) 85 88 1 247.9 4.0 2.4X
-Java HashMap (UnsafeRow) 549 550 1 38.2 26.2 0.4X
-LongToUnsafeRowMap (opt=false) 342 344 1 61.3 16.3 0.6X
-LongToUnsafeRowMap (opt=true) 73 74 1 287.4 3.5 2.8X
-BytesToBytesMap (off Heap) 607 609 2 34.6 28.9 0.3X
-BytesToBytesMap (on Heap) 609 623 17 34.4 29.0 0.3X
-Aggregate HashMap 31 31 0 686.4 1.5 6.7X
+UnsafeRowhash 204 204 0 102.9 9.7 1.0X
+murmur3 hash 70 70 0 301.0 3.3 2.9X
+fast hash 72 72 1 293.2 3.4 2.8X
+arrayEqual 144 144 0 145.7 6.9 1.4X
+Java HashMap (Long) 65 68 3 321.3 3.1 3.1X
+Java HashMap (two ints) 91 93 2 229.2 4.4 2.2X
+Java HashMap (UnsafeRow) 541 544 3 38.7 25.8 0.4X
+LongToUnsafeRowMap (opt=false) 345 346 1 60.9 16.4 0.6X
+LongToUnsafeRowMap (opt=true) 81 82 1 259.0 3.9 2.5X
+BytesToBytesMap (off Heap) 540 544 3 38.8 25.8 0.4X
+BytesToBytesMap (on Heap) 544 546 2 38.5 25.9 0.4X
+Aggregate HashMap 31 31 2 686.0 1.5 6.7X
diff --git a/sql/core/benchmarks/AnsiIntervalSortBenchmark-jdk21-results.txt b/sql/core/benchmarks/AnsiIntervalSortBenchmark-jdk21-results.txt
index b8fafc2f6104b..8ab7df150dc27 100644
--- a/sql/core/benchmarks/AnsiIntervalSortBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/AnsiIntervalSortBenchmark-jdk21-results.txt
@@ -1,28 +1,28 @@
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
year month interval one column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------
-year month interval one column enable radix 23323 23592 290 4.3 233.2 1.0X
-year month interval one column disable radix 33714 34091 548 3.0 337.1 0.7X
+year month interval one column enable radix 23086 23193 180 4.3 230.9 1.0X
+year month interval one column disable radix 33420 33638 280 3.0 334.2 0.7X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
year month interval two columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------
-year month interval two columns enable radix 35096 35612 485 2.8 351.0 1.0X
-year month interval two columns disable radix 35011 35229 196 2.9 350.1 1.0X
+year month interval two columns enable radix 34321 34376 49 2.9 343.2 1.0X
+year month interval two columns disable radix 34147 34406 226 2.9 341.5 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
day time interval one columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-day time interval one columns enable radix 22557 22651 141 4.4 225.6 1.0X
-day time interval one columns disable radix 45748 47238 1535 2.2 457.5 0.5X
+day time interval one columns enable radix 22025 22097 95 4.5 220.2 1.0X
+day time interval one columns disable radix 33367 33681 273 3.0 333.7 0.7X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
day time interval two columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-day time interval two columns enable radix 34733 36376 2685 2.9 347.3 1.0X
-day time interval two columns disable radix 34733 34798 57 2.9 347.3 1.0X
+day time interval two columns enable radix 34400 34546 199 2.9 344.0 1.0X
+day time interval two columns disable radix 34308 34467 148 2.9 343.1 1.0X
diff --git a/sql/core/benchmarks/AnsiIntervalSortBenchmark-results.txt b/sql/core/benchmarks/AnsiIntervalSortBenchmark-results.txt
index 2f43585165677..ab4243d9d9daa 100644
--- a/sql/core/benchmarks/AnsiIntervalSortBenchmark-results.txt
+++ b/sql/core/benchmarks/AnsiIntervalSortBenchmark-results.txt
@@ -1,28 +1,28 @@
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
year month interval one column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------
-year month interval one column enable radix 22341 22440 168 4.5 223.4 1.0X
-year month interval one column disable radix 32157 32216 51 3.1 321.6 0.7X
+year month interval one column enable radix 22721 22833 190 4.4 227.2 1.0X
+year month interval one column disable radix 32701 33148 769 3.1 327.0 0.7X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
year month interval two columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------
-year month interval two columns enable radix 33156 33365 284 3.0 331.6 1.0X
-year month interval two columns disable radix 33156 33661 697 3.0 331.6 1.0X
+year month interval two columns enable radix 33482 33621 208 3.0 334.8 1.0X
+year month interval two columns disable radix 34206 35140 824 2.9 342.1 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
day time interval one columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-day time interval one columns enable radix 21445 21467 21 4.7 214.5 1.0X
-day time interval one columns disable radix 33962 34125 193 2.9 339.6 0.6X
+day time interval one columns enable radix 21312 21459 239 4.7 213.1 1.0X
+day time interval one columns disable radix 32898 33575 820 3.0 329.0 0.6X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
day time interval two columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-day time interval two columns enable radix 33598 34314 627 3.0 336.0 1.0X
-day time interval two columns disable radix 34614 34755 192 2.9 346.1 1.0X
+day time interval two columns enable radix 35367 35398 27 2.8 353.7 1.0X
+day time interval two columns disable radix 33655 34642 856 3.0 336.5 1.1X
diff --git a/sql/core/benchmarks/Base64Benchmark-jdk21-results.txt b/sql/core/benchmarks/Base64Benchmark-jdk21-results.txt
index 1ffdb71362577..402818e8677f3 100644
--- a/sql/core/benchmarks/Base64Benchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/Base64Benchmark-jdk21-results.txt
@@ -1,56 +1,56 @@
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
encode for 1: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-java 2064 2074 13 9.7 103.2 1.0X
-apache 11112 11179 70 1.8 555.6 0.2X
+java 1702 1734 29 11.8 85.1 1.0X
+apache 12539 12638 149 1.6 626.9 0.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
encode for 3: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-java 2450 2454 4 8.2 122.5 1.0X
-apache 12408 12449 36 1.6 620.4 0.2X
+java 2296 2307 12 8.7 114.8 1.0X
+apache 13071 13126 95 1.5 653.5 0.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
encode for 5: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-java 3023 3025 2 6.6 151.1 1.0X
-apache 13242 13399 215 1.5 662.1 0.2X
+java 2547 2561 12 7.9 127.3 1.0X
+apache 13833 13954 190 1.4 691.7 0.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
encode for 7: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-java 3104 3107 3 6.4 155.2 1.0X
-apache 14314 14395 116 1.4 715.7 0.2X
+java 2364 2393 29 8.5 118.2 1.0X
+apache 14684 14846 151 1.4 734.2 0.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
decode for 1: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-java 3051 3110 69 6.6 152.5 1.0X
-apache 12294 12367 68 1.6 614.7 0.2X
+java 3451 3463 10 5.8 172.6 1.0X
+apache 13152 13264 99 1.5 657.6 0.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
decode for 3: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-java 4231 4235 6 4.7 211.5 1.0X
-apache 14025 14169 134 1.4 701.2 0.3X
+java 3957 3985 27 5.1 197.8 1.0X
+apache 15380 15852 454 1.3 769.0 0.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
decode for 5: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-java 5199 5209 14 3.8 260.0 1.0X
-apache 15985 16010 26 1.3 799.3 0.3X
+java 5316 5319 3 3.8 265.8 1.0X
+apache 16800 17066 241 1.2 840.0 0.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
decode for 7: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-java 5650 5664 18 3.5 282.5 1.0X
-apache 17240 17296 49 1.2 862.0 0.3X
+java 5803 5816 19 3.4 290.2 1.0X
+apache 17991 18178 257 1.1 899.5 0.3X
diff --git a/sql/core/benchmarks/Base64Benchmark-results.txt b/sql/core/benchmarks/Base64Benchmark-results.txt
index ee761ac20b45f..168622f3ee6d5 100644
--- a/sql/core/benchmarks/Base64Benchmark-results.txt
+++ b/sql/core/benchmarks/Base64Benchmark-results.txt
@@ -1,56 +1,56 @@
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
encode for 1: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-java 2137 2171 52 9.4 106.9 1.0X
-apache 11307 11383 122 1.8 565.4 0.2X
+java 1803 1819 20 11.1 90.2 1.0X
+apache 10576 10612 33 1.9 528.8 0.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
encode for 3: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-java 2607 2632 23 7.7 130.3 1.0X
-apache 12419 12504 77 1.6 620.9 0.2X
+java 2202 2206 4 9.1 110.1 1.0X
+apache 11399 11473 71 1.8 570.0 0.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
encode for 5: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-java 3217 3221 6 6.2 160.8 1.0X
-apache 13718 13802 123 1.5 685.9 0.2X
+java 2389 2413 22 8.4 119.5 1.0X
+apache 12749 12760 10 1.6 637.4 0.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
encode for 7: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-java 3680 3809 202 5.4 184.0 1.0X
-apache 14864 14911 41 1.3 743.2 0.2X
+java 2678 2692 19 7.5 133.9 1.0X
+apache 13702 13728 23 1.5 685.1 0.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
decode for 1: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-java 3284 3421 193 6.1 164.2 1.0X
-apache 12601 12635 45 1.6 630.0 0.3X
+java 2485 2607 135 8.0 124.2 1.0X
+apache 11603 11610 8 1.7 580.2 0.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
decode for 3: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-java 4651 4656 7 4.3 232.6 1.0X
-apache 14540 14547 8 1.4 727.0 0.3X
+java 3652 3680 25 5.5 182.6 1.0X
+apache 13428 13497 66 1.5 671.4 0.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
decode for 5: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-java 5689 5703 12 3.5 284.5 1.0X
-apache 16703 16719 16 1.2 835.1 0.3X
+java 4284 4359 66 4.7 214.2 1.0X
+apache 15058 15105 79 1.3 752.9 0.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
decode for 7: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-java 6467 6473 5 3.1 323.4 1.0X
-apache 18201 18301 124 1.1 910.1 0.4X
+java 5091 5176 74 3.9 254.6 1.0X
+apache 16424 16503 69 1.2 821.2 0.3X
diff --git a/sql/core/benchmarks/BloomFilterBenchmark-jdk21-results.txt b/sql/core/benchmarks/BloomFilterBenchmark-jdk21-results.txt
index 8f80b1e19fe32..93c71c9ed9b04 100644
--- a/sql/core/benchmarks/BloomFilterBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/BloomFilterBenchmark-jdk21-results.txt
@@ -2,195 +2,195 @@
ORC Write
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Without bloom filter 7475 7543 96 13.4 74.8 1.0X
-With bloom filter 9433 9445 16 10.6 94.3 0.8X
+Without bloom filter 7914 7933 26 12.6 79.1 1.0X
+With bloom filter 9778 9781 5 10.2 97.8 0.8X
================================================================================================
ORC Read
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Without bloom filter, blocksize: 2097152 871 881 13 114.8 8.7 1.0X
-With bloom filter, blocksize: 2097152 593 602 10 168.7 5.9 1.5X
+Without bloom filter, blocksize: 2097152 858 896 34 116.6 8.6 1.0X
+With bloom filter, blocksize: 2097152 576 592 18 173.5 5.8 1.5X
================================================================================================
ORC Read
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Without bloom filter, blocksize: 4194304 823 835 18 121.6 8.2 1.0X
-With bloom filter, blocksize: 4194304 535 544 8 186.8 5.4 1.5X
+Without bloom filter, blocksize: 4194304 834 846 11 119.9 8.3 1.0X
+With bloom filter, blocksize: 4194304 546 567 21 183.1 5.5 1.5X
================================================================================================
ORC Read
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Without bloom filter, blocksize: 6291456 820 831 17 121.9 8.2 1.0X
-With bloom filter, blocksize: 6291456 536 554 15 186.7 5.4 1.5X
+Without bloom filter, blocksize: 6291456 824 842 18 121.3 8.2 1.0X
+With bloom filter, blocksize: 6291456 560 591 22 178.6 5.6 1.5X
================================================================================================
ORC Read
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Without bloom filter, blocksize: 8388608 813 832 20 123.1 8.1 1.0X
-With bloom filter, blocksize: 8388608 538 568 30 185.7 5.4 1.5X
+Without bloom filter, blocksize: 8388608 816 839 21 122.6 8.2 1.0X
+With bloom filter, blocksize: 8388608 550 571 26 181.7 5.5 1.5X
================================================================================================
ORC Read
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
-Without bloom filter, blocksize: 12582912 868 874 9 115.2 8.7 1.0X
-With bloom filter, blocksize: 12582912 550 572 19 181.9 5.5 1.6X
+Without bloom filter, blocksize: 12582912 826 849 22 121.1 8.3 1.0X
+With bloom filter, blocksize: 12582912 586 597 8 170.5 5.9 1.4X
================================================================================================
ORC Read
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
-Without bloom filter, blocksize: 16777216 808 824 26 123.8 8.1 1.0X
-With bloom filter, blocksize: 16777216 541 561 14 184.7 5.4 1.5X
+Without bloom filter, blocksize: 16777216 811 826 13 123.2 8.1 1.0X
+With bloom filter, blocksize: 16777216 521 541 16 191.9 5.2 1.6X
================================================================================================
ORC Read
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
-Without bloom filter, blocksize: 33554432 805 818 14 124.3 8.0 1.0X
-With bloom filter, blocksize: 33554432 557 580 18 179.7 5.6 1.4X
+Without bloom filter, blocksize: 33554432 805 824 17 124.3 8.0 1.0X
+With bloom filter, blocksize: 33554432 526 540 10 190.1 5.3 1.5X
================================================================================================
Parquet Write
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-Without bloom filter 11626 11656 43 8.6 116.3 1.0X
-With bloom filter 14511 14577 93 6.9 145.1 0.8X
-With adaptive bloom filter & 3 candidates 14742 14886 203 6.8 147.4 0.8X
-With adaptive bloom filter & 5 candidates 14784 14789 8 6.8 147.8 0.8X
-With adaptive bloom filter & 9 candidates 14709 14725 23 6.8 147.1 0.8X
-With adaptive bloom filter & 15 candidates 14704 14749 64 6.8 147.0 0.8X
+Without bloom filter 9804 9869 91 10.2 98.0 1.0X
+With bloom filter 13700 13801 143 7.3 137.0 0.7X
+With adaptive bloom filter & 3 candidates 13475 13510 48 7.4 134.8 0.7X
+With adaptive bloom filter & 5 candidates 13308 13387 112 7.5 133.1 0.7X
+With adaptive bloom filter & 9 candidates 13471 13687 306 7.4 134.7 0.7X
+With adaptive bloom filter & 15 candidates 13292 13391 141 7.5 132.9 0.7X
================================================================================================
Parquet Read
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Without bloom filter, blocksize: 2097152 414 435 25 241.7 4.1 1.0X
-With bloom filter, blocksize: 2097152 180 196 7 556.2 1.8 2.3X
+Without bloom filter, blocksize: 2097152 429 464 34 232.8 4.3 1.0X
+With bloom filter, blocksize: 2097152 178 188 10 562.2 1.8 2.4X
================================================================================================
Parquet Read
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Without bloom filter, blocksize: 4194304 442 448 6 226.2 4.4 1.0X
-With bloom filter, blocksize: 4194304 121 128 7 829.6 1.2 3.7X
+Without bloom filter, blocksize: 4194304 405 410 4 247.1 4.0 1.0X
+With bloom filter, blocksize: 4194304 135 144 5 742.9 1.3 3.0X
================================================================================================
Parquet Read
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Without bloom filter, blocksize: 6291456 589 593 2 169.6 5.9 1.0X
-With bloom filter, blocksize: 6291456 125 134 7 799.9 1.3 4.7X
+Without bloom filter, blocksize: 6291456 399 409 8 250.7 4.0 1.0X
+With bloom filter, blocksize: 6291456 130 136 4 768.2 1.3 3.1X
================================================================================================
Parquet Read
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Without bloom filter, blocksize: 8388608 593 602 15 168.8 5.9 1.0X
-With bloom filter, blocksize: 8388608 281 292 11 355.3 2.8 2.1X
+Without bloom filter, blocksize: 8388608 403 410 5 248.1 4.0 1.0X
+With bloom filter, blocksize: 8388608 236 246 9 423.2 2.4 1.7X
================================================================================================
Parquet Read
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
-Without bloom filter, blocksize: 12582912 602 626 18 166.2 6.0 1.0X
-With bloom filter, blocksize: 12582912 477 502 29 209.8 4.8 1.3X
+Without bloom filter, blocksize: 12582912 416 453 25 240.1 4.2 1.0X
+With bloom filter, blocksize: 12582912 364 381 9 274.5 3.6 1.1X
================================================================================================
Parquet Read
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
-Without bloom filter, blocksize: 16777216 598 610 18 167.3 6.0 1.0X
-With bloom filter, blocksize: 16777216 520 524 4 192.5 5.2 1.2X
+Without bloom filter, blocksize: 16777216 443 474 21 225.8 4.4 1.0X
+With bloom filter, blocksize: 16777216 423 435 8 236.2 4.2 1.0X
================================================================================================
Parquet Read
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
-Without bloom filter, blocksize: 33554432 627 641 11 159.6 6.3 1.0X
-With bloom filter, blocksize: 33554432 615 629 13 162.6 6.1 1.0X
+Without bloom filter, blocksize: 33554432 421 440 15 237.3 4.2 1.0X
+With bloom filter, blocksize: 33554432 466 475 7 214.7 4.7 0.9X
diff --git a/sql/core/benchmarks/BloomFilterBenchmark-results.txt b/sql/core/benchmarks/BloomFilterBenchmark-results.txt
index 039dd1b73231e..2cdce5bfd7b72 100644
--- a/sql/core/benchmarks/BloomFilterBenchmark-results.txt
+++ b/sql/core/benchmarks/BloomFilterBenchmark-results.txt
@@ -2,195 +2,195 @@
ORC Write
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Without bloom filter 7671 7735 92 13.0 76.7 1.0X
-With bloom filter 9571 9658 123 10.4 95.7 0.8X
+Without bloom filter 7656 7767 157 13.1 76.6 1.0X
+With bloom filter 9477 9524 66 10.6 94.8 0.8X
================================================================================================
ORC Read
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Without bloom filter, blocksize: 2097152 912 932 19 109.6 9.1 1.0X
-With bloom filter, blocksize: 2097152 575 597 18 173.9 5.8 1.6X
+Without bloom filter, blocksize: 2097152 894 925 50 111.9 8.9 1.0X
+With bloom filter, blocksize: 2097152 586 596 9 170.5 5.9 1.5X
================================================================================================
ORC Read
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Without bloom filter, blocksize: 4194304 801 810 10 124.8 8.0 1.0X
-With bloom filter, blocksize: 4194304 547 562 11 182.7 5.5 1.5X
+Without bloom filter, blocksize: 4194304 856 879 20 116.8 8.6 1.0X
+With bloom filter, blocksize: 4194304 558 579 22 179.1 5.6 1.5X
================================================================================================
ORC Read
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Without bloom filter, blocksize: 6291456 814 820 5 122.8 8.1 1.0X
-With bloom filter, blocksize: 6291456 528 552 20 189.3 5.3 1.5X
+Without bloom filter, blocksize: 6291456 840 852 16 119.1 8.4 1.0X
+With bloom filter, blocksize: 6291456 544 566 15 183.8 5.4 1.5X
================================================================================================
ORC Read
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Without bloom filter, blocksize: 8388608 804 809 4 124.3 8.0 1.0X
-With bloom filter, blocksize: 8388608 531 556 25 188.3 5.3 1.5X
+Without bloom filter, blocksize: 8388608 813 816 5 123.0 8.1 1.0X
+With bloom filter, blocksize: 8388608 545 567 15 183.3 5.5 1.5X
================================================================================================
ORC Read
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
-Without bloom filter, blocksize: 12582912 816 822 11 122.6 8.2 1.0X
-With bloom filter, blocksize: 12582912 535 548 12 187.0 5.3 1.5X
+Without bloom filter, blocksize: 12582912 825 837 14 121.2 8.3 1.0X
+With bloom filter, blocksize: 12582912 542 547 6 184.6 5.4 1.5X
================================================================================================
ORC Read
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
-Without bloom filter, blocksize: 16777216 790 794 5 126.6 7.9 1.0X
-With bloom filter, blocksize: 16777216 792 798 9 126.3 7.9 1.0X
+Without bloom filter, blocksize: 16777216 827 835 10 120.9 8.3 1.0X
+With bloom filter, blocksize: 16777216 536 542 5 186.5 5.4 1.5X
================================================================================================
ORC Read
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
-Without bloom filter, blocksize: 33554432 1266 1267 1 79.0 12.7 1.0X
-With bloom filter, blocksize: 33554432 759 769 12 131.8 7.6 1.7X
+Without bloom filter, blocksize: 33554432 802 808 6 124.7 8.0 1.0X
+With bloom filter, blocksize: 33554432 546 570 23 183.0 5.5 1.5X
================================================================================================
Parquet Write
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-Without bloom filter 11243 11335 130 8.9 112.4 1.0X
-With bloom filter 14701 14728 38 6.8 147.0 0.8X
-With adaptive bloom filter & 3 candidates 14681 14747 93 6.8 146.8 0.8X
-With adaptive bloom filter & 5 candidates 14656 14680 34 6.8 146.6 0.8X
-With adaptive bloom filter & 9 candidates 14597 14739 201 6.9 146.0 0.8X
-With adaptive bloom filter & 15 candidates 14719 15232 726 6.8 147.2 0.8X
+Without bloom filter 10611 10612 1 9.4 106.1 1.0X
+With bloom filter 18632 18656 35 5.4 186.3 0.6X
+With adaptive bloom filter & 3 candidates 18536 18648 159 5.4 185.4 0.6X
+With adaptive bloom filter & 5 candidates 18673 18767 132 5.4 186.7 0.6X
+With adaptive bloom filter & 9 candidates 19199 19339 198 5.2 192.0 0.6X
+With adaptive bloom filter & 15 candidates 18736 18811 107 5.3 187.4 0.6X
================================================================================================
Parquet Read
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Without bloom filter, blocksize: 2097152 428 460 31 233.8 4.3 1.0X
-With bloom filter, blocksize: 2097152 180 185 5 556.9 1.8 2.4X
+Without bloom filter, blocksize: 2097152 454 464 13 220.1 4.5 1.0X
+With bloom filter, blocksize: 2097152 177 183 4 565.6 1.8 2.6X
================================================================================================
Parquet Read
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Without bloom filter, blocksize: 4194304 425 434 9 235.3 4.2 1.0X
-With bloom filter, blocksize: 4194304 132 142 9 758.3 1.3 3.2X
+Without bloom filter, blocksize: 4194304 424 432 5 235.7 4.2 1.0X
+With bloom filter, blocksize: 4194304 133 143 11 751.8 1.3 3.2X
================================================================================================
Parquet Read
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Without bloom filter, blocksize: 6291456 421 425 4 237.8 4.2 1.0X
-With bloom filter, blocksize: 6291456 133 140 5 750.9 1.3 3.2X
+Without bloom filter, blocksize: 6291456 427 434 5 234.1 4.3 1.0X
+With bloom filter, blocksize: 6291456 137 141 3 730.2 1.4 3.1X
================================================================================================
Parquet Read
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Without bloom filter, blocksize: 8388608 414 427 21 241.6 4.1 1.0X
-With bloom filter, blocksize: 8388608 180 188 6 555.3 1.8 2.3X
+Without bloom filter, blocksize: 8388608 419 429 5 238.5 4.2 1.0X
+With bloom filter, blocksize: 8388608 180 185 6 557.0 1.8 2.3X
================================================================================================
Parquet Read
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
-Without bloom filter, blocksize: 12582912 415 422 5 241.2 4.1 1.0X
-With bloom filter, blocksize: 12582912 277 284 8 360.9 2.8 1.5X
+Without bloom filter, blocksize: 12582912 427 435 6 234.4 4.3 1.0X
+With bloom filter, blocksize: 12582912 271 274 3 369.6 2.7 1.6X
================================================================================================
Parquet Read
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
-Without bloom filter, blocksize: 16777216 415 421 4 241.0 4.1 1.0X
-With bloom filter, blocksize: 16777216 345 348 2 289.7 3.5 1.2X
+Without bloom filter, blocksize: 16777216 442 457 17 226.2 4.4 1.0X
+With bloom filter, blocksize: 16777216 379 385 7 264.0 3.8 1.2X
================================================================================================
Parquet Read
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
-Without bloom filter, blocksize: 33554432 427 430 3 234.2 4.3 1.0X
-With bloom filter, blocksize: 33554432 508 520 12 196.9 5.1 0.8X
+Without bloom filter, blocksize: 33554432 507 520 10 197.1 5.1 1.0X
+With bloom filter, blocksize: 33554432 444 465 32 225.3 4.4 1.1X
diff --git a/sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-jdk21-results.txt b/sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-jdk21-results.txt
index f5a5e41abc411..f1a8a184aed2f 100644
--- a/sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-jdk21-results.txt
@@ -2,69 +2,69 @@
Parquet writer benchmark
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Parquet(PARQUET_1_0) writer benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Output Single Int Column 1728 1873 205 9.1 109.9 1.0X
-Output Single Double Column 1755 1770 22 9.0 111.6 1.0X
-Output Int and String Column 4182 4224 59 3.8 265.9 0.4X
-Output Partitions 3197 3308 157 4.9 203.2 0.5X
-Output Buckets 4471 4532 86 3.5 284.2 0.4X
+Output Single Int Column 1800 1830 42 8.7 114.5 1.0X
+Output Single Double Column 1730 1764 48 9.1 110.0 1.0X
+Output Int and String Column 4175 4245 98 3.8 265.5 0.4X
+Output Partitions 3118 3174 78 5.0 198.3 0.6X
+Output Buckets 4285 4407 172 3.7 272.4 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Parquet(PARQUET_2_0) writer benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Output Single Int Column 1911 1937 36 8.2 121.5 1.0X
-Output Single Double Column 1842 1857 21 8.5 117.1 1.0X
-Output Int and String Column 4738 4790 74 3.3 301.2 0.4X
-Output Partitions 3311 3314 5 4.8 210.5 0.6X
-Output Buckets 4457 4463 8 3.5 283.4 0.4X
+Output Single Int Column 1766 1792 36 8.9 112.3 1.0X
+Output Single Double Column 1730 1743 18 9.1 110.0 1.0X
+Output Int and String Column 5000 5014 19 3.1 317.9 0.4X
+Output Partitions 3373 3377 6 4.7 214.4 0.5X
+Output Buckets 4349 4363 21 3.6 276.5 0.4X
================================================================================================
ORC writer benchmark
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
ORC writer benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Output Single Int Column 1023 1029 9 15.4 65.0 1.0X
-Output Single Double Column 1585 1586 1 9.9 100.7 0.6X
-Output Int and String Column 3941 3976 50 4.0 250.6 0.3X
-Output Partitions 2392 2416 34 6.6 152.1 0.4X
-Output Buckets 3288 3336 69 4.8 209.0 0.3X
+Output Single Int Column 1045 1046 2 15.1 66.4 1.0X
+Output Single Double Column 1700 1700 1 9.3 108.1 0.6X
+Output Int and String Column 3891 3930 55 4.0 247.4 0.3X
+Output Partitions 2441 2453 17 6.4 155.2 0.4X
+Output Buckets 3401 3407 8 4.6 216.2 0.3X
================================================================================================
JSON writer benchmark
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
JSON writer benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Output Single Int Column 1549 1554 8 10.2 98.5 1.0X
-Output Single Double Column 2290 2293 4 6.9 145.6 0.7X
-Output Int and String Column 3888 3901 18 4.0 247.2 0.4X
-Output Partitions 3097 3098 2 5.1 196.9 0.5X
-Output Buckets 3967 3993 37 4.0 252.2 0.4X
+Output Single Int Column 1531 1534 5 10.3 97.3 1.0X
+Output Single Double Column 2285 2289 6 6.9 145.2 0.7X
+Output Int and String Column 3803 3811 12 4.1 241.8 0.4X
+Output Partitions 3025 3036 15 5.2 192.3 0.5X
+Output Buckets 3817 3827 14 4.1 242.7 0.4X
================================================================================================
CSV writer benchmark
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
CSV writer benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Output Single Int Column 3614 3631 23 4.4 229.8 1.0X
-Output Single Double Column 4315 4359 61 3.6 274.4 0.8X
-Output Int and String Column 6223 6230 10 2.5 395.6 0.6X
-Output Partitions 5652 5665 18 2.8 359.3 0.6X
-Output Buckets 6433 6439 8 2.4 409.0 0.6X
+Output Single Int Column 3485 3499 21 4.5 221.6 1.0X
+Output Single Double Column 4287 4316 41 3.7 272.6 0.8X
+Output Int and String Column 6416 6424 11 2.5 407.9 0.5X
+Output Partitions 4985 5016 45 3.2 316.9 0.7X
+Output Buckets 6180 6193 19 2.5 392.9 0.6X
diff --git a/sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-results.txt b/sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-results.txt
index d335986e385a4..f8805b65e8227 100644
--- a/sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-results.txt
+++ b/sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-results.txt
@@ -2,69 +2,69 @@
Parquet writer benchmark
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Parquet(PARQUET_1_0) writer benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Output Single Int Column 1807 1850 61 8.7 114.9 1.0X
-Output Single Double Column 1877 1878 2 8.4 119.3 1.0X
-Output Int and String Column 4562 4569 10 3.4 290.1 0.4X
-Output Partitions 3272 3312 57 4.8 208.0 0.6X
-Output Buckets 4431 4532 143 3.5 281.7 0.4X
+Output Single Int Column 1818 1898 114 8.7 115.6 1.0X
+Output Single Double Column 1847 1901 76 8.5 117.4 1.0X
+Output Int and String Column 4498 4547 69 3.5 286.0 0.4X
+Output Partitions 3369 3453 118 4.7 214.2 0.5X
+Output Buckets 4790 4814 34 3.3 304.5 0.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Parquet(PARQUET_2_0) writer benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Output Single Int Column 1750 1756 8 9.0 111.3 1.0X
-Output Single Double Column 1742 1749 11 9.0 110.7 1.0X
-Output Int and String Column 4936 4943 9 3.2 313.8 0.4X
-Output Partitions 3140 3151 16 5.0 199.7 0.6X
-Output Buckets 4037 4042 7 3.9 256.7 0.4X
+Output Single Int Column 1861 1901 57 8.5 118.3 1.0X
+Output Single Double Column 1866 1873 10 8.4 118.6 1.0X
+Output Int and String Column 4816 4830 19 3.3 306.2 0.4X
+Output Partitions 3270 3287 24 4.8 207.9 0.6X
+Output Buckets 4354 4359 7 3.6 276.8 0.4X
================================================================================================
ORC writer benchmark
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
ORC writer benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Output Single Int Column 1011 1023 17 15.6 64.3 1.0X
-Output Single Double Column 1523 1526 5 10.3 96.8 0.7X
-Output Int and String Column 3604 3645 58 4.4 229.2 0.3X
-Output Partitions 2397 2397 0 6.6 152.4 0.4X
-Output Buckets 3249 3278 40 4.8 206.6 0.3X
+Output Single Int Column 1004 1030 37 15.7 63.8 1.0X
+Output Single Double Column 1736 1742 9 9.1 110.4 0.6X
+Output Int and String Column 3888 3908 27 4.0 247.2 0.3X
+Output Partitions 2649 2654 7 5.9 168.4 0.4X
+Output Buckets 3544 3562 25 4.4 225.3 0.3X
================================================================================================
JSON writer benchmark
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
JSON writer benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Output Single Int Column 1558 1569 14 10.1 99.1 1.0X
-Output Single Double Column 2142 2163 30 7.3 136.2 0.7X
-Output Int and String Column 3999 4021 31 3.9 254.3 0.4X
-Output Partitions 2928 2945 23 5.4 186.2 0.5X
-Output Buckets 3971 3975 5 4.0 252.5 0.4X
+Output Single Int Column 1510 1528 24 10.4 96.0 1.0X
+Output Single Double Column 2198 2201 4 7.2 139.7 0.7X
+Output Int and String Column 3918 3948 41 4.0 249.1 0.4X
+Output Partitions 3058 3069 15 5.1 194.4 0.5X
+Output Buckets 3867 3892 36 4.1 245.8 0.4X
================================================================================================
CSV writer benchmark
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
CSV writer benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Output Single Int Column 2961 2980 27 5.3 188.3 1.0X
-Output Single Double Column 3846 3853 9 4.1 244.5 0.8X
-Output Int and String Column 5817 5827 15 2.7 369.8 0.5X
-Output Partitions 4568 4572 6 3.4 290.4 0.6X
-Output Buckets 5719 5753 48 2.8 363.6 0.5X
+Output Single Int Column 2989 3007 25 5.3 190.0 1.0X
+Output Single Double Column 3478 3498 28 4.5 221.1 0.9X
+Output Int and String Column 6016 6059 61 2.6 382.5 0.5X
+Output Partitions 5130 5149 26 3.1 326.2 0.6X
+Output Buckets 5946 5950 6 2.6 378.1 0.5X
diff --git a/sql/core/benchmarks/ByteArrayBenchmark-jdk21-results.txt b/sql/core/benchmarks/ByteArrayBenchmark-jdk21-results.txt
index f6fbe1845b326..86aeef46914fb 100644
--- a/sql/core/benchmarks/ByteArrayBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/ByteArrayBenchmark-jdk21-results.txt
@@ -2,26 +2,26 @@
byte array comparisons
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Byte Array compareTo: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-2-7 byte 252 259 4 259.7 3.9 1.0X
-8-16 byte 386 414 34 169.7 5.9 0.7X
-16-32 byte 387 393 7 169.3 5.9 0.7X
-512-1024 byte 521 528 10 125.7 8.0 0.5X
-512 byte slow 1537 1551 10 42.6 23.4 0.2X
-2-7 byte 315 316 1 208.0 4.8 0.8X
+2-7 byte 252 259 4 260.0 3.8 1.0X
+8-16 byte 412 438 34 159.0 6.3 0.6X
+16-32 byte 413 414 1 158.7 6.3 0.6X
+512-1024 byte 554 556 2 118.2 8.5 0.5X
+512 byte slow 1635 1704 64 40.1 25.0 0.2X
+2-7 byte 296 297 2 221.7 4.5 0.9X
================================================================================================
byte array equals
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Byte Array equals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Byte Array equals 789 801 15 202.8 4.9 1.0X
+Byte Array equals 810 814 4 197.6 5.1 1.0X
diff --git a/sql/core/benchmarks/ByteArrayBenchmark-results.txt b/sql/core/benchmarks/ByteArrayBenchmark-results.txt
index b94c773fc513f..2dcb98a837fcd 100644
--- a/sql/core/benchmarks/ByteArrayBenchmark-results.txt
+++ b/sql/core/benchmarks/ByteArrayBenchmark-results.txt
@@ -2,26 +2,26 @@
byte array comparisons
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Byte Array compareTo: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-2-7 byte 258 260 1 253.7 3.9 1.0X
-8-16 byte 388 440 37 169.0 5.9 0.7X
-16-32 byte 466 469 2 140.5 7.1 0.6X
-512-1024 byte 591 592 1 111.0 9.0 0.4X
-512 byte slow 1619 1694 72 40.5 24.7 0.2X
-2-7 byte 298 299 1 219.7 4.6 0.9X
+2-7 byte 259 260 1 253.3 3.9 1.0X
+8-16 byte 439 482 32 149.1 6.7 0.6X
+16-32 byte 501 503 1 130.7 7.7 0.5X
+512-1024 byte 648 652 1 101.1 9.9 0.4X
+512 byte slow 1721 1766 41 38.1 26.3 0.2X
+2-7 byte 296 296 0 221.5 4.5 0.9X
================================================================================================
byte array equals
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Byte Array equals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Byte Array equals 822 829 6 194.7 5.1 1.0X
+Byte Array equals 706 710 1 226.6 4.4 1.0X
diff --git a/sql/core/benchmarks/CSVBenchmark-jdk21-results.txt b/sql/core/benchmarks/CSVBenchmark-jdk21-results.txt
index c3d72d847800d..e2c27a52e1f4b 100644
--- a/sql/core/benchmarks/CSVBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/CSVBenchmark-jdk21-results.txt
@@ -2,76 +2,76 @@
Benchmark to measure CSV read/write performance
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Parsing quoted values: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-One quoted string 24351 24419 60 0.0 487014.8 1.0X
+One quoted string 24075 24176 88 0.0 481490.1 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Wide rows with 1000 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Select 1000 columns 56834 57144 501 0.0 56834.4 1.0X
-Select 100 columns 21054 21095 55 0.0 21054.0 2.7X
-Select one column 17523 17550 27 0.1 17522.9 3.2X
-count() 3658 3676 25 0.3 3657.7 15.5X
-Select 100 columns, one bad input field 25678 25832 245 0.0 25678.1 2.2X
-Select 100 columns, corrupt record field 29027 29102 75 0.0 29026.6 2.0X
+Select 1000 columns 58743 59075 481 0.0 58742.9 1.0X
+Select 100 columns 21215 21234 19 0.0 21215.2 2.8X
+Select one column 17492 17573 122 0.1 17491.7 3.4X
+count() 3652 3697 70 0.3 3652.5 16.1X
+Select 100 columns, one bad input field 25226 25290 75 0.0 25226.1 2.3X
+Select 100 columns, corrupt record field 28706 28800 139 0.0 28705.9 2.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Count a dataset with 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Select 10 columns + count() 10832 10860 39 0.9 1083.2 1.0X
-Select 1 column + count() 7372 7399 27 1.4 737.2 1.5X
-count() 1698 1706 8 5.9 169.8 6.4X
+Select 10 columns + count() 10639 10688 49 0.9 1063.9 1.0X
+Select 1 column + count() 7266 7274 7 1.4 726.6 1.5X
+count() 1565 1572 6 6.4 156.5 6.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Create a dataset of timestamps 864 867 2 11.6 86.4 1.0X
-to_csv(timestamp) 6183 6192 11 1.6 618.3 0.1X
-write timestamps to files 6506 6512 7 1.5 650.6 0.1X
-Create a dataset of dates 961 962 2 10.4 96.1 0.9X
-to_csv(date) 4597 4600 5 2.2 459.7 0.2X
-write dates to files 4608 4613 6 2.2 460.8 0.2X
+Create a dataset of timestamps 843 861 23 11.9 84.3 1.0X
+to_csv(timestamp) 5939 5965 45 1.7 593.9 0.1X
+write timestamps to files 6446 6456 9 1.6 644.6 0.1X
+Create a dataset of dates 936 941 5 10.7 93.6 0.9X
+to_csv(date) 4325 4331 5 2.3 432.5 0.2X
+write dates to files 4637 4646 8 2.2 463.7 0.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------------------------------
-read timestamp text from files 1311 1314 5 7.6 131.1 1.0X
-read timestamps from files 11583 11590 8 0.9 1158.3 0.1X
-infer timestamps from files 22995 23055 64 0.4 2299.5 0.1X
-read date text from files 1234 1276 37 8.1 123.4 1.1X
-read date from files 11216 11238 30 0.9 1121.6 0.1X
-infer date from files 22681 22718 35 0.4 2268.1 0.1X
-timestamp strings 1224 1227 2 8.2 122.4 1.1X
-parse timestamps from Dataset[String] 13706 13760 83 0.7 1370.6 0.1X
-infer timestamps from Dataset[String] 25170 25224 64 0.4 2517.0 0.1X
-date strings 1698 1704 5 5.9 169.8 0.8X
-parse dates from Dataset[String] 12766 12789 21 0.8 1276.6 0.1X
-from_csv(timestamp) 11607 11690 73 0.9 1160.7 0.1X
-from_csv(date) 11353 11364 13 0.9 1135.3 0.1X
-infer error timestamps from Dataset[String] with default format 14883 14927 46 0.7 1488.3 0.1X
-infer error timestamps from Dataset[String] with user-provided format 14897 14928 38 0.7 1489.7 0.1X
-infer error timestamps from Dataset[String] with legacy format 14893 14931 45 0.7 1489.3 0.1X
+read timestamp text from files 1204 1213 8 8.3 120.4 1.0X
+read timestamps from files 11651 11677 22 0.9 1165.1 0.1X
+infer timestamps from files 23349 23353 6 0.4 2334.9 0.1X
+read date text from files 1101 1108 9 9.1 110.1 1.1X
+read date from files 10918 10925 8 0.9 1091.8 0.1X
+infer date from files 22494 22523 26 0.4 2249.4 0.1X
+timestamp strings 1183 1188 5 8.5 118.3 1.0X
+parse timestamps from Dataset[String] 13334 13359 24 0.7 1333.4 0.1X
+infer timestamps from Dataset[String] 24804 24861 50 0.4 2480.4 0.0X
+date strings 1664 1666 3 6.0 166.4 0.7X
+parse dates from Dataset[String] 12782 12826 38 0.8 1278.2 0.1X
+from_csv(timestamp) 11198 11219 23 0.9 1119.8 0.1X
+from_csv(date) 11210 11217 11 0.9 1121.0 0.1X
+infer error timestamps from Dataset[String] with default format 14749 14806 52 0.7 1474.9 0.1X
+infer error timestamps from Dataset[String] with user-provided format 14727 14797 69 0.7 1472.7 0.1X
+infer error timestamps from Dataset[String] with legacy format 14750 14815 92 0.7 1475.0 0.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-w/o filters 4227 4239 14 0.0 42274.8 1.0X
-pushdown disabled 4259 4299 42 0.0 42592.2 1.0X
-w/ filters 741 746 4 0.1 7414.8 5.7X
+w/o filters 4312 4316 6 0.0 43118.3 1.0X
+pushdown disabled 4380 4388 10 0.0 43801.0 1.0X
+w/ filters 829 838 9 0.1 8288.7 5.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Read as Intervals 808 809 1 0.4 2693.2 1.0X
-Read Raw Strings 325 330 6 0.9 1082.6 2.5X
+Read as Intervals 772 785 16 0.4 2571.8 1.0X
+Read Raw Strings 323 330 6 0.9 1076.2 2.4X
diff --git a/sql/core/benchmarks/CSVBenchmark-results.txt b/sql/core/benchmarks/CSVBenchmark-results.txt
index 606263be672ef..8d5f3cf320743 100644
--- a/sql/core/benchmarks/CSVBenchmark-results.txt
+++ b/sql/core/benchmarks/CSVBenchmark-results.txt
@@ -2,76 +2,76 @@
Benchmark to measure CSV read/write performance
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Parsing quoted values: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-One quoted string 24626 24670 43 0.0 492514.1 1.0X
+One quoted string 24585 24692 144 0.0 491701.3 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Wide rows with 1000 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Select 1000 columns 51490 52060 751 0.0 51489.8 1.0X
-Select 100 columns 22350 22409 52 0.0 22349.9 2.3X
-Select one column 19397 19463 110 0.1 19397.4 2.7X
-count() 3518 3521 4 0.3 3517.6 14.6X
-Select 100 columns, one bad input field 27426 27460 48 0.0 27425.9 1.9X
-Select 100 columns, corrupt record field 30873 30931 59 0.0 30872.7 1.7X
+Select 1000 columns 56692 57020 438 0.0 56691.8 1.0X
+Select 100 columns 22207 22251 40 0.0 22206.5 2.6X
+Select one column 18890 18927 43 0.1 18889.9 3.0X
+count() 3530 3572 49 0.3 3530.3 16.1X
+Select 100 columns, one bad input field 27094 27109 23 0.0 27094.0 2.1X
+Select 100 columns, corrupt record field 30004 30083 125 0.0 30003.6 1.9X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Count a dataset with 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Select 10 columns + count() 9260 9278 26 1.1 926.0 1.0X
-Select 1 column + count() 6719 6732 15 1.5 671.9 1.4X
-count() 1605 1606 1 6.2 160.5 5.8X
+Select 10 columns + count() 9066 9092 33 1.1 906.6 1.0X
+Select 1 column + count() 6397 6416 16 1.6 639.7 1.4X
+count() 1500 1507 12 6.7 150.0 6.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Create a dataset of timestamps 853 872 30 11.7 85.3 1.0X
-to_csv(timestamp) 6321 6364 37 1.6 632.1 0.1X
-write timestamps to files 6787 6802 14 1.5 678.7 0.1X
-Create a dataset of dates 972 978 9 10.3 97.2 0.9X
-to_csv(date) 4729 4734 8 2.1 472.9 0.2X
-write dates to files 4759 4771 12 2.1 475.9 0.2X
+Create a dataset of timestamps 972 978 5 10.3 97.2 1.0X
+to_csv(timestamp) 6014 6026 13 1.7 601.4 0.2X
+write timestamps to files 6293 6302 9 1.6 629.3 0.2X
+Create a dataset of dates 971 974 6 10.3 97.1 1.0X
+to_csv(date) 4320 4327 6 2.3 432.0 0.2X
+write dates to files 4137 4153 17 2.4 413.7 0.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------------------------------
-read timestamp text from files 1229 1242 19 8.1 122.9 1.0X
-read timestamps from files 9798 9822 20 1.0 979.8 0.1X
-infer timestamps from files 19462 19478 24 0.5 1946.2 0.1X
-read date text from files 1116 1119 2 9.0 111.6 1.1X
-read date from files 9909 9922 12 1.0 990.9 0.1X
-infer date from files 20206 20218 19 0.5 2020.6 0.1X
-timestamp strings 1240 1243 3 8.1 124.0 1.0X
-parse timestamps from Dataset[String] 11635 11649 12 0.9 1163.5 0.1X
-infer timestamps from Dataset[String] 21255 21265 9 0.5 2125.5 0.1X
-date strings 1785 1787 3 5.6 178.5 0.7X
-parse dates from Dataset[String] 11553 11571 17 0.9 1155.3 0.1X
-from_csv(timestamp) 9827 9833 5 1.0 982.7 0.1X
-from_csv(date) 10621 10635 12 0.9 1062.1 0.1X
-infer error timestamps from Dataset[String] with default format 12066 12074 12 0.8 1206.6 0.1X
-infer error timestamps from Dataset[String] with user-provided format 12061 12066 6 0.8 1206.1 0.1X
-infer error timestamps from Dataset[String] with legacy format 12033 12042 9 0.8 1203.3 0.1X
+read timestamp text from files 1158 1165 8 8.6 115.8 1.0X
+read timestamps from files 9575 9587 11 1.0 957.5 0.1X
+infer timestamps from files 19365 19406 36 0.5 1936.5 0.1X
+read date text from files 1059 1064 8 9.4 105.9 1.1X
+read date from files 9650 9685 36 1.0 965.0 0.1X
+infer date from files 20159 20171 11 0.5 2015.9 0.1X
+timestamp strings 1180 1183 3 8.5 118.0 1.0X
+parse timestamps from Dataset[String] 11475 11488 20 0.9 1147.5 0.1X
+infer timestamps from Dataset[String] 21022 21030 7 0.5 2102.2 0.1X
+date strings 1640 1641 1 6.1 164.0 0.7X
+parse dates from Dataset[String] 11604 11616 19 0.9 1160.4 0.1X
+from_csv(timestamp) 9428 9456 25 1.1 942.8 0.1X
+from_csv(date) 10226 10268 65 1.0 1022.6 0.1X
+infer error timestamps from Dataset[String] with default format 11716 11728 14 0.9 1171.6 0.1X
+infer error timestamps from Dataset[String] with user-provided format 11726 11734 8 0.9 1172.6 0.1X
+infer error timestamps from Dataset[String] with legacy format 11730 11747 16 0.9 1173.0 0.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-w/o filters 4092 4098 6 0.0 40917.6 1.0X
-pushdown disabled 3990 4009 20 0.0 39897.7 1.0X
-w/ filters 717 721 4 0.1 7170.8 5.7X
+w/o filters 4305 4314 10 0.0 43050.4 1.0X
+pushdown disabled 4245 4256 10 0.0 42448.7 1.0X
+w/ filters 789 790 2 0.1 7886.0 5.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Read as Intervals 699 705 6 0.4 2331.5 1.0X
-Read Raw Strings 274 279 5 1.1 911.8 2.6X
+Read as Intervals 660 660 1 0.5 2198.5 1.0X
+Read Raw Strings 281 286 5 1.1 937.0 2.3X
diff --git a/sql/core/benchmarks/CharVarcharBenchmark-jdk21-results.txt b/sql/core/benchmarks/CharVarcharBenchmark-jdk21-results.txt
index 640b0c3095f0d..d9f8e2df2d9f3 100644
--- a/sql/core/benchmarks/CharVarcharBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/CharVarcharBenchmark-jdk21-results.txt
@@ -2,121 +2,121 @@
Char Varchar Write Side Perf w/o Tailing Spaces
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write with length 5: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-write string with length 5 6970 7325 311 5.7 174.3 1.0X
-write char with length 5 10016 10047 27 4.0 250.4 0.7X
-write varchar with length 5 7156 7176 22 5.6 178.9 1.0X
+write string with length 5 7168 7314 139 5.6 179.2 1.0X
+write char with length 5 10502 10571 66 3.8 262.6 0.7X
+write varchar with length 5 7223 7343 149 5.5 180.6 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write with length 10: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-write string with length 10 3332 3385 54 6.0 166.6 1.0X
-write char with length 10 6314 6319 6 3.2 315.7 0.5X
-write varchar with length 10 3647 3673 41 5.5 182.4 0.9X
+write string with length 10 3561 3573 11 5.6 178.1 1.0X
+write char with length 10 6447 6511 55 3.1 322.4 0.6X
+write varchar with length 10 3720 3728 7 5.4 186.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write with length 20: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-write string with length 20 1877 1880 4 5.3 187.7 1.0X
-write char with length 20 4522 4549 42 2.2 452.2 0.4X
-write varchar with length 20 1923 1933 12 5.2 192.3 1.0X
+write string with length 20 1684 1691 6 5.9 168.4 1.0X
+write char with length 20 4764 4788 37 2.1 476.4 0.4X
+write varchar with length 20 1907 1909 3 5.2 190.7 0.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write with length 40: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-write string with length 40 961 969 8 5.2 192.1 1.0X
-write char with length 40 3548 3567 22 1.4 709.6 0.3X
-write varchar with length 40 959 967 10 5.2 191.8 1.0X
+write string with length 40 880 885 4 5.7 176.1 1.0X
+write char with length 40 3718 3726 13 1.3 743.5 0.2X
+write varchar with length 40 989 991 3 5.1 197.8 0.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write with length 60: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-write string with length 60 616 648 33 5.4 184.9 1.0X
-write char with length 60 3199 3205 8 1.0 959.7 0.2X
-write varchar with length 60 695 698 4 4.8 208.5 0.9X
+write string with length 60 674 678 4 4.9 202.1 1.0X
+write char with length 60 3345 3364 17 1.0 1003.5 0.2X
+write varchar with length 60 705 706 1 4.7 211.4 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write with length 80: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-write string with length 80 490 497 6 5.1 196.1 1.0X
-write char with length 80 3006 3020 21 0.8 1202.3 0.2X
-write varchar with length 80 540 548 12 4.6 216.2 0.9X
+write string with length 80 497 501 4 5.0 198.9 1.0X
+write char with length 80 3126 3131 7 0.8 1250.3 0.2X
+write varchar with length 80 563 569 7 4.4 225.3 0.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write with length 100: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-write string with length 100 413 418 8 4.8 206.4 1.0X
-write char with length 100 2955 2960 9 0.7 1477.4 0.1X
-write varchar with length 100 452 454 2 4.4 226.1 0.9X
+write string with length 100 390 392 2 5.1 194.8 1.0X
+write char with length 100 3008 3019 10 0.7 1503.9 0.1X
+write varchar with length 100 460 463 5 4.4 229.8 0.8X
================================================================================================
Char Varchar Write Side Perf w/ Tailing Spaces
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write with length 5: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-write string with length 5 8601 8616 22 4.7 215.0 1.0X
-write char with length 5 12123 12186 101 3.3 303.1 0.7X
-write varchar with length 5 12329 12357 42 3.2 308.2 0.7X
+write string with length 5 8133 8163 28 4.9 203.3 1.0X
+write char with length 5 11464 11555 149 3.5 286.6 0.7X
+write varchar with length 5 10906 10970 56 3.7 272.7 0.7X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write with length 10: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-write string with length 10 5147 5184 33 3.9 257.3 1.0X
-write char with length 10 8036 8072 39 2.5 401.8 0.6X
-write varchar with length 10 7861 7876 19 2.5 393.1 0.7X
+write string with length 10 4726 4731 7 4.2 236.3 1.0X
+write char with length 10 6837 6849 12 2.9 341.9 0.7X
+write varchar with length 10 6910 6915 5 2.9 345.5 0.7X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write with length 20: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-write string with length 20 3124 3131 8 3.2 312.4 1.0X
-write char with length 20 5491 5499 10 1.8 549.1 0.6X
-write varchar with length 20 5562 5567 5 1.8 556.2 0.6X
+write string with length 20 2816 2822 5 3.6 281.6 1.0X
+write char with length 20 4928 4931 5 2.0 492.8 0.6X
+write varchar with length 20 4886 4888 2 2.0 488.6 0.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write with length 40: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-write string with length 40 2069 2072 5 2.4 413.8 1.0X
-write char with length 40 4264 4275 13 1.2 852.8 0.5X
-write varchar with length 40 4312 4317 6 1.2 862.3 0.5X
+write string with length 40 1517 1521 3 3.3 303.4 1.0X
+write char with length 40 3716 3719 2 1.3 743.3 0.4X
+write varchar with length 40 3724 3731 6 1.3 744.9 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write with length 60: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-write string with length 60 1771 1774 3 1.9 531.4 1.0X
-write char with length 60 3884 3887 3 0.9 1165.3 0.5X
-write varchar with length 60 3987 3988 1 0.8 1196.0 0.4X
+write string with length 60 1250 1251 2 2.7 375.0 1.0X
+write char with length 60 3404 3420 22 1.0 1021.1 0.4X
+write varchar with length 60 3412 3419 6 1.0 1023.5 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write with length 80: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-write string with length 80 1684 1691 6 1.5 673.6 1.0X
-write char with length 80 3712 3721 9 0.7 1484.8 0.5X
-write varchar with length 80 3729 3737 11 0.7 1491.5 0.5X
+write string with length 80 1159 1171 12 2.2 463.8 1.0X
+write char with length 80 3190 3202 17 0.8 1275.9 0.4X
+write varchar with length 80 2868 2870 4 0.9 1147.2 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write with length 100: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-write string with length 100 1593 1597 6 1.3 796.6 1.0X
-write char with length 100 3723 3726 5 0.5 1861.4 0.4X
-write varchar with length 100 3627 3641 13 0.6 1813.6 0.4X
+write string with length 100 990 994 6 2.0 494.8 1.0X
+write char with length 100 3104 3117 14 0.6 1552.1 0.3X
+write varchar with length 100 2748 2755 9 0.7 1374.1 0.4X
diff --git a/sql/core/benchmarks/CharVarcharBenchmark-results.txt b/sql/core/benchmarks/CharVarcharBenchmark-results.txt
index 49822b663aa12..d76b532eeb5fd 100644
--- a/sql/core/benchmarks/CharVarcharBenchmark-results.txt
+++ b/sql/core/benchmarks/CharVarcharBenchmark-results.txt
@@ -2,121 +2,121 @@
Char Varchar Write Side Perf w/o Tailing Spaces
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write with length 5: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-write string with length 5 7259 7346 87 5.5 181.5 1.0X
-write char with length 5 10259 10279 27 3.9 256.5 0.7X
-write varchar with length 5 7483 7499 23 5.3 187.1 1.0X
+write string with length 5 6572 6829 222 6.1 164.3 1.0X
+write char with length 5 9718 9745 43 4.1 242.9 0.7X
+write varchar with length 5 7419 7449 49 5.4 185.5 0.9X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write with length 10: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-write string with length 10 3732 3789 57 5.4 186.6 1.0X
-write char with length 10 6479 6493 14 3.1 324.0 0.6X
-write varchar with length 10 3708 3748 36 5.4 185.4 1.0X
+write string with length 10 3344 3352 8 6.0 167.2 1.0X
+write char with length 10 6395 6435 41 3.1 319.8 0.5X
+write varchar with length 10 3621 3627 6 5.5 181.0 0.9X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write with length 20: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-write string with length 20 1752 1761 16 5.7 175.2 1.0X
-write char with length 20 4446 4506 52 2.2 444.6 0.4X
-write varchar with length 20 1900 1923 22 5.3 190.0 0.9X
+write string with length 20 1703 1718 16 5.9 170.3 1.0X
+write char with length 20 4663 4701 37 2.1 466.3 0.4X
+write varchar with length 20 1836 1847 13 5.4 183.6 0.9X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write with length 40: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-write string with length 40 986 997 10 5.1 197.2 1.0X
-write char with length 40 3535 3540 4 1.4 707.1 0.3X
-write varchar with length 40 1037 1043 8 4.8 207.3 1.0X
+write string with length 40 900 911 9 5.6 180.1 1.0X
+write char with length 40 3683 3684 1 1.4 736.6 0.2X
+write varchar with length 40 1017 1028 10 4.9 203.3 0.9X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write with length 60: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-write string with length 60 682 684 2 4.9 204.6 1.0X
-write char with length 60 3192 3201 8 1.0 957.5 0.2X
-write varchar with length 60 715 720 4 4.7 214.5 1.0X
+write string with length 60 634 638 3 5.3 190.2 1.0X
+write char with length 60 3267 3270 3 1.0 980.1 0.2X
+write varchar with length 60 677 689 18 4.9 203.2 0.9X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write with length 80: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-write string with length 80 522 527 7 4.8 208.9 1.0X
-write char with length 80 2997 3006 10 0.8 1199.0 0.2X
-write varchar with length 80 542 553 15 4.6 216.7 1.0X
+write string with length 80 493 493 0 5.1 197.2 1.0X
+write char with length 80 3101 3167 62 0.8 1240.3 0.2X
+write varchar with length 80 557 560 3 4.5 222.9 0.9X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write with length 100: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-write string with length 100 414 420 9 4.8 207.0 1.0X
-write char with length 100 2917 2936 17 0.7 1458.7 0.1X
-write varchar with length 100 462 467 5 4.3 231.1 0.9X
+write string with length 100 399 408 10 5.0 199.7 1.0X
+write char with length 100 3032 3045 12 0.7 1516.1 0.1X
+write varchar with length 100 455 459 7 4.4 227.4 0.9X
================================================================================================
Char Varchar Write Side Perf w/ Tailing Spaces
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write with length 5: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-write string with length 5 8623 8642 16 4.6 215.6 1.0X
-write char with length 5 11664 11723 86 3.4 291.6 0.7X
-write varchar with length 5 11849 11900 60 3.4 296.2 0.7X
+write string with length 5 7780 7823 40 5.1 194.5 1.0X
+write char with length 5 10884 10938 70 3.7 272.1 0.7X
+write varchar with length 5 10615 10673 88 3.8 265.4 0.7X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write with length 10: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-write string with length 10 5569 5576 9 3.6 278.5 1.0X
-write char with length 10 7953 7968 25 2.5 397.6 0.7X
-write varchar with length 10 7745 7758 20 2.6 387.3 0.7X
+write string with length 10 4629 4641 11 4.3 231.4 1.0X
+write char with length 10 7046 7050 6 2.8 352.3 0.7X
+write varchar with length 10 6512 6540 24 3.1 325.6 0.7X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write with length 20: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-write string with length 20 3400 3409 15 2.9 340.0 1.0X
-write char with length 20 5593 5599 9 1.8 559.3 0.6X
-write varchar with length 20 5128 5134 11 2.0 512.8 0.7X
+write string with length 20 2486 2497 12 4.0 248.6 1.0X
+write char with length 20 4770 4787 15 2.1 477.0 0.5X
+write varchar with length 20 4508 4513 4 2.2 450.8 0.6X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write with length 40: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-write string with length 40 2194 2196 3 2.3 438.8 1.0X
-write char with length 40 4255 4262 7 1.2 850.9 0.5X
-write varchar with length 40 4032 4035 3 1.2 806.5 0.5X
+write string with length 40 1580 1586 5 3.2 316.1 1.0X
+write char with length 40 3773 3777 5 1.3 754.7 0.4X
+write varchar with length 40 3411 3417 8 1.5 682.2 0.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write with length 60: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-write string with length 60 1932 1936 5 1.7 579.5 1.0X
-write char with length 60 3976 3991 14 0.8 1192.9 0.5X
-write varchar with length 60 3690 3692 4 0.9 1106.9 0.5X
+write string with length 60 1303 1305 3 2.6 390.9 1.0X
+write char with length 60 3443 3447 4 1.0 1032.9 0.4X
+write varchar with length 60 3097 3099 2 1.1 929.0 0.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write with length 80: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-write string with length 80 1751 1759 12 1.4 700.3 1.0X
-write char with length 80 3842 3854 17 0.7 1536.9 0.5X
-write varchar with length 80 3478 3482 5 0.7 1391.2 0.5X
+write string with length 80 1164 1173 7 2.1 465.7 1.0X
+write char with length 80 3342 3346 5 0.7 1336.9 0.3X
+write varchar with length 80 2912 2926 16 0.9 1164.7 0.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write with length 100: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-write string with length 100 1579 1588 13 1.3 789.7 1.0X
-write char with length 100 3644 3658 18 0.5 1821.9 0.4X
-write varchar with length 100 3390 3393 4 0.6 1694.9 0.5X
+write string with length 100 1064 1071 5 1.9 532.2 1.0X
+write char with length 100 3252 3261 11 0.6 1625.9 0.3X
+write varchar with length 100 2846 2855 8 0.7 1422.9 0.4X
diff --git a/sql/core/benchmarks/CollationBenchmark-jdk21-results.txt b/sql/core/benchmarks/CollationBenchmark-jdk21-results.txt
index 771b6526a7136..e15be5c51f72e 100644
--- a/sql/core/benchmarks/CollationBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/CollationBenchmark-jdk21-results.txt
@@ -1,88 +1,88 @@
-OpenJDK 64-Bit Server VM 21.0.7+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - equalsFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
--------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 1702 1703 1 0.1 17020.6 1.0X
-UTF8_LCASE 2636 2639 5 0.0 26357.2 1.5X
-UNICODE 17060 17064 7 0.0 170597.2 10.0X
-UNICODE_CI 16499 16522 33 0.0 164987.9 9.7X
+UTF8_BINARY 1769 1769 1 0.1 17688.3 1.0X
+UTF8_LCASE 2662 2662 1 0.0 26617.3 1.5X
+UNICODE 16947 16955 12 0.0 169467.7 9.6X
+UNICODE_CI 16600 16610 14 0.0 165998.2 9.4X
-OpenJDK 64-Bit Server VM 21.0.7+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - compareFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
---------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 1783 1783 0 0.1 17825.5 1.0X
-UTF8_LCASE 2598 2598 0 0.0 25975.5 1.5X
-UNICODE 17608 17611 5 0.0 176075.6 9.9X
-UNICODE_CI 17425 17425 0 0.0 174251.6 9.8X
+UTF8_BINARY 1778 1779 1 0.1 17782.8 1.0X
+UTF8_LCASE 3755 3756 1 0.0 37550.7 2.1X
+UNICODE 17556 17569 18 0.0 175562.4 9.9X
+UNICODE_CI 17348 17360 17 0.0 173481.2 9.8X
-OpenJDK 64-Bit Server VM 21.0.7+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - hashFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 2844 2845 1 0.0 28440.6 1.0X
-UTF8_LCASE 5455 5460 8 0.0 54547.5 1.9X
-UNICODE 68308 68344 50 0.0 683078.9 24.0X
-UNICODE_CI 57344 57360 23 0.0 573442.6 20.2X
+UTF8_BINARY 12827 12831 6 0.0 128267.0 1.0X
+UTF8_LCASE 19675 19693 25 0.0 196751.6 1.5X
+UNICODE 106365 106368 4 0.0 1063650.1 8.3X
+UNICODE_CI 153190 153205 20 0.0 1531904.4 11.9X
-OpenJDK 64-Bit Server VM 21.0.7+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - contains: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 1704 1704 0 0.1 17035.8 1.0X
-UTF8_LCASE 15349 15352 5 0.0 153485.8 9.0X
-UNICODE 315270 315691 595 0.0 3152700.6 185.1X
-UNICODE_CI 318287 318890 852 0.0 3182870.3 186.8X
+UTF8_BINARY 2879 2881 3 0.0 28786.7 1.0X
+UTF8_LCASE 15268 15275 11 0.0 152677.0 5.3X
+UNICODE 316698 317100 569 0.0 3166978.2 110.0X
+UNICODE_CI 321232 321330 138 0.0 3212319.7 111.6X
-OpenJDK 64-Bit Server VM 21.0.7+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - startsWith: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 1992 1992 0 0.1 19918.0 1.0X
-UTF8_LCASE 10388 10388 0 0.0 103878.0 5.2X
-UNICODE 311363 311436 103 0.0 3113630.8 156.3X
-UNICODE_CI 314454 315188 1039 0.0 3144536.7 157.9X
+UTF8_BINARY 2046 2047 1 0.0 20457.6 1.0X
+UTF8_LCASE 11301 11302 1 0.0 113011.9 5.5X
+UNICODE 317856 318087 326 0.0 3178561.7 155.4X
+UNICODE_CI 321502 321592 127 0.0 3215019.0 157.2X
-OpenJDK 64-Bit Server VM 21.0.7+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - endsWith: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 2052 2052 1 0.0 20517.7 1.0X
-UTF8_LCASE 11198 11206 11 0.0 111983.3 5.5X
-UNICODE 330704 330753 70 0.0 3307040.2 161.2X
-UNICODE_CI 333624 334191 802 0.0 3336242.4 162.6X
+UTF8_BINARY 2053 2056 4 0.0 20526.5 1.0X
+UTF8_LCASE 11156 11158 2 0.0 111560.7 5.4X
+UNICODE 325433 327150 2428 0.0 3254333.6 158.5X
+UNICODE_CI 331312 331923 864 0.0 3313119.8 161.4X
-OpenJDK 64-Bit Server VM 21.0.7+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - initCap using impl execICU: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
--------------------------------------------------------------------------------------------------------------------------------------
-UNICODE 393 394 1 0.3 3934.9 1.0X
-UNICODE_CI 388 390 1 0.3 3882.2 1.0X
+UNICODE 369 370 0 0.3 3690.8 1.0X
+UNICODE_CI 369 370 0 0.3 3689.7 1.0X
-OpenJDK 64-Bit Server VM 21.0.7+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - initCap using impl execBinaryICU: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
--------------------------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 573 574 2 0.2 5727.7 1.0X
-UTF8_LCASE 573 574 1 0.2 5726.9 1.0X
-UNICODE 572 574 2 0.2 5722.1 1.0X
-UNICODE_CI 573 589 33 0.2 5726.9 1.0X
+UTF8_BINARY 565 566 1 0.2 5650.4 1.0X
+UTF8_LCASE 565 566 1 0.2 5653.0 1.0X
+UNICODE 566 566 1 0.2 5655.2 1.0X
+UNICODE_CI 565 566 0 0.2 5651.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.7+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - initCap using impl execBinary: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
-----------------------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 137 138 1 0.7 1371.0 1.0X
-UTF8_LCASE 137 138 1 0.7 1371.4 1.0X
-UNICODE 137 138 1 0.7 1370.2 1.0X
-UNICODE_CI 137 138 1 0.7 1370.3 1.0X
+UTF8_BINARY 137 138 1 0.7 1372.6 1.0X
+UTF8_LCASE 137 138 0 0.7 1371.7 1.0X
+UNICODE 137 138 1 0.7 1373.9 1.0X
+UNICODE_CI 137 138 0 0.7 1373.3 1.0X
-OpenJDK 64-Bit Server VM 21.0.7+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - initCap using impl execLowercase: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
--------------------------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 388 389 1 0.3 3878.0 1.0X
-UTF8_LCASE 388 389 1 0.3 3879.3 1.0X
-UNICODE 388 389 1 0.3 3879.9 1.0X
-UNICODE_CI 388 389 0 0.3 3878.4 1.0X
+UTF8_BINARY 370 371 1 0.3 3696.7 1.0X
+UTF8_LCASE 369 370 1 0.3 3685.8 1.0X
+UNICODE 369 370 1 0.3 3690.9 1.0X
+UNICODE_CI 369 370 0 0.3 3693.8 1.0X
diff --git a/sql/core/benchmarks/CollationBenchmark-results.txt b/sql/core/benchmarks/CollationBenchmark-results.txt
index 4b178530f1c6e..bf5cd74a86c6f 100644
--- a/sql/core/benchmarks/CollationBenchmark-results.txt
+++ b/sql/core/benchmarks/CollationBenchmark-results.txt
@@ -1,88 +1,88 @@
-OpenJDK 64-Bit Server VM 17.0.15+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - equalsFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
--------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 1750 1751 2 0.1 17498.8 1.0X
-UTF8_LCASE 2882 2884 3 0.0 28824.3 1.6X
-UNICODE 19141 19143 3 0.0 191411.8 10.9X
-UNICODE_CI 18699 18703 6 0.0 186988.2 10.7X
+UTF8_BINARY 1748 1750 2 0.1 17482.9 1.0X
+UTF8_LCASE 3042 3045 4 0.0 30424.8 1.7X
+UNICODE 19142 19156 19 0.0 191422.9 10.9X
+UNICODE_CI 19141 19147 10 0.0 191405.2 10.9X
-OpenJDK 64-Bit Server VM 17.0.15+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - compareFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
---------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 1737 1738 2 0.1 17369.8 1.0X
-UTF8_LCASE 3079 3082 4 0.0 30791.2 1.8X
-UNICODE 19852 19859 9 0.0 198518.8 11.4X
-UNICODE_CI 19727 19728 1 0.0 197272.5 11.4X
+UTF8_BINARY 1708 1709 2 0.1 17079.7 1.0X
+UTF8_LCASE 3149 3150 2 0.0 31487.7 1.8X
+UNICODE 19352 19361 13 0.0 193515.4 11.3X
+UNICODE_CI 19185 19188 4 0.0 191851.2 11.2X
-OpenJDK 64-Bit Server VM 17.0.15+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - hashFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 3092 3093 1 0.0 30922.4 1.0X
-UTF8_LCASE 6473 6474 2 0.0 64728.0 2.1X
-UNICODE 66907 66913 9 0.0 669070.9 21.6X
-UNICODE_CI 55134 55219 119 0.0 551343.8 17.8X
+UTF8_BINARY 13960 13960 0 0.0 139601.6 1.0X
+UTF8_LCASE 25003 25005 2 0.0 250031.9 1.8X
+UNICODE 117914 117918 6 0.0 1179141.1 8.4X
+UNICODE_CI 160718 160758 57 0.0 1607177.0 11.5X
-OpenJDK 64-Bit Server VM 17.0.15+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - contains: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 2107 2109 2 0.0 21073.8 1.0X
-UTF8_LCASE 17995 17997 4 0.0 179946.3 8.5X
-UNICODE 318380 318887 717 0.0 3183800.7 151.1X
-UNICODE_CI 309694 309747 74 0.0 3096942.4 147.0X
+UTF8_BINARY 2244 2246 3 0.0 22444.5 1.0X
+UTF8_LCASE 17111 17112 1 0.0 171107.2 7.6X
+UNICODE 317260 318448 1680 0.0 3172604.4 141.4X
+UNICODE_CI 313112 313564 639 0.0 3131124.6 139.5X
-OpenJDK 64-Bit Server VM 17.0.15+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - startsWith: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 2502 2504 2 0.0 25023.7 1.0X
-UTF8_LCASE 17341 17344 4 0.0 173412.8 6.9X
-UNICODE 304576 305362 1112 0.0 3045764.2 121.7X
-UNICODE_CI 308506 309328 1161 0.0 3085064.4 123.3X
+UTF8_BINARY 2365 2374 13 0.0 23648.0 1.0X
+UTF8_LCASE 17098 17103 7 0.0 170981.8 7.2X
+UNICODE 308770 308831 86 0.0 3087700.9 130.6X
+UNICODE_CI 312503 312717 303 0.0 3125027.8 132.1X
-OpenJDK 64-Bit Server VM 17.0.15+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - endsWith: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 2518 2518 0 0.0 25184.0 1.0X
-UTF8_LCASE 17347 17352 7 0.0 173470.3 6.9X
-UNICODE 319689 320154 658 0.0 3196888.5 126.9X
-UNICODE_CI 322042 322706 939 0.0 3220420.5 127.9X
+UTF8_BINARY 2481 2481 0 0.0 24811.2 1.0X
+UTF8_LCASE 17154 17155 2 0.0 171542.6 6.9X
+UNICODE 328959 329495 758 0.0 3289591.2 132.6X
+UNICODE_CI 333560 334318 1073 0.0 3335595.5 134.4X
-OpenJDK 64-Bit Server VM 17.0.15+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - initCap using impl execICU: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
--------------------------------------------------------------------------------------------------------------------------------------
-UNICODE 420 421 2 0.2 4196.0 1.0X
-UNICODE_CI 419 420 1 0.2 4190.7 1.0X
+UNICODE 298 299 1 0.3 2982.0 1.0X
+UNICODE_CI 298 299 0 0.3 2981.1 1.0X
-OpenJDK 64-Bit Server VM 17.0.15+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - initCap using impl execBinaryICU: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
--------------------------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 598 599 1 0.2 5982.9 1.0X
-UTF8_LCASE 598 599 1 0.2 5980.5 1.0X
-UNICODE 598 600 3 0.2 5979.3 1.0X
-UNICODE_CI 598 599 1 0.2 5978.9 1.0X
+UTF8_BINARY 577 578 1 0.2 5767.1 1.0X
+UTF8_LCASE 576 577 1 0.2 5762.1 1.0X
+UNICODE 577 577 1 0.2 5768.5 1.0X
+UNICODE_CI 577 577 0 0.2 5768.9 1.0X
-OpenJDK 64-Bit Server VM 17.0.15+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - initCap using impl execBinary: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
-----------------------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 181 182 1 0.6 1810.0 1.0X
-UTF8_LCASE 181 182 1 0.6 1810.8 1.0X
-UNICODE 181 182 1 0.6 1810.1 1.0X
-UNICODE_CI 181 182 0 0.6 1810.9 1.0X
+UTF8_BINARY 185 185 1 0.5 1847.2 1.0X
+UTF8_LCASE 185 185 0 0.5 1846.8 1.0X
+UNICODE 185 185 0 0.5 1847.5 1.0X
+UNICODE_CI 185 185 1 0.5 1846.5 1.0X
-OpenJDK 64-Bit Server VM 17.0.15+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - initCap using impl execLowercase: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
--------------------------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 419 420 1 0.2 4190.1 1.0X
-UTF8_LCASE 419 420 1 0.2 4194.1 1.0X
-UNICODE 419 420 1 0.2 4188.6 1.0X
-UNICODE_CI 419 420 1 0.2 4189.5 1.0X
+UTF8_BINARY 317 317 0 0.3 3166.9 1.0X
+UTF8_LCASE 317 318 0 0.3 3170.9 1.0X
+UNICODE 316 317 0 0.3 3163.9 1.0X
+UNICODE_CI 316 317 1 0.3 3163.6 1.0X
diff --git a/sql/core/benchmarks/CollationNonASCIIBenchmark-jdk21-results.txt b/sql/core/benchmarks/CollationNonASCIIBenchmark-jdk21-results.txt
index 3c0b9d1647ee7..cdce26c80a5d1 100644
--- a/sql/core/benchmarks/CollationNonASCIIBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/CollationNonASCIIBenchmark-jdk21-results.txt
@@ -1,88 +1,88 @@
-OpenJDK 64-Bit Server VM 21.0.7+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - equalsFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
--------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 193 193 1 0.2 4819.3 1.0X
-UTF8_LCASE 7333 7334 1 0.0 183334.3 38.0X
-UNICODE 5117 5122 8 0.0 127913.6 26.5X
-UNICODE_CI 5127 5138 16 0.0 128171.7 26.6X
+UTF8_BINARY 193 194 2 0.2 4825.3 1.0X
+UTF8_LCASE 7364 7365 2 0.0 184098.9 38.2X
+UNICODE 5138 5143 7 0.0 128446.5 26.6X
+UNICODE_CI 5273 5274 1 0.0 131830.6 27.3X
-OpenJDK 64-Bit Server VM 21.0.7+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - compareFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
---------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 320 320 0 0.1 7988.9 1.0X
-UTF8_LCASE 7380 7384 6 0.0 184501.6 23.1X
-UNICODE 5380 5382 2 0.0 134502.1 16.8X
-UNICODE_CI 5329 5330 1 0.0 133234.0 16.7X
+UTF8_BINARY 297 297 0 0.1 7428.2 1.0X
+UTF8_LCASE 7223 7224 1 0.0 180585.9 24.3X
+UNICODE 5167 5168 1 0.0 129185.4 17.4X
+UNICODE_CI 5118 5124 7 0.0 127959.9 17.2X
-OpenJDK 64-Bit Server VM 21.0.7+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - hashFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 389 391 2 0.1 9731.7 1.0X
-UTF8_LCASE 3765 3767 2 0.0 94136.1 9.7X
-UNICODE 14511 14522 16 0.0 362778.4 37.3X
-UNICODE_CI 11611 11629 25 0.0 290286.6 29.8X
+UTF8_BINARY 1781 1785 6 0.0 44533.6 1.0X
+UTF8_LCASE 6031 6031 1 0.0 150765.3 3.4X
+UNICODE 21149 21157 12 0.0 528720.3 11.9X
+UNICODE_CI 27480 27489 13 0.0 686999.9 15.4X
-OpenJDK 64-Bit Server VM 21.0.7+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - contains: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 397 398 1 0.1 9932.6 1.0X
-UTF8_LCASE 8126 8133 10 0.0 203142.7 20.5X
-UNICODE 59149 59172 32 0.0 1478723.8 148.9X
-UNICODE_CI 59069 59086 24 0.0 1476724.4 148.7X
+UTF8_BINARY 527 527 0 0.1 13166.7 1.0X
+UTF8_LCASE 9131 9135 6 0.0 228267.9 17.3X
+UNICODE 60953 60989 52 0.0 1523813.1 115.7X
+UNICODE_CI 62495 62503 12 0.0 1562377.7 118.7X
-OpenJDK 64-Bit Server VM 21.0.7+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - startsWith: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 322 322 0 0.1 8057.5 1.0X
-UTF8_LCASE 5402 5402 0 0.0 135048.8 16.8X
-UNICODE 58820 58831 15 0.0 1470505.1 182.5X
-UNICODE_CI 59412 59560 210 0.0 1485296.8 184.3X
+UTF8_BINARY 275 276 0 0.1 6882.2 1.0X
+UTF8_LCASE 5751 5751 1 0.0 143769.6 20.9X
+UNICODE 62099 62196 139 0.0 1552462.9 225.6X
+UNICODE_CI 62870 62928 82 0.0 1571750.5 228.4X
-OpenJDK 64-Bit Server VM 21.0.7+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - endsWith: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 313 313 0 0.1 7828.0 1.0X
-UTF8_LCASE 5456 5474 26 0.0 136400.1 17.4X
-UNICODE 64778 64790 16 0.0 1619459.4 206.9X
-UNICODE_CI 63831 63915 117 0.0 1595786.8 203.9X
+UTF8_BINARY 260 260 0 0.2 6502.2 1.0X
+UTF8_LCASE 5910 5914 6 0.0 147754.7 22.7X
+UNICODE 68219 68299 114 0.0 1705463.3 262.3X
+UNICODE_CI 67093 67462 523 0.0 1677318.2 258.0X
-OpenJDK 64-Bit Server VM 21.0.7+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - initCap using impl execICU: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
--------------------------------------------------------------------------------------------------------------------------------------
-UNICODE 216 216 0 0.2 5394.2 1.0X
-UNICODE_CI 214 215 0 0.2 5352.1 1.0X
+UNICODE 207 208 1 0.2 5176.3 1.0X
+UNICODE_CI 207 208 1 0.2 5174.7 1.0X
-OpenJDK 64-Bit Server VM 21.0.7+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - initCap using impl execBinaryICU: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
--------------------------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 323 334 25 0.1 8067.4 1.0X
-UTF8_LCASE 323 323 0 0.1 8064.4 1.0X
-UNICODE 322 323 1 0.1 8057.2 1.0X
-UNICODE_CI 323 323 1 0.1 8063.9 1.0X
+UTF8_BINARY 323 323 0 0.1 8067.0 1.0X
+UTF8_LCASE 323 323 0 0.1 8067.9 1.0X
+UNICODE 322 323 1 0.1 8062.3 1.0X
+UNICODE_CI 322 323 0 0.1 8061.2 1.0X
-OpenJDK 64-Bit Server VM 21.0.7+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - initCap using impl execBinary: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
-----------------------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 86 87 0 0.5 2149.3 1.0X
-UTF8_LCASE 86 87 0 0.5 2145.8 1.0X
-UNICODE 86 87 0 0.5 2150.8 1.0X
-UNICODE_CI 86 87 1 0.5 2151.2 1.0X
+UTF8_BINARY 85 86 0 0.5 2121.4 1.0X
+UTF8_LCASE 85 86 0 0.5 2122.4 1.0X
+UNICODE 85 86 0 0.5 2120.1 1.0X
+UNICODE_CI 85 86 0 0.5 2119.1 1.0X
-OpenJDK 64-Bit Server VM 21.0.7+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - initCap using impl execLowercase: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
--------------------------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 214 214 0 0.2 5345.7 1.0X
-UTF8_LCASE 214 220 12 0.2 5349.8 1.0X
-UNICODE 214 214 1 0.2 5340.1 1.0X
-UNICODE_CI 214 214 0 0.2 5339.8 1.0X
+UTF8_BINARY 208 209 1 0.2 5195.0 1.0X
+UTF8_LCASE 208 208 0 0.2 5197.6 1.0X
+UNICODE 208 208 0 0.2 5196.5 1.0X
+UNICODE_CI 208 209 1 0.2 5198.4 1.0X
diff --git a/sql/core/benchmarks/CollationNonASCIIBenchmark-results.txt b/sql/core/benchmarks/CollationNonASCIIBenchmark-results.txt
index 21edf6d64859c..3f2d6fc1edce8 100644
--- a/sql/core/benchmarks/CollationNonASCIIBenchmark-results.txt
+++ b/sql/core/benchmarks/CollationNonASCIIBenchmark-results.txt
@@ -1,88 +1,88 @@
-OpenJDK 64-Bit Server VM 17.0.15+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - equalsFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
--------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 171 172 1 0.2 4277.4 1.0X
-UTF8_LCASE 7351 7351 1 0.0 183764.8 43.0X
-UNICODE 5655 5657 3 0.0 141374.5 33.1X
-UNICODE_CI 5739 5741 2 0.0 143486.9 33.5X
+UTF8_BINARY 174 176 4 0.2 4357.6 1.0X
+UTF8_LCASE 7310 7312 2 0.0 182759.8 41.9X
+UNICODE 6486 6486 1 0.0 162152.0 37.2X
+UNICODE_CI 6365 6365 1 0.0 159118.5 36.5X
-OpenJDK 64-Bit Server VM 17.0.15+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - compareFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
---------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 300 300 0 0.1 7500.6 1.0X
-UTF8_LCASE 7137 7165 40 0.0 178413.7 23.8X
-UNICODE 5506 5510 5 0.0 137660.9 18.4X
-UNICODE_CI 5468 5473 7 0.0 136699.1 18.2X
+UTF8_BINARY 479 480 1 0.1 11966.1 1.0X
+UTF8_LCASE 7479 7480 1 0.0 186981.5 15.6X
+UNICODE 6517 6526 13 0.0 162913.6 13.6X
+UNICODE_CI 6467 6488 29 0.0 161684.0 13.5X
-OpenJDK 64-Bit Server VM 17.0.15+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - hashFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 717 718 2 0.1 17925.8 1.0X
-UTF8_LCASE 3642 3643 1 0.0 91045.7 5.1X
-UNICODE 14060 14072 17 0.0 351498.4 19.6X
-UNICODE_CI 12921 12923 2 0.0 323026.0 18.0X
+UTF8_BINARY 1976 1978 2 0.0 49402.1 1.0X
+UTF8_LCASE 6319 6322 4 0.0 157976.7 3.2X
+UNICODE 22688 22698 14 0.0 567210.9 11.5X
+UNICODE_CI 29579 29579 1 0.0 739471.9 15.0X
-OpenJDK 64-Bit Server VM 17.0.15+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - contains: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 552 553 1 0.1 13806.7 1.0X
-UTF8_LCASE 8430 8436 8 0.0 210740.5 15.3X
-UNICODE 59134 59277 202 0.0 1478349.1 107.1X
-UNICODE_CI 55157 55368 299 0.0 1378923.9 99.9X
+UTF8_BINARY 604 604 1 0.1 15092.4 1.0X
+UTF8_LCASE 8491 8518 38 0.0 212269.4 14.1X
+UNICODE 60312 60430 167 0.0 1507792.3 99.9X
+UNICODE_CI 57877 57884 10 0.0 1446922.4 95.9X
-OpenJDK 64-Bit Server VM 17.0.15+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - startsWith: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 335 336 0 0.1 8384.1 1.0X
-UTF8_LCASE 5312 5314 3 0.0 132802.3 15.8X
-UNICODE 54850 54887 53 0.0 1371249.7 163.6X
-UNICODE_CI 55529 55678 211 0.0 1388216.9 165.6X
+UTF8_BINARY 376 377 1 0.1 9402.1 1.0X
+UTF8_LCASE 5370 5371 1 0.0 134249.8 14.3X
+UNICODE 57074 57101 39 0.0 1426838.7 151.8X
+UNICODE_CI 57428 57492 90 0.0 1435699.4 152.7X
-OpenJDK 64-Bit Server VM 17.0.15+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - endsWith: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 338 338 1 0.1 8437.7 1.0X
-UTF8_LCASE 5363 5389 37 0.0 134077.3 15.9X
-UNICODE 61997 62088 128 0.0 1549932.7 183.7X
-UNICODE_CI 61712 61836 176 0.0 1542800.4 182.8X
+UTF8_BINARY 376 377 1 0.1 9389.3 1.0X
+UTF8_LCASE 5340 5341 0 0.0 133505.3 14.2X
+UNICODE 63927 63986 83 0.0 1598179.3 170.2X
+UNICODE_CI 63400 63579 253 0.0 1585004.3 168.8X
-OpenJDK 64-Bit Server VM 17.0.15+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - initCap using impl execICU: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
--------------------------------------------------------------------------------------------------------------------------------------
-UNICODE 225 226 0 0.2 5628.8 1.0X
-UNICODE_CI 225 226 0 0.2 5627.4 1.0X
+UNICODE 266 268 2 0.2 6649.1 1.0X
+UNICODE_CI 267 269 1 0.1 6681.8 1.0X
-OpenJDK 64-Bit Server VM 17.0.15+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - initCap using impl execBinaryICU: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
--------------------------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 316 317 1 0.1 7904.0 1.0X
-UTF8_LCASE 316 317 1 0.1 7905.4 1.0X
-UNICODE 316 317 1 0.1 7899.7 1.0X
-UNICODE_CI 316 317 0 0.1 7904.5 1.0X
+UTF8_BINARY 314 315 0 0.1 7854.1 1.0X
+UTF8_LCASE 315 315 0 0.1 7863.0 1.0X
+UNICODE 314 315 0 0.1 7858.8 1.0X
+UNICODE_CI 314 315 1 0.1 7852.1 1.0X
-OpenJDK 64-Bit Server VM 17.0.15+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - initCap using impl execBinary: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
-----------------------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 126 127 0 0.3 3147.2 1.0X
-UTF8_LCASE 126 127 1 0.3 3154.2 1.0X
-UNICODE 126 127 1 0.3 3149.0 1.0X
-UNICODE_CI 126 127 1 0.3 3157.4 1.0X
+UTF8_BINARY 158 160 1 0.3 3947.4 1.0X
+UTF8_LCASE 159 160 1 0.3 3979.2 1.0X
+UNICODE 158 160 1 0.3 3962.3 1.0X
+UNICODE_CI 159 160 0 0.3 3983.1 1.0X
-OpenJDK 64-Bit Server VM 17.0.15+6-LTS on Linux 6.11.0-1012-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collation unit benchmarks - initCap using impl execLowercase: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time
--------------------------------------------------------------------------------------------------------------------------------------------
-UTF8_BINARY 224 225 1 0.2 5598.3 1.0X
-UTF8_LCASE 224 225 1 0.2 5596.6 1.0X
-UNICODE 224 225 0 0.2 5599.3 1.0X
-UNICODE_CI 224 225 1 0.2 5596.4 1.0X
+UTF8_BINARY 265 271 8 0.2 6630.7 1.0X
+UTF8_LCASE 265 267 2 0.2 6622.7 1.0X
+UNICODE 265 266 1 0.2 6633.8 1.0X
+UNICODE_CI 267 269 1 0.1 6677.1 1.0X
diff --git a/sql/core/benchmarks/ColumnarBatchBenchmark-jdk21-results.txt b/sql/core/benchmarks/ColumnarBatchBenchmark-jdk21-results.txt
index 5e4032460e6af..b0a81778443ee 100644
--- a/sql/core/benchmarks/ColumnarBatchBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/ColumnarBatchBenchmark-jdk21-results.txt
@@ -2,58 +2,58 @@
Int Read/Write
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Int Read/Write: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Java Array 119 120 0 2745.9 0.4 1.0X
-ByteBuffer Unsafe 199 203 5 1648.0 0.6 0.6X
-ByteBuffer API 349 350 1 937.6 1.1 0.3X
-DirectByteBuffer 419 420 1 781.8 1.3 0.3X
-Unsafe Buffer 155 155 0 2119.1 0.5 0.8X
-Column(on heap) 124 124 0 2642.9 0.4 1.0X
-Column(off heap) 154 155 1 2124.3 0.5 0.8X
-Column(off heap direct) 155 155 0 2118.0 0.5 0.8X
-UnsafeRow (on heap) 435 435 0 753.9 1.3 0.3X
-UnsafeRow (off heap) 295 296 1 1111.1 0.9 0.4X
-Column On Heap Append 312 319 6 1049.0 1.0 0.4X
+Java Array 123 123 1 2664.6 0.4 1.0X
+ByteBuffer Unsafe 191 197 7 1714.7 0.6 0.6X
+ByteBuffer API 443 443 0 740.2 1.4 0.3X
+DirectByteBuffer 419 420 0 781.4 1.3 0.3X
+Unsafe Buffer 154 154 0 2124.5 0.5 0.8X
+Column(on heap) 124 124 0 2643.6 0.4 1.0X
+Column(off heap) 154 155 0 2121.4 0.5 0.8X
+Column(off heap direct) 154 155 0 2121.0 0.5 0.8X
+UnsafeRow (on heap) 435 440 12 754.0 1.3 0.3X
+UnsafeRow (off heap) 295 296 1 1110.5 0.9 0.4X
+Column On Heap Append 313 314 2 1048.3 1.0 0.4X
================================================================================================
Boolean Read/Write
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Boolean Read/Write: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Bitset 523 525 2 641.1 1.6 1.0X
-Byte Array 250 251 2 1341.9 0.7 2.1X
+Bitset 432 433 0 776.0 1.3 1.0X
+Byte Array 237 237 0 1417.2 0.7 1.8X
================================================================================================
String Read/Write
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
String Read/Write: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-On Heap 126 128 2 130.2 7.7 1.0X
-Off Heap 402 424 19 40.8 24.5 0.3X
+On Heap 126 129 2 129.7 7.7 1.0X
+Off Heap 410 469 34 40.0 25.0 0.3X
================================================================================================
Array Vector Read
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Array Vector Read: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-On Heap Read Size Only 86 86 0 1898.6 0.5 1.0X
-Off Heap Read Size Only 358 361 2 457.2 2.2 0.2X
-On Heap Read Elements 2405 2407 3 68.1 14.7 0.0X
-Off Heap Read Elements 2648 2652 5 61.9 16.2 0.0X
+On Heap Read Size Only 88 88 0 1865.6 0.5 1.0X
+Off Heap Read Size Only 427 427 0 383.6 2.6 0.2X
+On Heap Read Elements 2480 2564 120 66.1 15.1 0.0X
+Off Heap Read Elements 2448 2496 68 66.9 14.9 0.0X
diff --git a/sql/core/benchmarks/ColumnarBatchBenchmark-results.txt b/sql/core/benchmarks/ColumnarBatchBenchmark-results.txt
index d2123457fe149..e0a39640e547c 100644
--- a/sql/core/benchmarks/ColumnarBatchBenchmark-results.txt
+++ b/sql/core/benchmarks/ColumnarBatchBenchmark-results.txt
@@ -2,58 +2,58 @@
Int Read/Write
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Int Read/Write: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Java Array 175 176 0 1869.0 0.5 1.0X
-ByteBuffer Unsafe 304 307 1 1077.0 0.9 0.6X
-ByteBuffer API 510 511 1 642.7 1.6 0.3X
-DirectByteBuffer 406 406 0 807.8 1.2 0.4X
-Unsafe Buffer 162 163 1 2023.2 0.5 1.1X
-Column(on heap) 171 172 1 1912.8 0.5 1.0X
-Column(off heap) 163 163 1 2015.1 0.5 1.1X
-Column(off heap direct) 158 158 0 2078.2 0.5 1.1X
-UnsafeRow (on heap) 439 439 0 747.1 1.3 0.4X
-UnsafeRow (off heap) 315 316 2 1041.6 1.0 0.6X
-Column On Heap Append 389 390 1 842.3 1.2 0.5X
+Java Array 196 196 0 1673.2 0.6 1.0X
+ByteBuffer Unsafe 288 288 0 1139.3 0.9 0.7X
+ByteBuffer API 510 511 1 643.0 1.6 0.4X
+DirectByteBuffer 469 470 0 698.1 1.4 0.4X
+Unsafe Buffer 162 163 0 2016.7 0.5 1.2X
+Column(on heap) 171 172 0 1911.9 0.5 1.1X
+Column(off heap) 163 163 0 2010.0 0.5 1.2X
+Column(off heap direct) 158 158 0 2074.5 0.5 1.2X
+UnsafeRow (on heap) 439 442 6 747.2 1.3 0.4X
+UnsafeRow (off heap) 314 317 2 1043.7 1.0 0.6X
+Column On Heap Append 363 364 0 902.1 1.1 0.5X
================================================================================================
Boolean Read/Write
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Boolean Read/Write: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Bitset 445 447 1 753.3 1.3 1.0X
-Byte Array 263 264 1 1274.2 0.8 1.7X
+Bitset 479 480 0 699.8 1.4 1.0X
+Byte Array 263 264 0 1274.3 0.8 1.8X
================================================================================================
String Read/Write
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
String Read/Write: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-On Heap 142 144 1 115.3 8.7 1.0X
-Off Heap 393 410 12 41.7 24.0 0.4X
+On Heap 142 143 2 115.2 8.7 1.0X
+Off Heap 367 394 18 44.7 22.4 0.4X
================================================================================================
Array Vector Read
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Array Vector Read: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-On Heap Read Size Only 88 88 0 1869.3 0.5 1.0X
-Off Heap Read Size Only 85 86 0 1920.0 0.5 1.0X
-On Heap Read Elements 2415 2417 3 67.8 14.7 0.0X
-Off Heap Read Elements 2880 2880 1 56.9 17.6 0.0X
+On Heap Read Size Only 90 90 0 1818.9 0.5 1.0X
+Off Heap Read Size Only 86 86 0 1915.7 0.5 1.1X
+On Heap Read Elements 2195 2199 6 74.6 13.4 0.0X
+Off Heap Read Elements 2791 2793 3 58.7 17.0 0.0X
diff --git a/sql/core/benchmarks/CompressionSchemeBenchmark-jdk21-results.txt b/sql/core/benchmarks/CompressionSchemeBenchmark-jdk21-results.txt
index e177803bb1efa..6901556ad1faf 100644
--- a/sql/core/benchmarks/CompressionSchemeBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/CompressionSchemeBenchmark-jdk21-results.txt
@@ -2,136 +2,136 @@
Compression Scheme Benchmark
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
BOOLEAN Encode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000) 1 1 0 47214.4 0.0 1.0X
-RunLengthEncoding(2.515) 943 1094 261 71.2 14.1 0.0X
-BooleanBitSet(0.125) 292 293 0 229.5 4.4 0.0X
+PassThrough(1.000) 1 1 0 48542.5 0.0 1.0X
+RunLengthEncoding(2.497) 940 946 9 71.4 14.0 0.0X
+BooleanBitSet(0.125) 274 274 0 245.1 4.1 0.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
BOOLEAN Decode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough 212 213 1 317.1 3.2 1.0X
-RunLengthEncoding 618 618 0 108.6 9.2 0.3X
-BooleanBitSet 668 669 0 100.4 10.0 0.3X
+PassThrough 211 212 1 318.3 3.1 1.0X
+RunLengthEncoding 580 581 1 115.7 8.6 0.4X
+BooleanBitSet 678 681 4 98.9 10.1 0.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SHORT Encode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000) 3 3 0 24468.7 0.0 1.0X
-RunLengthEncoding(1.504) 1066 1068 3 63.0 15.9 0.0X
+PassThrough(1.000) 3 3 0 24472.0 0.0 1.0X
+RunLengthEncoding(1.501) 1131 1131 0 59.3 16.9 0.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SHORT Decode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough 710 710 0 94.5 10.6 1.0X
-RunLengthEncoding 1000 1000 0 67.1 14.9 0.7X
+PassThrough 627 628 1 107.0 9.3 1.0X
+RunLengthEncoding 986 987 0 68.0 14.7 0.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SHORT Encode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000) 3 3 0 24523.0 0.0 1.0X
-RunLengthEncoding(2.017) 1108 1109 0 60.6 16.5 0.0X
+PassThrough(1.000) 3 3 0 24323.5 0.0 1.0X
+RunLengthEncoding(2.008) 1170 1170 0 57.4 17.4 0.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SHORT Decode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough 648 650 3 103.6 9.7 1.0X
-RunLengthEncoding 939 942 2 71.4 14.0 0.7X
+PassThrough 606 607 1 110.8 9.0 1.0X
+RunLengthEncoding 962 963 1 69.7 14.3 0.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
INT Encode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000) 6 6 0 11110.5 0.1 1.0X
-RunLengthEncoding(0.996) 1029 1029 1 65.2 15.3 0.0X
-DictionaryEncoding(0.500) 343 344 1 195.6 5.1 0.0X
-IntDelta(0.250) 110 110 0 609.4 1.6 0.1X
+PassThrough(1.000) 6 6 0 11620.1 0.1 1.0X
+RunLengthEncoding(0.998) 1048 1050 2 64.0 15.6 0.0X
+DictionaryEncoding(0.500) 518 519 0 129.5 7.7 0.0X
+IntDelta(0.250) 141 142 1 477.0 2.1 0.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
INT Decode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough 648 651 2 103.6 9.7 1.0X
-RunLengthEncoding 1222 1224 3 54.9 18.2 0.5X
-DictionaryEncoding 544 545 0 123.3 8.1 1.2X
-IntDelta 502 504 2 133.6 7.5 1.3X
+PassThrough 647 648 2 103.7 9.6 1.0X
+RunLengthEncoding 1120 1127 9 59.9 16.7 0.6X
+DictionaryEncoding 549 551 3 122.1 8.2 1.2X
+IntDelta 501 502 0 133.9 7.5 1.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
INT Encode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000) 6 6 0 11275.3 0.1 1.0X
-RunLengthEncoding(1.336) 1064 1064 0 63.1 15.9 0.0X
-DictionaryEncoding(0.501) 343 343 0 195.6 5.1 0.0X
-IntDelta(0.250) 111 111 1 606.8 1.6 0.1X
+PassThrough(1.000) 6 6 0 11163.4 0.1 1.0X
+RunLengthEncoding(1.335) 1078 1078 0 62.3 16.1 0.0X
+DictionaryEncoding(0.501) 399 399 0 168.2 5.9 0.0X
+IntDelta(0.250) 132 133 0 506.7 2.0 0.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
INT Decode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough 731 733 2 91.8 10.9 1.0X
-RunLengthEncoding 1220 1223 4 55.0 18.2 0.6X
-DictionaryEncoding 671 673 2 100.0 10.0 1.1X
-IntDelta 524 603 53 128.1 7.8 1.4X
+PassThrough 710 710 1 94.5 10.6 1.0X
+RunLengthEncoding 1216 1217 1 55.2 18.1 0.6X
+DictionaryEncoding 671 672 2 100.1 10.0 1.1X
+IntDelta 522 522 0 128.6 7.8 1.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
LONG Encode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000) 13 14 0 4998.1 0.2 1.0X
-RunLengthEncoding(0.749) 1056 1056 0 63.5 15.7 0.0X
-DictionaryEncoding(0.250) 409 410 2 164.2 6.1 0.0X
-LongDelta(0.125) 111 111 0 603.3 1.7 0.1X
+PassThrough(1.000) 14 14 0 4948.2 0.2 1.0X
+RunLengthEncoding(0.749) 1072 1072 0 62.6 16.0 0.0X
+DictionaryEncoding(0.250) 424 425 1 158.3 6.3 0.0X
+LongDelta(0.125) 133 134 0 504.1 2.0 0.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
LONG Decode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough 837 838 2 80.2 12.5 1.0X
-RunLengthEncoding 1260 1263 3 53.2 18.8 0.7X
-DictionaryEncoding 810 818 11 82.9 12.1 1.0X
-LongDelta 544 546 2 123.3 8.1 1.5X
+PassThrough 774 775 0 86.7 11.5 1.0X
+RunLengthEncoding 1244 1255 16 54.0 18.5 0.6X
+DictionaryEncoding 733 733 0 91.6 10.9 1.1X
+LongDelta 543 543 0 123.6 8.1 1.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
LONG Encode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000) 13 13 0 5021.1 0.2 1.0X
-RunLengthEncoding(0.999) 1087 1090 4 61.7 16.2 0.0X
-DictionaryEncoding(0.251) 409 410 2 164.0 6.1 0.0X
-LongDelta(0.125) 111 112 1 603.0 1.7 0.1X
+PassThrough(1.000) 18 18 0 3753.2 0.3 1.0X
+RunLengthEncoding(1.000) 1103 1104 1 60.8 16.4 0.0X
+DictionaryEncoding(0.251) 426 426 0 157.6 6.3 0.0X
+LongDelta(0.125) 133 134 0 503.1 2.0 0.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
LONG Decode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough 816 817 2 82.3 12.2 1.0X
-RunLengthEncoding 1235 1236 1 54.4 18.4 0.7X
-DictionaryEncoding 811 812 2 82.8 12.1 1.0X
-LongDelta 671 674 3 100.0 10.0 1.2X
+PassThrough 774 775 1 86.7 11.5 1.0X
+RunLengthEncoding 1242 1243 1 54.0 18.5 0.6X
+DictionaryEncoding 734 735 1 91.4 10.9 1.1X
+LongDelta 671 671 0 100.0 10.0 1.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
STRING Encode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000) 26 29 1 2629.0 0.4 1.0X
-RunLengthEncoding(0.889) 2060 2063 6 32.6 30.7 0.0X
-DictionaryEncoding(0.167) 2176 2179 5 30.8 32.4 0.0X
+PassThrough(1.000) 20 20 0 3349.9 0.3 1.0X
+RunLengthEncoding(0.894) 2062 2062 0 32.5 30.7 0.0X
+DictionaryEncoding(0.167) 2312 2317 8 29.0 34.4 0.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
STRING Decode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough 1650 1670 28 40.7 24.6 1.0X
-RunLengthEncoding 2510 2512 3 26.7 37.4 0.7X
-DictionaryEncoding 1990 1998 12 33.7 29.6 0.8X
+PassThrough 1632 1654 31 41.1 24.3 1.0X
+RunLengthEncoding 2508 2510 2 26.8 37.4 0.7X
+DictionaryEncoding 2009 2011 2 33.4 29.9 0.8X
diff --git a/sql/core/benchmarks/CompressionSchemeBenchmark-results.txt b/sql/core/benchmarks/CompressionSchemeBenchmark-results.txt
index 0e846555ef114..644366d42a968 100644
--- a/sql/core/benchmarks/CompressionSchemeBenchmark-results.txt
+++ b/sql/core/benchmarks/CompressionSchemeBenchmark-results.txt
@@ -2,136 +2,136 @@
Compression Scheme Benchmark
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
BOOLEAN Encode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000) 1 1 0 49097.6 0.0 1.0X
-RunLengthEncoding(2.503) 887 892 5 75.6 13.2 0.0X
-BooleanBitSet(0.125) 297 297 0 226.1 4.4 0.0X
+PassThrough(1.000) 1 1 0 49126.6 0.0 1.0X
+RunLengthEncoding(2.503) 886 888 2 75.8 13.2 0.0X
+BooleanBitSet(0.125) 293 294 1 228.8 4.4 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
BOOLEAN Decode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough 168 168 0 400.3 2.5 1.0X
-RunLengthEncoding 554 554 1 121.2 8.3 0.3X
-BooleanBitSet 652 657 4 102.9 9.7 0.3X
+PassThrough 168 168 0 399.9 2.5 1.0X
+RunLengthEncoding 540 541 1 124.3 8.0 0.3X
+BooleanBitSet 655 659 4 102.5 9.8 0.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SHORT Encode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000) 3 3 0 24577.6 0.0 1.0X
-RunLengthEncoding(1.494) 998 1000 3 67.3 14.9 0.0X
+PassThrough(1.000) 3 3 0 23278.0 0.0 1.0X
+RunLengthEncoding(1.499) 1038 1039 1 64.7 15.5 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SHORT Decode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough 689 691 2 97.4 10.3 1.0X
-RunLengthEncoding 969 970 2 69.3 14.4 0.7X
+PassThrough 543 544 1 123.5 8.1 1.0X
+RunLengthEncoding 963 964 1 69.7 14.4 0.6X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SHORT Encode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000) 3 3 0 22331.2 0.0 1.0X
-RunLengthEncoding(2.005) 1008 1008 0 66.6 15.0 0.0X
+PassThrough(1.000) 3 3 0 23167.5 0.0 1.0X
+RunLengthEncoding(2.010) 1059 1061 3 63.4 15.8 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SHORT Decode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough 689 691 2 97.4 10.3 1.0X
-RunLengthEncoding 927 928 1 72.4 13.8 0.7X
+PassThrough 544 544 0 123.4 8.1 1.0X
+RunLengthEncoding 908 909 2 73.9 13.5 0.6X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
INT Encode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000) 6 6 0 11223.2 0.1 1.0X
-RunLengthEncoding(1.000) 958 959 1 70.0 14.3 0.0X
-DictionaryEncoding(0.500) 370 371 2 181.5 5.5 0.0X
-IntDelta(0.250) 110 110 1 609.9 1.6 0.1X
+PassThrough(1.000) 6 6 0 11254.0 0.1 1.0X
+RunLengthEncoding(1.006) 990 991 1 67.8 14.8 0.0X
+DictionaryEncoding(0.500) 384 385 0 174.6 5.7 0.0X
+IntDelta(0.250) 115 115 0 582.0 1.7 0.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
INT Decode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough 626 629 2 107.1 9.3 1.0X
-RunLengthEncoding 1051 1052 0 63.8 15.7 0.6X
-DictionaryEncoding 523 524 2 128.3 7.8 1.2X
-IntDelta 459 460 1 146.1 6.8 1.4X
+PassThrough 627 627 0 107.1 9.3 1.0X
+RunLengthEncoding 1177 1177 0 57.0 17.5 0.5X
+DictionaryEncoding 523 524 2 128.4 7.8 1.2X
+IntDelta 459 459 0 146.2 6.8 1.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
INT Encode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000) 6 6 0 11176.2 0.1 1.0X
-RunLengthEncoding(1.333) 975 977 2 68.8 14.5 0.0X
-DictionaryEncoding(0.501) 369 371 2 181.8 5.5 0.0X
-IntDelta(0.250) 110 110 1 609.1 1.6 0.1X
+PassThrough(1.000) 6 6 0 11285.5 0.1 1.0X
+RunLengthEncoding(1.339) 1009 1009 0 66.5 15.0 0.0X
+DictionaryEncoding(0.501) 386 386 1 174.0 5.7 0.0X
+IntDelta(0.250) 115 115 0 581.8 1.7 0.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
INT Decode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
PassThrough 710 712 2 94.5 10.6 1.0X
-RunLengthEncoding 1157 1159 3 58.0 17.2 0.6X
-DictionaryEncoding 544 570 50 123.3 8.1 1.3X
-IntDelta 598 599 1 112.2 8.9 1.2X
+RunLengthEncoding 1111 1111 0 60.4 16.6 0.6X
+DictionaryEncoding 542 548 7 123.8 8.1 1.3X
+IntDelta 595 596 1 112.7 8.9 1.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
LONG Encode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000) 13 18 1 5006.5 0.2 1.0X
-RunLengthEncoding(0.757) 1008 1008 1 66.6 15.0 0.0X
-DictionaryEncoding(0.250) 433 434 0 154.9 6.5 0.0X
-LongDelta(0.125) 116 116 1 580.0 1.7 0.1X
+PassThrough(1.000) 13 13 0 5021.3 0.2 1.0X
+RunLengthEncoding(0.751) 1045 1046 0 64.2 15.6 0.0X
+DictionaryEncoding(0.250) 456 461 12 147.2 6.8 0.0X
+LongDelta(0.125) 111 111 0 605.7 1.7 0.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
LONG Decode (Lower Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough 782 785 3 85.8 11.7 1.0X
-RunLengthEncoding 1187 1188 2 56.5 17.7 0.7X
-DictionaryEncoding 714 716 2 94.0 10.6 1.1X
-LongDelta 506 506 0 132.6 7.5 1.5X
+PassThrough 712 712 0 94.3 10.6 1.0X
+RunLengthEncoding 1169 1169 1 57.4 17.4 0.6X
+DictionaryEncoding 710 710 0 94.5 10.6 1.0X
+LongDelta 522 522 0 128.5 7.8 1.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
LONG Encode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000) 14 15 2 4970.3 0.2 1.0X
-RunLengthEncoding(1.003) 1009 1010 1 66.5 15.0 0.0X
-DictionaryEncoding(0.251) 435 436 2 154.3 6.5 0.0X
-LongDelta(0.125) 116 116 1 579.1 1.7 0.1X
+PassThrough(1.000) 13 14 0 5032.9 0.2 1.0X
+RunLengthEncoding(0.999) 1064 1065 0 63.0 15.9 0.0X
+DictionaryEncoding(0.251) 452 453 1 148.3 6.7 0.0X
+LongDelta(0.125) 111 111 0 603.9 1.7 0.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
LONG Decode (Higher Skew): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough 781 782 1 85.9 11.6 1.0X
-RunLengthEncoding 1195 1197 4 56.2 17.8 0.7X
-DictionaryEncoding 715 716 2 93.9 10.6 1.1X
-LongDelta 634 635 2 105.9 9.4 1.2X
+PassThrough 713 714 0 94.1 10.6 1.0X
+RunLengthEncoding 1175 1177 3 57.1 17.5 0.6X
+DictionaryEncoding 687 688 1 97.7 10.2 1.0X
+LongDelta 611 612 1 109.8 9.1 1.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
STRING Encode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough(1.000) 26 27 0 2611.3 0.4 1.0X
-RunLengthEncoding(0.892) 2003 2003 0 33.5 29.8 0.0X
-DictionaryEncoding(0.167) 2471 2472 2 27.2 36.8 0.0X
+PassThrough(1.000) 25 29 1 2732.0 0.4 1.0X
+RunLengthEncoding(0.895) 1991 1991 0 33.7 29.7 0.0X
+DictionaryEncoding(0.167) 1965 1966 2 34.2 29.3 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
STRING Decode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-PassThrough 1447 1472 35 46.4 21.6 1.0X
-RunLengthEncoding 2274 2275 1 29.5 33.9 0.6X
-DictionaryEncoding 2100 2100 1 32.0 31.3 0.7X
+PassThrough 1603 1604 2 41.9 23.9 1.0X
+RunLengthEncoding 2452 2453 1 27.4 36.5 0.7X
+DictionaryEncoding 2090 2090 0 32.1 31.1 0.8X
diff --git a/sql/core/benchmarks/ConstantColumnVectorBenchmark-jdk21-results.txt b/sql/core/benchmarks/ConstantColumnVectorBenchmark-jdk21-results.txt
index 873f19d3fff43..ef3b3ef806417 100644
--- a/sql/core/benchmarks/ConstantColumnVectorBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/ConstantColumnVectorBenchmark-jdk21-results.txt
@@ -1,280 +1,280 @@
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write with StringType, row length = 1: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 1 1 0 403153.6 0.0 1.0X
-OnHeapColumnVector 2682 2693 16 152.7 6.5 0.0X
-OffHeapColumnVector 3740 3741 1 109.5 9.1 0.0X
+ConstantColumnVector 1 1 0 394213.8 0.0 1.0X
+OnHeapColumnVector 2886 2891 7 141.9 7.0 0.0X
+OffHeapColumnVector 5302 5304 2 77.3 12.9 0.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write with StringType, row length = 5: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 1 1 0 458928.9 0.0 1.0X
-OnHeapColumnVector 4029 4031 4 101.7 9.8 0.0X
-OffHeapColumnVector 4613 4614 1 88.8 11.3 0.0X
+ConstantColumnVector 1 1 0 422517.2 0.0 1.0X
+OnHeapColumnVector 4022 4022 1 101.8 9.8 0.0X
+OffHeapColumnVector 4704 4706 3 87.1 11.5 0.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write with StringType, row length = 10: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 1 1 0 458451.7 0.0 1.0X
-OnHeapColumnVector 4208 4211 4 97.3 10.3 0.0X
-OffHeapColumnVector 4774 4774 0 85.8 11.7 0.0X
+ConstantColumnVector 1 1 0 422455.8 0.0 1.0X
+OnHeapColumnVector 4232 4255 32 96.8 10.3 0.0X
+OffHeapColumnVector 5072 5074 2 80.8 12.4 0.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write with StringType, row length = 15: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 1 1 0 458729.0 0.0 1.0X
-OnHeapColumnVector 4053 4053 0 101.1 9.9 0.0X
-OffHeapColumnVector 4629 4635 10 88.5 11.3 0.0X
+ConstantColumnVector 1 1 0 422731.3 0.0 1.0X
+OnHeapColumnVector 4260 4273 20 96.2 10.4 0.0X
+OffHeapColumnVector 4725 4735 14 86.7 11.5 0.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write with StringType, row length = 20: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 1 1 0 459972.5 0.0 1.0X
-OnHeapColumnVector 3920 3933 18 104.5 9.6 0.0X
-OffHeapColumnVector 4652 4656 5 88.0 11.4 0.0X
+ConstantColumnVector 1 1 0 422774.9 0.0 1.0X
+OnHeapColumnVector 4526 4526 0 90.5 11.0 0.0X
+OffHeapColumnVector 5028 5030 2 81.5 12.3 0.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write with StringType, row length = 30: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 1 1 0 458806.5 0.0 1.0X
-OnHeapColumnVector 4289 4289 0 95.5 10.5 0.0X
-OffHeapColumnVector 4958 4960 3 82.6 12.1 0.0X
+ConstantColumnVector 1 1 0 422525.9 0.0 1.0X
+OnHeapColumnVector 4698 4700 3 87.2 11.5 0.0X
+OffHeapColumnVector 5213 5215 4 78.6 12.7 0.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write with IntegerType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 1 1 0 720559.7 0.0 1.0X
-OnHeapColumnVector 15 15 0 27615.9 0.0 0.0X
-OffHeapColumnVector 61 62 1 6721.8 0.1 0.0X
+ConstantColumnVector 1 1 0 664352.7 0.0 1.0X
+OnHeapColumnVector 16 16 0 25464.2 0.0 0.0X
+OffHeapColumnVector 65 65 0 6277.8 0.2 0.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write with LongType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 1 1 0 655063.4 0.0 1.0X
-OnHeapColumnVector 30 31 1 13471.6 0.1 0.0X
-OffHeapColumnVector 61 62 2 6691.9 0.1 0.0X
+ConstantColumnVector 1 1 0 632716.8 0.0 1.0X
+OnHeapColumnVector 33 33 0 12471.3 0.1 0.0X
+OffHeapColumnVector 67 67 0 6156.8 0.2 0.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write with FloatType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 1 1 0 496475.8 0.0 1.0X
-OnHeapColumnVector 15 15 0 27289.6 0.0 0.1X
-OffHeapColumnVector 118 119 1 3470.7 0.3 0.0X
+ConstantColumnVector 1 1 0 457741.1 0.0 1.0X
+OnHeapColumnVector 16 16 0 25377.8 0.0 0.1X
+OffHeapColumnVector 128 128 0 3210.0 0.3 0.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write with DoubleType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 1 1 0 553741.0 0.0 1.0X
-OnHeapColumnVector 31 32 1 13108.1 0.1 0.0X
-OffHeapColumnVector 119 120 1 3454.9 0.3 0.0X
+ConstantColumnVector 1 1 0 366593.5 0.0 1.0X
+OnHeapColumnVector 33 33 0 12450.1 0.1 0.0X
+OffHeapColumnVector 129 130 0 3164.6 0.3 0.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test read with StringType, row length = 1: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 0 0 0 14401237.6 0.0 1.0X
-OnHeapColumnVector 100 102 1 4086.1 0.2 0.0X
-OffHeapColumnVector 4198 4227 41 97.6 10.3 0.0X
+ConstantColumnVector 0 0 0 13274135.5 0.0 1.0X
+OnHeapColumnVector 106 106 0 3881.1 0.3 0.0X
+OffHeapColumnVector 3773 3773 1 108.6 9.2 0.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test read with StringType, row length = 5: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 0 0 0 14401237.6 0.0 1.0X
-OnHeapColumnVector 3811 3832 30 107.5 9.3 0.0X
-OffHeapColumnVector 4210 4215 6 97.3 10.3 0.0X
+ConstantColumnVector 0 0 0 13274135.5 0.0 1.0X
+OnHeapColumnVector 4106 4112 10 99.8 10.0 0.0X
+OffHeapColumnVector 3778 3788 14 108.4 9.2 0.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test read with StringType, row length = 10: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 0 0 0 14401237.6 0.0 1.0X
-OnHeapColumnVector 3810 3820 14 107.5 9.3 0.0X
-OffHeapColumnVector 4138 4197 83 99.0 10.1 0.0X
+ConstantColumnVector 0 0 0 13274135.5 0.0 1.0X
+OnHeapColumnVector 4100 4101 2 99.9 10.0 0.0X
+OffHeapColumnVector 3780 3791 15 108.4 9.2 0.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test read with StringType, row length = 15: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 0 0 0 14401237.6 0.0 1.0X
-OnHeapColumnVector 3801 3831 42 107.8 9.3 0.0X
-OffHeapColumnVector 4196 4200 6 97.6 10.2 0.0X
+ConstantColumnVector 0 0 0 13274135.5 0.0 1.0X
+OnHeapColumnVector 4099 4103 5 99.9 10.0 0.0X
+OffHeapColumnVector 3758 3759 0 109.0 9.2 0.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test read with StringType, row length = 20: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 0 0 0 14401237.6 0.0 1.0X
-OnHeapColumnVector 3810 3824 20 107.5 9.3 0.0X
-OffHeapColumnVector 4163 4165 2 98.4 10.2 0.0X
+ConstantColumnVector 0 0 0 13274135.5 0.0 1.0X
+OnHeapColumnVector 4096 4097 2 100.0 10.0 0.0X
+OffHeapColumnVector 3752 3754 2 109.2 9.2 0.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test read with StringType, row length = 30: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 0 0 0 14401237.6 0.0 1.0X
-OnHeapColumnVector 3789 3794 8 108.1 9.2 0.0X
-OffHeapColumnVector 4180 4203 33 98.0 10.2 0.0X
+ConstantColumnVector 0 0 0 13274135.5 0.0 1.0X
+OnHeapColumnVector 4088 4089 1 100.2 10.0 0.0X
+OffHeapColumnVector 3751 3754 5 109.2 9.2 0.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test read with IntegerType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 0 0 0 3602747.8 0.0 1.0X
-OnHeapColumnVector 1 1 0 554206.3 0.0 0.2X
-OffHeapColumnVector 828 829 1 494.8 2.0 0.0X
+ConstantColumnVector 0 0 0 13274135.5 0.0 1.0X
+OnHeapColumnVector 1 1 0 510968.5 0.0 0.0X
+OffHeapColumnVector 893 893 0 458.6 2.2 0.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test read with LongType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 1723 1731 12 237.8 4.2 1.0X
-OnHeapColumnVector 1972 1974 3 207.7 4.8 0.9X
-OffHeapColumnVector 2497 2517 29 164.0 6.1 0.7X
+ConstantColumnVector 1831 1832 0 223.7 4.5 1.0X
+OnHeapColumnVector 2087 2087 0 196.3 5.1 0.9X
+OffHeapColumnVector 2609 2609 1 157.0 6.4 0.7X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test read with FloatType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 1741 1746 7 235.3 4.3 1.0X
-OnHeapColumnVector 1998 2009 16 205.0 4.9 0.9X
-OffHeapColumnVector 2598 2598 1 157.7 6.3 0.7X
+ConstantColumnVector 1854 1855 1 220.9 4.5 1.0X
+OnHeapColumnVector 2110 2110 0 194.1 5.2 0.9X
+OffHeapColumnVector 2623 2632 13 156.2 6.4 0.7X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test read with DoubleType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 1886 1890 6 217.2 4.6 1.0X
-OnHeapColumnVector 1994 1999 8 205.4 4.9 0.9X
-OffHeapColumnVector 2582 2594 17 158.6 6.3 0.7X
+ConstantColumnVector 1991 1991 1 205.8 4.9 1.0X
+OnHeapColumnVector 2143 2146 5 191.2 5.2 0.9X
+OffHeapColumnVector 2764 2765 1 148.2 6.7 0.7X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write and read with StringType, row length = 1: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 2352 2367 21 174.2 5.7 1.0X
-OnHeapColumnVector 5306 5327 30 77.2 13.0 0.4X
-OffHeapColumnVector 5835 5836 2 70.2 14.2 0.4X
+ConstantColumnVector 1953 1953 0 209.8 4.8 1.0X
+OnHeapColumnVector 5871 5914 61 69.8 14.3 0.3X
+OffHeapColumnVector 5194 5198 6 78.9 12.7 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write and read with StringType, row length = 5: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 2348 2350 2 174.4 5.7 1.0X
-OnHeapColumnVector 5374 5387 18 76.2 13.1 0.4X
-OffHeapColumnVector 5940 5990 71 69.0 14.5 0.4X
+ConstantColumnVector 1952 1953 2 209.9 4.8 1.0X
+OnHeapColumnVector 5843 5843 1 70.1 14.3 0.3X
+OffHeapColumnVector 5217 5218 2 78.5 12.7 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write and read with StringType, row length = 10: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 2335 2350 21 175.4 5.7 1.0X
-OnHeapColumnVector 5323 5370 66 77.0 13.0 0.4X
-OffHeapColumnVector 5828 5835 10 70.3 14.2 0.4X
+ConstantColumnVector 1954 1955 2 209.6 4.8 1.0X
+OnHeapColumnVector 5850 5851 1 70.0 14.3 0.3X
+OffHeapColumnVector 5215 5216 1 78.5 12.7 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write and read with StringType, row length = 15: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 2323 2332 13 176.3 5.7 1.0X
-OnHeapColumnVector 5271 5305 48 77.7 12.9 0.4X
-OffHeapColumnVector 5856 5861 7 69.9 14.3 0.4X
+ConstantColumnVector 1951 1952 2 210.0 4.8 1.0X
+OnHeapColumnVector 5848 5849 1 70.0 14.3 0.3X
+OffHeapColumnVector 5217 5219 2 78.5 12.7 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write and read with StringType, row length = 20: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 2323 2327 6 176.3 5.7 1.0X
-OnHeapColumnVector 5313 5318 8 77.1 13.0 0.4X
-OffHeapColumnVector 5814 5821 10 70.4 14.2 0.4X
+ConstantColumnVector 1952 1952 0 209.8 4.8 1.0X
+OnHeapColumnVector 5867 5869 2 69.8 14.3 0.3X
+OffHeapColumnVector 5198 5200 2 78.8 12.7 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write and read with StringType, row length = 30: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 2315 2326 15 176.9 5.7 1.0X
-OnHeapColumnVector 5274 5281 10 77.7 12.9 0.4X
-OffHeapColumnVector 5834 5857 33 70.2 14.2 0.4X
+ConstantColumnVector 1968 1969 2 208.1 4.8 1.0X
+OnHeapColumnVector 5847 5861 21 70.1 14.3 0.3X
+OffHeapColumnVector 5202 5206 5 78.7 12.7 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write and read with IntegerType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 708 711 2 578.4 1.7 1.0X
-OnHeapColumnVector 827 829 3 495.2 2.0 0.9X
-OffHeapColumnVector 827 833 7 495.0 2.0 0.9X
+ConstantColumnVector 893 893 1 458.9 2.2 1.0X
+OnHeapColumnVector 1020 1022 2 401.6 2.5 0.9X
+OffHeapColumnVector 893 893 1 458.9 2.2 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write and read with LongType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 2707 2710 4 151.3 6.6 1.0X
-OnHeapColumnVector 2801 2815 20 146.2 6.8 1.0X
-OffHeapColumnVector 2832 2839 11 144.7 6.9 1.0X
+ConstantColumnVector 2871 2872 1 142.7 7.0 1.0X
+OnHeapColumnVector 2994 2995 1 136.8 7.3 1.0X
+OffHeapColumnVector 2999 3013 20 136.6 7.3 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write and read with FloatType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 2220 2231 15 184.5 5.4 1.0X
-OnHeapColumnVector 2972 2995 32 137.8 7.3 0.7X
-OffHeapColumnVector 2971 2974 3 137.8 7.3 0.7X
+ConstantColumnVector 2884 2885 2 142.0 7.0 1.0X
+OnHeapColumnVector 3040 3041 1 134.8 7.4 0.9X
+OffHeapColumnVector 3010 3010 1 136.1 7.3 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write and read with DoubleType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 2836 2843 10 144.4 6.9 1.0X
-OnHeapColumnVector 2973 2995 30 137.8 7.3 1.0X
-OffHeapColumnVector 2948 2953 7 138.9 7.2 1.0X
+ConstantColumnVector 3024 3026 2 135.4 7.4 1.0X
+OnHeapColumnVector 3154 3155 1 129.9 7.7 1.0X
+OffHeapColumnVector 3150 3151 2 130.0 7.7 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test isNull with StringType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 0 0 0 439957035.4 0.0 1.0X
-OnHeapColumnVector 0 0 0 3602747.8 0.0 0.0X
-OffHeapColumnVector 0 0 0 439957035.4 0.0 1.0X
+ConstantColumnVector 0 0 0 405143422.4 0.0 1.0X
+OnHeapColumnVector 0 0 0 3321467.1 0.0 0.0X
+OffHeapColumnVector 0 0 0 405143422.4 0.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test isNull with IntegerType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 0 0 0 439957035.4 0.0 1.0X
-OnHeapColumnVector 0 0 0 3602747.8 0.0 0.0X
-OffHeapColumnVector 0 0 0 439957035.4 0.0 1.0X
+ConstantColumnVector 0 0 0 405143422.4 0.0 1.0X
+OnHeapColumnVector 0 0 0 3321467.1 0.0 0.0X
+OffHeapColumnVector 0 0 0 405143422.4 0.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test isNull with LongType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 0 0 0 439957035.4 0.0 1.0X
-OnHeapColumnVector 0 0 0 3602747.8 0.0 0.0X
-OffHeapColumnVector 0 0 0 439957035.4 0.0 1.0X
+ConstantColumnVector 0 0 0 405143422.4 0.0 1.0X
+OnHeapColumnVector 0 0 0 3321467.1 0.0 0.0X
+OffHeapColumnVector 0 0 0 405143422.4 0.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test isNull with FloatType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 0 0 0 439957035.4 0.0 1.0X
-OnHeapColumnVector 0 0 0 3602747.8 0.0 0.0X
-OffHeapColumnVector 0 0 0 439957035.4 0.0 1.0X
+ConstantColumnVector 0 0 0 405143422.4 0.0 1.0X
+OnHeapColumnVector 0 0 0 3321467.1 0.0 0.0X
+OffHeapColumnVector 0 0 0 405143422.4 0.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test isNull with DoubleType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 0 0 0 439957035.4 0.0 1.0X
-OnHeapColumnVector 0 0 0 3602747.8 0.0 0.0X
-OffHeapColumnVector 0 0 0 439957035.4 0.0 1.0X
+ConstantColumnVector 0 0 0 405143422.4 0.0 1.0X
+OnHeapColumnVector 0 0 0 3321467.1 0.0 0.0X
+OffHeapColumnVector 0 0 0 405143422.4 0.0 1.0X
diff --git a/sql/core/benchmarks/ConstantColumnVectorBenchmark-results.txt b/sql/core/benchmarks/ConstantColumnVectorBenchmark-results.txt
index c7c6f7d6f5fee..9d1a8cbbaaac7 100644
--- a/sql/core/benchmarks/ConstantColumnVectorBenchmark-results.txt
+++ b/sql/core/benchmarks/ConstantColumnVectorBenchmark-results.txt
@@ -1,280 +1,280 @@
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write with StringType, row length = 1: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 1 1 0 325448.5 0.0 1.0X
-OnHeapColumnVector 2823 2826 4 145.1 6.9 0.0X
-OffHeapColumnVector 3497 3499 3 117.1 8.5 0.0X
+ConstantColumnVector 1 1 0 326662.9 0.0 1.0X
+OnHeapColumnVector 2826 2830 5 144.9 6.9 0.0X
+OffHeapColumnVector 3311 3319 12 123.7 8.1 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write with StringType, row length = 5: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 1 1 0 383288.8 0.0 1.0X
-OnHeapColumnVector 3660 3661 2 111.9 8.9 0.0X
-OffHeapColumnVector 4384 4387 4 93.4 10.7 0.0X
+ConstantColumnVector 1 1 0 383970.7 0.0 1.0X
+OnHeapColumnVector 3676 3694 25 111.4 9.0 0.0X
+OffHeapColumnVector 4324 4326 3 94.7 10.6 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write with StringType, row length = 10: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 1 1 0 383417.9 0.0 1.0X
-OnHeapColumnVector 3845 3846 1 106.5 9.4 0.0X
-OffHeapColumnVector 4622 4624 2 88.6 11.3 0.0X
+ConstantColumnVector 1 1 0 383962.4 0.0 1.0X
+OnHeapColumnVector 3742 3742 1 109.5 9.1 0.0X
+OffHeapColumnVector 4315 4320 8 94.9 10.5 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write with StringType, row length = 15: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 1 1 0 383417.6 0.0 1.0X
-OnHeapColumnVector 3682 3685 3 111.2 9.0 0.0X
-OffHeapColumnVector 4408 4424 23 92.9 10.8 0.0X
+ConstantColumnVector 1 1 0 384110.8 0.0 1.0X
+OnHeapColumnVector 3914 3914 0 104.7 9.6 0.0X
+OffHeapColumnVector 4335 4339 6 94.5 10.6 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write with StringType, row length = 20: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 1 1 0 383346.2 0.0 1.0X
-OnHeapColumnVector 4120 4120 1 99.4 10.1 0.0X
-OffHeapColumnVector 4573 4573 0 89.6 11.2 0.0X
+ConstantColumnVector 1 1 0 384114.0 0.0 1.0X
+OnHeapColumnVector 3759 3759 0 109.0 9.2 0.0X
+OffHeapColumnVector 4346 4348 4 94.3 10.6 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write with StringType, row length = 30: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 1 1 0 383533.2 0.0 1.0X
-OnHeapColumnVector 4446 4460 20 92.1 10.9 0.0X
-OffHeapColumnVector 4740 4746 9 86.4 11.6 0.0X
+ConstantColumnVector 1 1 0 384121.2 0.0 1.0X
+OnHeapColumnVector 3820 3828 11 107.2 9.3 0.0X
+OffHeapColumnVector 4250 4252 3 96.4 10.4 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write with IntegerType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 1 1 0 699318.6 0.0 1.0X
-OnHeapColumnVector 16 16 0 25434.3 0.0 0.0X
-OffHeapColumnVector 65 66 0 6261.4 0.2 0.0X
+ConstantColumnVector 1 1 0 699312.6 0.0 1.0X
+OnHeapColumnVector 16 16 0 25439.2 0.0 0.0X
+OffHeapColumnVector 65 65 0 6290.8 0.2 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write with LongType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 1 1 0 664349.4 0.0 1.0X
-OnHeapColumnVector 34 34 0 12170.8 0.1 0.0X
-OffHeapColumnVector 67 67 0 6157.8 0.2 0.0X
+ConstantColumnVector 1 1 0 664346.2 0.0 1.0X
+OnHeapColumnVector 33 34 0 12270.8 0.1 0.0X
+OffHeapColumnVector 66 66 0 6202.4 0.2 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write with FloatType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 1 1 0 553055.4 0.0 1.0X
-OnHeapColumnVector 16 16 0 25045.2 0.0 0.0X
-OffHeapColumnVector 128 128 0 3202.8 0.3 0.0X
+ConstantColumnVector 1 1 0 553051.6 0.0 1.0X
+OnHeapColumnVector 16 16 0 25077.4 0.0 0.0X
+OffHeapColumnVector 128 128 0 3197.1 0.3 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write with DoubleType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 1 1 0 510539.9 0.0 1.0X
-OnHeapColumnVector 35 35 0 11831.2 0.1 0.0X
-OffHeapColumnVector 129 130 0 3171.6 0.3 0.0X
+ConstantColumnVector 1 1 0 510536.0 0.0 1.0X
+OnHeapColumnVector 34 35 0 11916.5 0.1 0.0X
+OffHeapColumnVector 129 129 0 3175.9 0.3 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test read with StringType, row length = 1: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 684 687 3 598.9 1.7 1.0X
-OnHeapColumnVector 1801 1801 1 227.5 4.4 0.4X
-OffHeapColumnVector 4435 4437 3 92.4 10.8 0.2X
+ConstantColumnVector 0 0 0 13274135.5 0.0 1.0X
+OnHeapColumnVector 98 99 0 4159.9 0.2 0.0X
+OffHeapColumnVector 3722 3724 3 110.0 9.1 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test read with StringType, row length = 5: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 939 947 14 436.3 2.3 1.0X
-OnHeapColumnVector 4945 4946 2 82.8 12.1 0.2X
-OffHeapColumnVector 4579 4580 1 89.5 11.2 0.2X
+ConstantColumnVector 0 0 0 13274135.5 0.0 1.0X
+OnHeapColumnVector 4109 4115 9 99.7 10.0 0.0X
+OffHeapColumnVector 3686 3695 12 111.1 9.0 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test read with StringType, row length = 10: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 942 944 2 434.9 2.3 1.0X
-OnHeapColumnVector 5119 5121 3 80.0 12.5 0.2X
-OffHeapColumnVector 4486 4489 3 91.3 11.0 0.2X
+ConstantColumnVector 0 0 0 13278008.3 0.0 1.0X
+OnHeapColumnVector 4114 4115 0 99.6 10.0 0.0X
+OffHeapColumnVector 3700 3709 12 110.7 9.0 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test read with StringType, row length = 15: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 937 939 2 437.0 2.3 1.0X
-OnHeapColumnVector 5118 5119 2 80.0 12.5 0.2X
-OffHeapColumnVector 4492 4495 3 91.2 11.0 0.2X
+ConstantColumnVector 0 0 0 13274135.5 0.0 1.0X
+OnHeapColumnVector 4127 4131 5 99.3 10.1 0.0X
+OffHeapColumnVector 3693 3699 9 110.9 9.0 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test read with StringType, row length = 20: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 941 942 1 435.1 2.3 1.0X
-OnHeapColumnVector 5111 5122 15 80.1 12.5 0.2X
-OffHeapColumnVector 4484 4488 6 91.3 10.9 0.2X
+ConstantColumnVector 0 0 0 13274135.5 0.0 1.0X
+OnHeapColumnVector 4117 4127 14 99.5 10.1 0.0X
+OffHeapColumnVector 3680 3685 7 111.3 9.0 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test read with StringType, row length = 30: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 938 939 1 436.8 2.3 1.0X
-OnHeapColumnVector 5116 5119 5 80.1 12.5 0.2X
-OffHeapColumnVector 4582 4584 4 89.4 11.2 0.2X
+ConstantColumnVector 0 0 0 13278438.7 0.0 1.0X
+OnHeapColumnVector 4125 4136 15 99.3 10.1 0.0X
+OffHeapColumnVector 3691 3697 9 111.0 9.0 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test read with IntegerType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 2595 2597 2 157.8 6.3 1.0X
-OnHeapColumnVector 2720 2720 1 150.6 6.6 1.0X
-OffHeapColumnVector 2600 2602 2 157.5 6.3 1.0X
+ConstantColumnVector 2472 2472 1 165.7 6.0 1.0X
+OnHeapColumnVector 2600 2601 1 157.5 6.3 1.0X
+OffHeapColumnVector 2482 2483 0 165.0 6.1 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test read with LongType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 0 0 0 1476318.0 0.0 1.0X
-OnHeapColumnVector 0 0 0 949059.9 0.0 0.6X
-OffHeapColumnVector 767 768 1 533.9 1.9 0.0X
+ConstantColumnVector 0 0 0 1660841.3 0.0 1.0X
+OnHeapColumnVector 1 1 0 474470.6 0.0 0.3X
+OffHeapColumnVector 767 768 1 534.1 1.9 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test read with FloatType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 0 0 0 1476318.0 0.0 1.0X
-OnHeapColumnVector 0 0 0 1022063.2 0.0 0.7X
-OffHeapColumnVector 767 767 0 534.0 1.9 0.0X
+ConstantColumnVector 0 0 0 1660848.0 0.0 1.0X
+OnHeapColumnVector 0 0 0 1022058.1 0.0 0.6X
+OffHeapColumnVector 767 773 5 533.8 1.9 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test read with DoubleType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 0 0 0 1476318.0 0.0 1.0X
-OnHeapColumnVector 0 1 0 885790.2 0.0 0.6X
-OffHeapColumnVector 696 696 0 588.9 1.7 0.0X
+ConstantColumnVector 0 0 0 1476307.4 0.0 1.0X
+OnHeapColumnVector 1 1 0 459321.6 0.0 0.3X
+OffHeapColumnVector 893 893 0 458.7 2.2 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write and read with StringType, row length = 1: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 0 0 0 1021805.7 0.0 1.0X
-OnHeapColumnVector 3895 3903 12 105.2 9.5 0.0X
-OffHeapColumnVector 3725 3734 13 110.0 9.1 0.0X
+ConstantColumnVector 0 0 0 948833.4 0.0 1.0X
+OnHeapColumnVector 3903 3908 7 104.9 9.5 0.0X
+OffHeapColumnVector 3765 3769 6 108.8 9.2 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write and read with StringType, row length = 5: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 0 0 0 830408.9 0.0 1.0X
-OnHeapColumnVector 3894 3907 17 105.2 9.5 0.0X
-OffHeapColumnVector 3714 3721 10 110.3 9.1 0.0X
+ConstantColumnVector 0 0 0 830400.4 0.0 1.0X
+OnHeapColumnVector 3900 3904 6 105.0 9.5 0.0X
+OffHeapColumnVector 3731 3746 21 109.8 9.1 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write and read with StringType, row length = 10: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 0 0 0 830392.0 0.0 1.0X
-OnHeapColumnVector 3895 3896 2 105.2 9.5 0.0X
-OffHeapColumnVector 3723 3723 0 110.0 9.1 0.0X
+ConstantColumnVector 0 0 0 830403.8 0.0 1.0X
+OnHeapColumnVector 3904 3912 11 104.9 9.5 0.0X
+OffHeapColumnVector 3749 3759 14 109.3 9.2 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write and read with StringType, row length = 15: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 0 0 0 830388.7 0.0 1.0X
-OnHeapColumnVector 3890 3896 8 105.3 9.5 0.0X
-OffHeapColumnVector 3763 3764 1 108.8 9.2 0.0X
+ConstantColumnVector 0 0 0 830403.8 0.0 1.0X
+OnHeapColumnVector 3899 3899 0 105.1 9.5 0.0X
+OffHeapColumnVector 3728 3731 4 109.9 9.1 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write and read with StringType, row length = 20: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------------
ConstantColumnVector 0 0 0 830403.8 0.0 1.0X
-OnHeapColumnVector 3882 3886 6 105.5 9.5 0.0X
-OffHeapColumnVector 3697 3699 3 110.8 9.0 0.0X
+OnHeapColumnVector 3900 3907 10 105.0 9.5 0.0X
+OffHeapColumnVector 3731 3766 49 109.8 9.1 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write and read with StringType, row length = 30: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 0 0 0 830407.2 0.0 1.0X
-OnHeapColumnVector 3887 3891 5 105.4 9.5 0.0X
-OffHeapColumnVector 3700 3701 1 110.7 9.0 0.0X
+ConstantColumnVector 0 1 0 830403.8 0.0 1.0X
+OnHeapColumnVector 3905 3911 9 104.9 9.5 0.0X
+OffHeapColumnVector 3725 3726 1 109.9 9.1 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write and read with IntegerType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 2967 2968 2 138.1 7.2 1.0X
-OnHeapColumnVector 2714 2716 3 150.9 6.6 1.1X
-OffHeapColumnVector 2714 2715 2 150.9 6.6 1.1X
+ConstantColumnVector 2847 2858 16 143.9 7.0 1.0X
+OnHeapColumnVector 2607 2609 2 157.1 6.4 1.1X
+OffHeapColumnVector 2496 2497 1 164.1 6.1 1.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write and read with LongType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 765 765 0 535.7 1.9 1.0X
-OnHeapColumnVector 893 894 2 458.6 2.2 0.9X
-OffHeapColumnVector 893 893 0 458.8 2.2 0.9X
+ConstantColumnVector 764 765 0 535.8 1.9 1.0X
+OnHeapColumnVector 894 894 1 458.4 2.2 0.9X
+OffHeapColumnVector 893 895 2 458.5 2.2 0.9X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write and read with FloatType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 765 765 0 535.5 1.9 1.0X
-OnHeapColumnVector 767 767 1 534.3 1.9 1.0X
-OffHeapColumnVector 766 769 3 535.0 1.9 1.0X
+ConstantColumnVector 892 893 1 459.0 2.2 1.0X
+OnHeapColumnVector 766 767 0 534.5 1.9 1.2X
+OffHeapColumnVector 766 766 1 535.0 1.9 1.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test write and read with DoubleType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 0 0 0 1898027.4 0.0 1.0X
-OnHeapColumnVector 656 657 3 624.5 1.6 0.0X
-OffHeapColumnVector 696 696 0 588.8 1.7 0.0X
+ConstantColumnVector 765 765 0 535.3 1.9 1.0X
+OnHeapColumnVector 893 895 2 458.5 2.2 0.9X
+OffHeapColumnVector 894 895 0 458.2 2.2 0.9X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test isNull with StringType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 0 0 0 409190809.2 0.0 1.0X
+ConstantColumnVector 0 0 0 209728622.6 0.0 1.0X
OnHeapColumnVector 0 0 0 2212069.2 0.0 0.0X
-OffHeapColumnVector 0 0 0 409190809.2 0.0 1.0X
+OffHeapColumnVector 0 0 0 209728622.6 0.0 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test isNull with IntegerType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 0 0 0 409190809.2 0.0 1.0X
-OnHeapColumnVector 0 0 0 2211961.7 0.0 0.0X
-OffHeapColumnVector 0 0 0 409190809.2 0.0 1.0X
+ConstantColumnVector 0 0 0 209728622.6 0.0 1.0X
+OnHeapColumnVector 0 0 0 2212057.2 0.0 0.0X
+OffHeapColumnVector 0 0 0 209728622.6 0.0 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test isNull with LongType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 0 0 0 409190809.2 0.0 1.0X
-OnHeapColumnVector 0 0 0 2211961.7 0.0 0.0X
-OffHeapColumnVector 0 0 0 409190809.2 0.0 1.0X
+ConstantColumnVector 0 0 0 209728622.6 0.0 1.0X
+OnHeapColumnVector 0 0 0 2212069.2 0.0 0.0X
+OffHeapColumnVector 0 0 0 209728622.6 0.0 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test isNull with FloatType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 0 0 0 409190809.2 0.0 1.0X
-OnHeapColumnVector 0 0 0 2212057.2 0.0 0.0X
-OffHeapColumnVector 0 0 0 409190809.2 0.0 1.0X
+ConstantColumnVector 0 0 0 209728622.6 0.0 1.0X
+OnHeapColumnVector 0 0 0 2212069.2 0.0 0.0X
+OffHeapColumnVector 0 0 0 209728622.6 0.0 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Test isNull with DoubleType: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ConstantColumnVector 0 0 0 409190809.2 0.0 1.0X
-OnHeapColumnVector 0 0 0 2211949.7 0.0 0.0X
-OffHeapColumnVector 0 0 0 409190809.2 0.0 1.0X
+ConstantColumnVector 0 0 0 209728622.6 0.0 1.0X
+OnHeapColumnVector 0 0 0 2212057.2 0.0 0.0X
+OffHeapColumnVector 0 0 0 209728622.6 0.0 1.0X
diff --git a/sql/core/benchmarks/DataSourceReadBenchmark-jdk21-results.txt b/sql/core/benchmarks/DataSourceReadBenchmark-jdk21-results.txt
index b716c18deba73..be11d3a0d8b0e 100644
--- a/sql/core/benchmarks/DataSourceReadBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/DataSourceReadBenchmark-jdk21-results.txt
@@ -2,437 +2,437 @@
SQL Single Numeric Column Scan
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SQL Single BOOLEAN Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 9693 9747 77 1.6 616.2 1.0X
-SQL Json 7743 7871 181 2.0 492.3 1.3X
-SQL Json with UnsafeRow 8445 8509 90 1.9 536.9 1.1X
-SQL Parquet Vectorized: DataPageV1 107 118 7 147.3 6.8 90.7X
-SQL Parquet Vectorized: DataPageV2 125 132 4 126.2 7.9 77.8X
-SQL Parquet MR: DataPageV1 1743 1747 5 9.0 110.8 5.6X
-SQL Parquet MR: DataPageV2 1608 1608 0 9.8 102.2 6.0X
-SQL ORC Vectorized 139 152 18 113.5 8.8 70.0X
-SQL ORC MR 1452 1466 20 10.8 92.3 6.7X
-
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+SQL CSV 10016 10139 174 1.6 636.8 1.0X
+SQL Json 8116 8156 56 1.9 516.0 1.2X
+SQL Json with UnsafeRow 8940 8963 33 1.8 568.4 1.1X
+SQL Parquet Vectorized: DataPageV1 90 102 12 175.7 5.7 111.9X
+SQL Parquet Vectorized: DataPageV2 93 102 6 168.9 5.9 107.6X
+SQL Parquet MR: DataPageV1 1798 1838 56 8.7 114.3 5.6X
+SQL Parquet MR: DataPageV2 1674 1679 7 9.4 106.4 6.0X
+SQL ORC Vectorized 143 159 24 110.2 9.1 70.2X
+SQL ORC MR 1552 1562 15 10.1 98.6 6.5X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Parquet Reader Single BOOLEAN Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-ParquetReader Vectorized: DataPageV1 94 95 2 167.7 6.0 1.0X
-ParquetReader Vectorized: DataPageV2 124 126 1 126.5 7.9 0.8X
-ParquetReader Vectorized -> Row: DataPageV1 75 76 2 211.1 4.7 1.3X
-ParquetReader Vectorized -> Row: DataPageV2 107 108 1 147.5 6.8 0.9X
+ParquetReader Vectorized: DataPageV1 94 96 2 166.9 6.0 1.0X
+ParquetReader Vectorized: DataPageV2 106 109 2 148.6 6.7 0.9X
+ParquetReader Vectorized -> Row: DataPageV1 76 78 2 207.7 4.8 1.2X
+ParquetReader Vectorized -> Row: DataPageV2 89 91 2 177.6 5.6 1.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SQL Single TINYINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 9054 9113 83 1.7 575.7 1.0X
-SQL Json 8690 8707 24 1.8 552.5 1.0X
-SQL Json with UnsafeRow 9308 9317 13 1.7 591.8 1.0X
-SQL Parquet Vectorized: DataPageV1 99 110 8 159.5 6.3 91.8X
-SQL Parquet Vectorized: DataPageV2 97 111 17 162.1 6.2 93.3X
-SQL Parquet MR: DataPageV1 1818 1820 3 8.7 115.6 5.0X
-SQL Parquet MR: DataPageV2 1864 1866 3 8.4 118.5 4.9X
-SQL ORC Vectorized 110 134 19 142.6 7.0 82.1X
-SQL ORC MR 1592 1600 10 9.9 101.2 5.7X
-
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+SQL CSV 9341 9372 44 1.7 593.9 1.0X
+SQL Json 8686 8757 100 1.8 552.2 1.1X
+SQL Json with UnsafeRow 9507 9517 13 1.7 604.5 1.0X
+SQL Parquet Vectorized: DataPageV1 150 158 6 104.5 9.6 62.1X
+SQL Parquet Vectorized: DataPageV2 149 156 7 105.5 9.5 62.7X
+SQL Parquet MR: DataPageV1 1959 1967 12 8.0 124.6 4.8X
+SQL Parquet MR: DataPageV2 1856 1880 34 8.5 118.0 5.0X
+SQL ORC Vectorized 141 150 12 111.5 9.0 66.2X
+SQL ORC MR 1669 1670 2 9.4 106.1 5.6X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Parquet Reader Single TINYINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-ParquetReader Vectorized: DataPageV1 78 81 6 202.9 4.9 1.0X
-ParquetReader Vectorized: DataPageV2 79 81 1 199.7 5.0 1.0X
-ParquetReader Vectorized -> Row: DataPageV1 46 49 4 339.0 2.9 1.7X
-ParquetReader Vectorized -> Row: DataPageV2 46 50 4 339.3 2.9 1.7X
+ParquetReader Vectorized: DataPageV1 125 129 6 125.9 7.9 1.0X
+ParquetReader Vectorized: DataPageV2 124 128 3 126.8 7.9 1.0X
+ParquetReader Vectorized -> Row: DataPageV1 101 117 33 155.9 6.4 1.2X
+ParquetReader Vectorized -> Row: DataPageV2 101 103 2 156.1 6.4 1.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SQL Single SMALLINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 9898 9911 19 1.6 629.3 1.0X
-SQL Json 9233 9238 7 1.7 587.0 1.1X
-SQL Json with UnsafeRow 9788 9804 24 1.6 622.3 1.0X
-SQL Parquet Vectorized: DataPageV1 121 135 14 130.2 7.7 82.0X
-SQL Parquet Vectorized: DataPageV2 138 152 16 114.1 8.8 71.8X
-SQL Parquet MR: DataPageV1 2029 2035 7 7.8 129.0 4.9X
-SQL Parquet MR: DataPageV2 1949 1964 22 8.1 123.9 5.1X
-SQL ORC Vectorized 140 147 11 112.5 8.9 70.8X
-SQL ORC MR 1607 1619 16 9.8 102.2 6.2X
-
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+SQL CSV 10510 10522 17 1.5 668.2 1.0X
+SQL Json 9264 9265 2 1.7 589.0 1.1X
+SQL Json with UnsafeRow 9945 9947 3 1.6 632.3 1.1X
+SQL Parquet Vectorized: DataPageV1 123 140 13 127.5 7.8 85.2X
+SQL Parquet Vectorized: DataPageV2 142 171 24 110.6 9.0 73.9X
+SQL Parquet MR: DataPageV1 2103 2104 1 7.5 133.7 5.0X
+SQL Parquet MR: DataPageV2 2072 2075 4 7.6 131.7 5.1X
+SQL ORC Vectorized 147 162 19 107.2 9.3 71.6X
+SQL ORC MR 1769 1770 1 8.9 112.4 5.9X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Parquet Reader Single SMALLINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-ParquetReader Vectorized: DataPageV1 149 164 11 105.3 9.5 1.0X
-ParquetReader Vectorized: DataPageV2 168 176 6 93.8 10.7 0.9X
-ParquetReader Vectorized -> Row: DataPageV1 145 153 10 108.3 9.2 1.0X
-ParquetReader Vectorized -> Row: DataPageV2 165 176 8 95.3 10.5 0.9X
+ParquetReader Vectorized: DataPageV1 150 159 7 105.0 9.5 1.0X
+ParquetReader Vectorized: DataPageV2 196 203 5 80.2 12.5 0.8X
+ParquetReader Vectorized -> Row: DataPageV1 146 149 2 107.9 9.3 1.0X
+ParquetReader Vectorized -> Row: DataPageV2 175 183 7 90.1 11.1 0.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SQL Single INT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 11119 11129 15 1.4 706.9 1.0X
-SQL Json 9483 9484 2 1.7 602.9 1.2X
-SQL Json with UnsafeRow 10139 10156 24 1.6 644.6 1.1X
-SQL Parquet Vectorized: DataPageV1 103 118 15 152.0 6.6 107.5X
-SQL Parquet Vectorized: DataPageV2 181 197 15 86.8 11.5 61.4X
-SQL Parquet MR: DataPageV1 1907 1930 32 8.2 121.3 5.8X
-SQL Parquet MR: DataPageV2 1942 1983 57 8.1 123.5 5.7X
-SQL ORC Vectorized 180 191 12 87.6 11.4 61.9X
-SQL ORC MR 1642 1642 0 9.6 104.4 6.8X
-
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+SQL CSV 11428 11428 1 1.4 726.6 1.0X
+SQL Json 9760 9761 2 1.6 620.5 1.2X
+SQL Json with UnsafeRow 10321 10322 0 1.5 656.2 1.1X
+SQL Parquet Vectorized: DataPageV1 104 126 20 150.7 6.6 109.5X
+SQL Parquet Vectorized: DataPageV2 189 209 21 83.2 12.0 60.4X
+SQL Parquet MR: DataPageV1 2071 2086 21 7.6 131.7 5.5X
+SQL Parquet MR: DataPageV2 2040 2061 30 7.7 129.7 5.6X
+SQL ORC Vectorized 188 205 17 83.7 12.0 60.8X
+SQL ORC MR 1807 1818 15 8.7 114.9 6.3X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Parquet Reader Single INT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-ParquetReader Vectorized: DataPageV1 131 134 7 119.9 8.3 1.0X
-ParquetReader Vectorized: DataPageV2 210 215 7 74.9 13.4 0.6X
-ParquetReader Vectorized -> Row: DataPageV1 135 135 1 116.9 8.6 1.0X
-ParquetReader Vectorized -> Row: DataPageV2 207 219 10 76.0 13.2 0.6X
+ParquetReader Vectorized: DataPageV1 146 159 9 107.6 9.3 1.0X
+ParquetReader Vectorized: DataPageV2 228 242 9 69.0 14.5 0.6X
+ParquetReader Vectorized -> Row: DataPageV1 135 148 12 116.6 8.6 1.1X
+ParquetReader Vectorized -> Row: DataPageV2 220 239 12 71.5 14.0 0.7X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SQL Single BIGINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 11093 11109 23 1.4 705.3 1.0X
-SQL Json 9583 9600 24 1.6 609.3 1.2X
-SQL Json with UnsafeRow 10139 10159 28 1.6 644.6 1.1X
-SQL Parquet Vectorized: DataPageV1 285 300 14 55.2 18.1 38.9X
-SQL Parquet Vectorized: DataPageV2 232 264 21 67.9 14.7 47.9X
-SQL Parquet MR: DataPageV1 2439 2442 4 6.4 155.1 4.5X
-SQL Parquet MR: DataPageV2 2014 2020 8 7.8 128.0 5.5X
-SQL ORC Vectorized 165 173 9 95.2 10.5 67.1X
-SQL ORC MR 1682 1690 12 9.4 106.9 6.6X
-
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+SQL CSV 11431 11447 23 1.4 726.8 1.0X
+SQL Json 9801 9807 9 1.6 623.1 1.2X
+SQL Json with UnsafeRow 10539 10543 6 1.5 670.0 1.1X
+SQL Parquet Vectorized: DataPageV1 312 337 19 50.4 19.8 36.7X
+SQL Parquet Vectorized: DataPageV2 243 277 20 64.7 15.5 47.0X
+SQL Parquet MR: DataPageV1 2547 2571 34 6.2 161.9 4.5X
+SQL Parquet MR: DataPageV2 2120 2122 3 7.4 134.8 5.4X
+SQL ORC Vectorized 148 161 16 106.1 9.4 77.1X
+SQL ORC MR 1789 1808 28 8.8 113.7 6.4X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Parquet Reader Single BIGINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-ParquetReader Vectorized: DataPageV1 352 357 5 44.7 22.4 1.0X
-ParquetReader Vectorized: DataPageV2 309 315 8 50.9 19.7 1.1X
-ParquetReader Vectorized -> Row: DataPageV1 330 335 5 47.6 21.0 1.1X
-ParquetReader Vectorized -> Row: DataPageV2 264 273 8 59.6 16.8 1.3X
+ParquetReader Vectorized: DataPageV1 333 338 3 47.2 21.2 1.0X
+ParquetReader Vectorized: DataPageV2 278 288 10 56.6 17.7 1.2X
+ParquetReader Vectorized -> Row: DataPageV1 324 330 6 48.5 20.6 1.0X
+ParquetReader Vectorized -> Row: DataPageV2 275 286 8 57.3 17.5 1.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SQL Single FLOAT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 11874 11897 32 1.3 754.9 1.0X
-SQL Json 11112 11136 34 1.4 706.5 1.1X
-SQL Json with UnsafeRow 12000 12000 1 1.3 762.9 1.0X
-SQL Parquet Vectorized: DataPageV1 85 104 16 184.8 5.4 139.5X
-SQL Parquet Vectorized: DataPageV2 84 99 15 186.3 5.4 140.6X
-SQL Parquet MR: DataPageV1 1962 1964 3 8.0 124.7 6.1X
-SQL Parquet MR: DataPageV2 1899 1993 132 8.3 120.8 6.3X
-SQL ORC Vectorized 232 241 16 67.7 14.8 51.1X
-SQL ORC MR 1718 1729 17 9.2 109.2 6.9X
-
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+SQL CSV 11476 11505 41 1.4 729.6 1.0X
+SQL Json 11463 11529 93 1.4 728.8 1.0X
+SQL Json with UnsafeRow 11909 11918 13 1.3 757.1 1.0X
+SQL Parquet Vectorized: DataPageV1 92 110 15 170.2 5.9 124.2X
+SQL Parquet Vectorized: DataPageV2 93 106 14 169.7 5.9 123.8X
+SQL Parquet MR: DataPageV1 2111 2120 13 7.5 134.2 5.4X
+SQL Parquet MR: DataPageV2 2023 2032 12 7.8 128.6 5.7X
+SQL ORC Vectorized 227 239 15 69.4 14.4 50.6X
+SQL ORC MR 1846 1864 26 8.5 117.4 6.2X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Parquet Reader Single FLOAT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-ParquetReader Vectorized: DataPageV1 133 139 6 118.4 8.4 1.0X
-ParquetReader Vectorized: DataPageV2 131 138 7 119.8 8.3 1.0X
-ParquetReader Vectorized -> Row: DataPageV1 128 133 6 122.7 8.1 1.0X
-ParquetReader Vectorized -> Row: DataPageV2 129 133 4 122.1 8.2 1.0X
+ParquetReader Vectorized: DataPageV1 145 154 6 108.5 9.2 1.0X
+ParquetReader Vectorized: DataPageV2 143 151 8 110.0 9.1 1.0X
+ParquetReader Vectorized -> Row: DataPageV1 136 141 6 115.9 8.6 1.1X
+ParquetReader Vectorized -> Row: DataPageV2 134 142 7 117.3 8.5 1.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SQL Single DOUBLE Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 11741 11743 3 1.3 746.5 1.0X
-SQL Json 11539 11542 4 1.4 733.7 1.0X
-SQL Json with UnsafeRow 12260 12274 20 1.3 779.5 1.0X
-SQL Parquet Vectorized: DataPageV1 308 315 7 51.1 19.6 38.1X
-SQL Parquet Vectorized: DataPageV2 308 316 8 51.0 19.6 38.1X
-SQL Parquet MR: DataPageV1 2475 2493 25 6.4 157.4 4.7X
-SQL Parquet MR: DataPageV2 2406 2414 11 6.5 153.0 4.9X
-SQL ORC Vectorized 584 599 16 27.0 37.1 20.1X
-SQL ORC MR 2152 2167 21 7.3 136.8 5.5X
-
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+SQL CSV 11669 11686 25 1.3 741.9 1.0X
+SQL Json 11629 11633 6 1.4 739.3 1.0X
+SQL Json with UnsafeRow 12510 12513 5 1.3 795.3 0.9X
+SQL Parquet Vectorized: DataPageV1 325 332 9 48.3 20.7 35.9X
+SQL Parquet Vectorized: DataPageV2 275 307 22 57.1 17.5 42.4X
+SQL Parquet MR: DataPageV1 2500 2508 11 6.3 158.9 4.7X
+SQL Parquet MR: DataPageV2 2442 2454 16 6.4 155.3 4.8X
+SQL ORC Vectorized 599 613 26 26.3 38.1 19.5X
+SQL ORC MR 2361 2364 5 6.7 150.1 4.9X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Parquet Reader Single DOUBLE Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-ParquetReader Vectorized: DataPageV1 314 319 4 50.1 20.0 1.0X
-ParquetReader Vectorized: DataPageV2 325 330 6 48.4 20.7 1.0X
-ParquetReader Vectorized -> Row: DataPageV1 324 331 9 48.5 20.6 1.0X
-ParquetReader Vectorized -> Row: DataPageV2 326 330 4 48.3 20.7 1.0X
+ParquetReader Vectorized: DataPageV1 342 354 8 46.0 21.7 1.0X
+ParquetReader Vectorized: DataPageV2 343 349 5 45.8 21.8 1.0X
+ParquetReader Vectorized -> Row: DataPageV1 338 345 5 46.6 21.5 1.0X
+ParquetReader Vectorized -> Row: DataPageV2 335 341 5 46.9 21.3 1.0X
================================================================================================
SQL Single Numeric Column Scan in Struct
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SQL Single TINYINT Column Scan in Struct: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------
-SQL ORC MR 2079 2176 137 7.6 132.2 1.0X
-SQL ORC Vectorized (Nested Column Disabled) 2147 2151 6 7.3 136.5 1.0X
-SQL ORC Vectorized (Nested Column Enabled) 144 153 20 109.1 9.2 14.4X
-SQL Parquet MR: DataPageV1 2361 2371 14 6.7 150.1 0.9X
-SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 2672 2690 26 5.9 169.9 0.8X
-SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 106 115 12 148.8 6.7 19.7X
-SQL Parquet MR: DataPageV2 2298 2311 18 6.8 146.1 0.9X
-SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2604 2608 6 6.0 165.6 0.8X
-SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 106 119 17 148.9 6.7 19.7X
-
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+SQL ORC MR 2301 2403 145 6.8 146.3 1.0X
+SQL ORC Vectorized (Nested Column Disabled) 2357 2394 51 6.7 149.9 1.0X
+SQL ORC Vectorized (Nested Column Enabled) 148 157 13 106.3 9.4 15.5X
+SQL Parquet MR: DataPageV1 2363 2396 47 6.7 150.2 1.0X
+SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 2914 2915 2 5.4 185.3 0.8X
+SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 157 182 25 100.1 10.0 14.6X
+SQL Parquet MR: DataPageV2 2343 2347 7 6.7 148.9 1.0X
+SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2892 2902 14 5.4 183.9 0.8X
+SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 165 179 18 95.2 10.5 13.9X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SQL Single SMALLINT Column Scan in Struct: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------
-SQL ORC MR 2110 2113 5 7.5 134.2 1.0X
-SQL ORC Vectorized (Nested Column Disabled) 2097 2119 32 7.5 133.3 1.0X
-SQL ORC Vectorized (Nested Column Enabled) 280 287 13 56.2 17.8 7.5X
-SQL Parquet MR: DataPageV1 2483 2487 5 6.3 157.9 0.8X
-SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 2984 2985 1 5.3 189.7 0.7X
-SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 111 143 26 141.6 7.1 19.0X
-SQL Parquet MR: DataPageV2 2440 2442 4 6.4 155.1 0.9X
-SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2873 2896 32 5.5 182.7 0.7X
-SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 206 232 26 76.5 13.1 10.3X
-
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+SQL ORC MR 2471 2484 17 6.4 157.1 1.0X
+SQL ORC Vectorized (Nested Column Disabled) 2471 2492 29 6.4 157.1 1.0X
+SQL ORC Vectorized (Nested Column Enabled) 273 286 17 57.6 17.4 9.1X
+SQL Parquet MR: DataPageV1 2486 2507 29 6.3 158.1 1.0X
+SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 2970 2974 6 5.3 188.8 0.8X
+SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 119 139 17 132.4 7.6 20.8X
+SQL Parquet MR: DataPageV2 2499 2526 38 6.3 158.9 1.0X
+SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2896 2910 21 5.4 184.1 0.9X
+SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 155 194 21 101.2 9.9 15.9X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SQL Single INT Column Scan in Struct: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------
-SQL ORC MR 2198 2266 97 7.2 139.7 1.0X
-SQL ORC Vectorized (Nested Column Disabled) 2204 2227 33 7.1 140.1 1.0X
-SQL ORC Vectorized (Nested Column Enabled) 299 312 19 52.6 19.0 7.4X
-SQL Parquet MR: DataPageV1 2426 2442 23 6.5 154.2 0.9X
-SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 3032 3036 5 5.2 192.8 0.7X
-SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 106 130 21 148.4 6.7 20.7X
-SQL Parquet MR: DataPageV2 2427 2441 20 6.5 154.3 0.9X
-SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2892 2906 20 5.4 183.9 0.8X
-SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 289 305 17 54.5 18.3 7.6X
-
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+SQL ORC MR 2430 2496 92 6.5 154.5 1.0X
+SQL ORC Vectorized (Nested Column Disabled) 2576 2588 17 6.1 163.8 0.9X
+SQL ORC Vectorized (Nested Column Enabled) 275 291 22 57.1 17.5 8.8X
+SQL Parquet MR: DataPageV1 2578 2587 13 6.1 163.9 0.9X
+SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 3173 3204 45 5.0 201.7 0.8X
+SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 132 140 4 118.9 8.4 18.4X
+SQL Parquet MR: DataPageV2 2580 2608 40 6.1 164.0 0.9X
+SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 3159 3179 28 5.0 200.8 0.8X
+SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 231 247 18 68.2 14.7 10.5X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SQL Single BIGINT Column Scan in Struct: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------
-SQL ORC MR 2180 2185 7 7.2 138.6 1.0X
-SQL ORC Vectorized (Nested Column Disabled) 2182 2185 4 7.2 138.7 1.0X
-SQL ORC Vectorized (Nested Column Enabled) 302 311 18 52.1 19.2 7.2X
-SQL Parquet MR: DataPageV1 2798 2824 36 5.6 177.9 0.8X
-SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 3348 3353 7 4.7 212.9 0.7X
-SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 322 330 8 48.8 20.5 6.8X
-SQL Parquet MR: DataPageV2 2371 2385 20 6.6 150.7 0.9X
-SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2861 2873 17 5.5 181.9 0.8X
-SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 254 297 23 61.9 16.2 8.6X
-
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+SQL ORC MR 2695 2705 13 5.8 171.4 1.0X
+SQL ORC Vectorized (Nested Column Disabled) 2702 2716 20 5.8 171.8 1.0X
+SQL ORC Vectorized (Nested Column Enabled) 277 300 34 56.7 17.6 9.7X
+SQL Parquet MR: DataPageV1 3101 3103 2 5.1 197.1 0.9X
+SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 3865 3870 6 4.1 245.7 0.7X
+SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 346 359 21 45.4 22.0 7.8X
+SQL Parquet MR: DataPageV2 2590 2608 25 6.1 164.7 1.0X
+SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 3178 3185 10 4.9 202.0 0.8X
+SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 279 314 16 56.3 17.8 9.7X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SQL Single FLOAT Column Scan in Struct: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------
-SQL ORC MR 2367 2453 123 6.6 150.5 1.0X
-SQL ORC Vectorized (Nested Column Disabled) 2321 2339 26 6.8 147.6 1.0X
-SQL ORC Vectorized (Nested Column Enabled) 357 368 22 44.0 22.7 6.6X
-SQL Parquet MR: DataPageV1 2306 2327 30 6.8 146.6 1.0X
-SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 2783 2784 1 5.7 177.0 0.9X
-SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 93 103 6 169.4 5.9 25.5X
-SQL Parquet MR: DataPageV2 2306 2312 8 6.8 146.6 1.0X
-SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2710 2724 19 5.8 172.3 0.9X
-SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 90 116 23 174.0 5.7 26.2X
-
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+SQL ORC MR 2490 2582 130 6.3 158.3 1.0X
+SQL ORC Vectorized (Nested Column Disabled) 2698 2699 1 5.8 171.5 0.9X
+SQL ORC Vectorized (Nested Column Enabled) 348 354 3 45.2 22.1 7.2X
+SQL Parquet MR: DataPageV1 2637 2653 22 6.0 167.7 0.9X
+SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 3035 3051 22 5.2 193.0 0.8X
+SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 117 154 17 134.8 7.4 21.3X
+SQL Parquet MR: DataPageV2 2560 2579 26 6.1 162.8 1.0X
+SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2924 2926 3 5.4 185.9 0.9X
+SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 99 118 15 158.5 6.3 25.1X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SQL Single DOUBLE Column Scan in Struct: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------
-SQL ORC MR 2547 2564 24 6.2 162.0 1.0X
-SQL ORC Vectorized (Nested Column Disabled) 2591 2602 15 6.1 164.7 1.0X
-SQL ORC Vectorized (Nested Column Enabled) 721 722 1 21.8 45.8 3.5X
-SQL Parquet MR: DataPageV1 2777 2783 8 5.7 176.6 0.9X
-SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 3278 3282 6 4.8 208.4 0.8X
-SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 310 324 11 50.7 19.7 8.2X
-SQL Parquet MR: DataPageV2 2742 2758 22 5.7 174.3 0.9X
-SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 3149 3165 22 5.0 200.2 0.8X
-SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 307 317 8 51.3 19.5 8.3X
+SQL ORC MR 2811 2839 41 5.6 178.7 1.0X
+SQL ORC Vectorized (Nested Column Disabled) 2866 2870 5 5.5 182.2 1.0X
+SQL ORC Vectorized (Nested Column Enabled) 722 726 5 21.8 45.9 3.9X
+SQL Parquet MR: DataPageV1 2998 2998 1 5.2 190.6 0.9X
+SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 3518 3530 16 4.5 223.7 0.8X
+SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 336 341 7 46.9 21.3 8.4X
+SQL Parquet MR: DataPageV2 2939 2954 22 5.4 186.8 1.0X
+SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 3408 3419 15 4.6 216.7 0.8X
+SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 326 341 10 48.3 20.7 8.6X
================================================================================================
SQL Nested Column Scan
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SQL Nested Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------
-SQL ORC MR 13328 13415 73 0.1 12711.0 1.0X
-SQL ORC Vectorized (Nested Column Disabled) 13285 13333 28 0.1 12670.0 1.0X
-SQL ORC Vectorized (Nested Column Enabled) 7170 7196 14 0.1 6837.5 1.9X
-SQL Parquet MR: DataPageV1 9251 9339 83 0.1 8822.7 1.4X
-SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 9546 9592 39 0.1 9103.6 1.4X
-SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 5892 5932 24 0.2 5618.8 2.3X
-SQL Parquet MR: DataPageV2 9848 10065 239 0.1 9392.2 1.4X
-SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 10250 10501 199 0.1 9775.5 1.3X
-SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 5653 5720 31 0.2 5390.8 2.4X
+SQL ORC MR 14501 14817 190 0.1 13828.8 1.0X
+SQL ORC Vectorized (Nested Column Disabled) 14626 15009 205 0.1 13948.9 1.0X
+SQL ORC Vectorized (Nested Column Enabled) 7701 7776 33 0.1 7344.1 1.9X
+SQL Parquet MR: DataPageV1 9945 10009 58 0.1 9484.3 1.5X
+SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 10227 10319 57 0.1 9753.7 1.4X
+SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 6799 6970 93 0.2 6483.6 2.1X
+SQL Parquet MR: DataPageV2 10726 11061 289 0.1 10229.5 1.4X
+SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 10919 11197 260 0.1 10413.5 1.3X
+SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 6398 6474 57 0.2 6101.6 2.3X
================================================================================================
Int and String Scan
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Int and String Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 10418 10449 44 1.0 993.5 1.0X
-SQL Json 10216 10220 5 1.0 974.3 1.0X
-SQL Parquet Vectorized: DataPageV1 1761 1786 35 6.0 168.0 5.9X
-SQL Parquet Vectorized: DataPageV2 1978 1994 22 5.3 188.7 5.3X
-SQL Parquet MR: DataPageV1 4085 4092 9 2.6 389.6 2.6X
-SQL Parquet MR: DataPageV2 4058 4074 22 2.6 387.0 2.6X
-SQL ORC Vectorized 1905 1927 31 5.5 181.6 5.5X
-SQL ORC MR 3633 3644 15 2.9 346.5 2.9X
+SQL CSV 10362 10412 70 1.0 988.2 1.0X
+SQL Json 10177 10208 45 1.0 970.5 1.0X
+SQL Parquet Vectorized: DataPageV1 1821 1832 15 5.8 173.7 5.7X
+SQL Parquet Vectorized: DataPageV2 2053 2057 5 5.1 195.8 5.0X
+SQL Parquet MR: DataPageV1 4043 4056 18 2.6 385.5 2.6X
+SQL Parquet MR: DataPageV2 3960 3966 8 2.6 377.7 2.6X
+SQL ORC Vectorized 1785 1791 8 5.9 170.2 5.8X
+SQL ORC MR 3663 3687 33 2.9 349.3 2.8X
================================================================================================
Repeated String Scan
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Repeated String: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 5454 5472 25 1.9 520.1 1.0X
-SQL Json 6444 6445 2 1.6 614.6 0.8X
-SQL Parquet Vectorized: DataPageV1 529 538 9 19.8 50.5 10.3X
-SQL Parquet Vectorized: DataPageV2 543 556 13 19.3 51.8 10.0X
-SQL Parquet MR: DataPageV1 1776 1792 23 5.9 169.4 3.1X
-SQL Parquet MR: DataPageV2 1662 1671 13 6.3 158.5 3.3X
-SQL ORC Vectorized 367 371 3 28.6 35.0 14.9X
-SQL ORC MR 1795 1816 29 5.8 171.2 3.0X
+SQL CSV 5839 5892 75 1.8 556.9 1.0X
+SQL Json 6442 6448 8 1.6 614.3 0.9X
+SQL Parquet Vectorized: DataPageV1 490 496 5 21.4 46.7 11.9X
+SQL Parquet Vectorized: DataPageV2 466 471 4 22.5 44.5 12.5X
+SQL Parquet MR: DataPageV1 1670 1688 26 6.3 159.2 3.5X
+SQL Parquet MR: DataPageV2 1601 1631 42 6.5 152.7 3.6X
+SQL ORC Vectorized 390 396 6 26.9 37.2 15.0X
+SQL ORC MR 1846 1867 29 5.7 176.1 3.2X
================================================================================================
Partitioned Table Scan
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Partitioned Table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------
-Data column - CSV 10869 10873 6 1.4 691.0 1.0X
-Data column - Json 9344 9391 67 1.7 594.1 1.2X
-Data column - Parquet Vectorized: DataPageV1 112 126 10 140.1 7.1 96.8X
-Data column - Parquet Vectorized: DataPageV2 218 241 19 72.0 13.9 49.7X
-Data column - Parquet MR: DataPageV1 2361 2395 48 6.7 150.1 4.6X
-Data column - Parquet MR: DataPageV2 2300 2301 2 6.8 146.2 4.7X
-Data column - ORC Vectorized 154 165 13 101.8 9.8 70.4X
-Data column - ORC MR 1990 2000 15 7.9 126.5 5.5X
-Partition column - CSV 3690 3704 20 4.3 234.6 2.9X
-Partition column - Json 8499 8531 46 1.9 540.3 1.3X
-Partition column - Parquet Vectorized: DataPageV1 30 41 10 521.6 1.9 360.5X
-Partition column - Parquet Vectorized: DataPageV2 29 34 6 548.7 1.8 379.2X
-Partition column - Parquet MR: DataPageV1 1245 1260 22 12.6 79.1 8.7X
-Partition column - Parquet MR: DataPageV2 1250 1261 15 12.6 79.5 8.7X
-Partition column - ORC Vectorized 30 34 6 520.5 1.9 359.6X
-Partition column - ORC MR 1220 1221 1 12.9 77.6 8.9X
-Both columns - CSV 10588 10682 134 1.5 673.2 1.0X
-Both columns - Json 9542 9545 4 1.6 606.7 1.1X
-Both columns - Parquet Vectorized: DataPageV1 117 141 20 134.0 7.5 92.6X
-Both columns - Parquet Vectorized: DataPageV2 241 262 20 65.2 15.3 45.0X
-Both columns - Parquet MR: DataPageV1 2266 2270 6 6.9 144.1 4.8X
-Both columns - Parquet MR: DataPageV2 2202 2237 50 7.1 140.0 4.9X
-Both columns - ORC Vectorized 158 200 32 99.4 10.1 68.7X
-Both columns - ORC MR 1941 1950 13 8.1 123.4 5.6X
+Data column - CSV 11085 11095 14 1.4 704.8 1.0X
+Data column - Json 9608 9800 272 1.6 610.8 1.2X
+Data column - Parquet Vectorized: DataPageV1 108 130 20 145.0 6.9 102.2X
+Data column - Parquet Vectorized: DataPageV2 234 254 19 67.1 14.9 47.3X
+Data column - Parquet MR: DataPageV1 2352 2355 4 6.7 149.5 4.7X
+Data column - Parquet MR: DataPageV2 2308 2384 107 6.8 146.7 4.8X
+Data column - ORC Vectorized 175 187 16 90.0 11.1 63.4X
+Data column - ORC MR 2001 2009 12 7.9 127.2 5.5X
+Partition column - CSV 3583 3630 67 4.4 227.8 3.1X
+Partition column - Json 8739 8762 32 1.8 555.6 1.3X
+Partition column - Parquet Vectorized: DataPageV1 35 44 11 448.9 2.2 316.4X
+Partition column - Parquet Vectorized: DataPageV2 34 43 7 467.8 2.1 329.7X
+Partition column - Parquet MR: DataPageV1 1249 1258 13 12.6 79.4 8.9X
+Partition column - Parquet MR: DataPageV2 1237 1248 15 12.7 78.7 9.0X
+Partition column - ORC Vectorized 34 40 7 465.0 2.2 327.7X
+Partition column - ORC MR 1290 1291 1 12.2 82.0 8.6X
+Both columns - CSV 11214 11244 43 1.4 712.9 1.0X
+Both columns - Json 9546 9567 30 1.6 606.9 1.2X
+Both columns - Parquet Vectorized: DataPageV1 130 161 20 120.7 8.3 85.0X
+Both columns - Parquet Vectorized: DataPageV2 248 273 26 63.3 15.8 44.6X
+Both columns - Parquet MR: DataPageV1 2305 2321 22 6.8 146.6 4.8X
+Both columns - Parquet MR: DataPageV2 2274 2288 21 6.9 144.6 4.9X
+Both columns - ORC Vectorized 171 203 29 91.9 10.9 64.7X
+Both columns - ORC MR 2067 2083 23 7.6 131.4 5.4X
================================================================================================
String with Nulls Scan
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
String with Nulls Scan (0.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 7234 7254 29 1.4 689.9 1.0X
-SQL Json 8754 8758 5 1.2 834.9 0.8X
-SQL Parquet Vectorized: DataPageV1 1139 1151 17 9.2 108.6 6.4X
-SQL Parquet Vectorized: DataPageV2 1469 1482 18 7.1 140.1 4.9X
-SQL Parquet MR: DataPageV1 3437 3441 5 3.1 327.8 2.1X
-SQL Parquet MR: DataPageV2 3643 3666 33 2.9 347.4 2.0X
-ParquetReader Vectorized: DataPageV1 759 765 9 13.8 72.4 9.5X
-ParquetReader Vectorized: DataPageV2 1099 1105 8 9.5 104.8 6.6X
-SQL ORC Vectorized 820 834 22 12.8 78.2 8.8X
-SQL ORC MR 2782 2792 14 3.8 265.3 2.6X
-
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+SQL CSV 7331 7353 30 1.4 699.2 1.0X
+SQL Json 9139 9253 160 1.1 871.6 0.8X
+SQL Parquet Vectorized: DataPageV1 1184 1208 35 8.9 112.9 6.2X
+SQL Parquet Vectorized: DataPageV2 1517 1526 12 6.9 144.7 4.8X
+SQL Parquet MR: DataPageV1 3688 3709 29 2.8 351.7 2.0X
+SQL Parquet MR: DataPageV2 3820 3831 16 2.7 364.3 1.9X
+ParquetReader Vectorized: DataPageV1 785 800 13 13.4 74.8 9.3X
+ParquetReader Vectorized: DataPageV2 1096 1114 25 9.6 104.5 6.7X
+SQL ORC Vectorized 785 850 57 13.4 74.9 9.3X
+SQL ORC MR 3070 3081 15 3.4 292.8 2.4X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
String with Nulls Scan (50.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 5381 5407 37 1.9 513.2 1.0X
-SQL Json 7194 7264 99 1.5 686.1 0.7X
-SQL Parquet Vectorized: DataPageV1 742 753 16 14.1 70.7 7.3X
-SQL Parquet Vectorized: DataPageV2 976 978 3 10.7 93.1 5.5X
-SQL Parquet MR: DataPageV1 2764 2777 17 3.8 263.6 1.9X
-SQL Parquet MR: DataPageV2 2846 2853 10 3.7 271.4 1.9X
-ParquetReader Vectorized: DataPageV1 721 732 12 14.6 68.7 7.5X
-ParquetReader Vectorized: DataPageV2 916 919 3 11.5 87.3 5.9X
-SQL ORC Vectorized 974 987 19 10.8 92.9 5.5X
-SQL ORC MR 2695 2707 17 3.9 257.0 2.0X
-
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+SQL CSV 5577 5588 15 1.9 531.9 1.0X
+SQL Json 7460 7462 3 1.4 711.4 0.7X
+SQL Parquet Vectorized: DataPageV1 807 824 15 13.0 76.9 6.9X
+SQL Parquet Vectorized: DataPageV2 1117 1119 2 9.4 106.5 5.0X
+SQL Parquet MR: DataPageV1 2808 2815 10 3.7 267.8 2.0X
+SQL Parquet MR: DataPageV2 2946 2957 15 3.6 280.9 1.9X
+ParquetReader Vectorized: DataPageV1 715 723 8 14.7 68.1 7.8X
+ParquetReader Vectorized: DataPageV2 936 940 3 11.2 89.3 6.0X
+SQL ORC Vectorized 953 971 24 11.0 90.8 5.9X
+SQL ORC MR 2817 2848 43 3.7 268.7 2.0X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
String with Nulls Scan (95.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 3904 3912 11 2.7 372.4 1.0X
-SQL Json 5105 5123 26 2.1 486.8 0.8X
-SQL Parquet Vectorized: DataPageV1 175 179 5 59.8 16.7 22.3X
-SQL Parquet Vectorized: DataPageV2 195 197 1 53.7 18.6 20.0X
-SQL Parquet MR: DataPageV1 1629 1637 11 6.4 155.4 2.4X
-SQL Parquet MR: DataPageV2 1575 1609 48 6.7 150.2 2.5X
-ParquetReader Vectorized: DataPageV1 174 176 1 60.1 16.6 22.4X
-ParquetReader Vectorized: DataPageV2 204 205 1 51.5 19.4 19.2X
-SQL ORC Vectorized 344 351 6 30.5 32.8 11.4X
-SQL ORC MR 1582 1603 29 6.6 150.9 2.5X
+SQL CSV 4347 4352 6 2.4 414.6 1.0X
+SQL Json 5219 5224 6 2.0 497.7 0.8X
+SQL Parquet Vectorized: DataPageV1 163 173 10 64.3 15.6 26.6X
+SQL Parquet Vectorized: DataPageV2 201 227 15 52.1 19.2 21.6X
+SQL Parquet MR: DataPageV1 1765 1774 13 5.9 168.3 2.5X
+SQL Parquet MR: DataPageV2 1710 1712 2 6.1 163.1 2.5X
+ParquetReader Vectorized: DataPageV1 172 179 7 61.0 16.4 25.3X
+ParquetReader Vectorized: DataPageV2 205 211 3 51.0 19.6 21.2X
+SQL ORC Vectorized 332 337 8 31.6 31.6 13.1X
+SQL ORC MR 1636 1637 1 6.4 156.1 2.7X
================================================================================================
Single Column Scan From Wide Columns
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Single Column Scan from 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 1131 1159 40 0.9 1078.4 1.0X
-SQL Json 1731 1740 13 0.6 1650.7 0.7X
-SQL Parquet Vectorized: DataPageV1 25 29 6 42.1 23.7 45.4X
-SQL Parquet Vectorized: DataPageV2 33 37 6 32.1 31.1 34.7X
-SQL Parquet MR: DataPageV1 153 160 6 6.8 146.3 7.4X
-SQL Parquet MR: DataPageV2 151 161 8 6.9 144.1 7.5X
-SQL ORC Vectorized 27 31 6 39.0 25.6 42.0X
-SQL ORC MR 128 135 6 8.2 121.8 8.9X
-
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+SQL CSV 1131 1132 2 0.9 1078.4 1.0X
+SQL Json 1788 1791 4 0.6 1704.9 0.6X
+SQL Parquet Vectorized: DataPageV1 29 35 6 35.8 28.0 38.6X
+SQL Parquet Vectorized: DataPageV2 37 44 7 28.6 35.0 30.8X
+SQL Parquet MR: DataPageV1 157 166 8 6.7 150.2 7.2X
+SQL Parquet MR: DataPageV2 162 175 7 6.5 154.5 7.0X
+SQL ORC Vectorized 32 38 7 32.7 30.6 35.2X
+SQL ORC MR 150 158 8 7.0 142.9 7.5X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Single Column Scan from 50 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 2518 2519 1 0.4 2401.5 1.0X
-SQL Json 5418 5442 33 0.2 5167.0 0.5X
-SQL Parquet Vectorized: DataPageV1 28 34 7 38.1 26.2 91.6X
-SQL Parquet Vectorized: DataPageV2 31 36 6 33.4 30.0 80.2X
-SQL Parquet MR: DataPageV1 159 169 5 6.6 151.7 15.8X
-SQL Parquet MR: DataPageV2 162 171 5 6.5 154.9 15.5X
-SQL ORC Vectorized 30 35 6 34.8 28.7 83.6X
-SQL ORC MR 132 143 7 7.9 126.0 19.1X
-
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+SQL CSV 2424 2438 20 0.4 2311.5 1.0X
+SQL Json 6479 6527 69 0.2 6178.6 0.4X
+SQL Parquet Vectorized: DataPageV1 31 36 6 33.5 29.8 77.5X
+SQL Parquet Vectorized: DataPageV2 38 46 8 27.8 36.0 64.2X
+SQL Parquet MR: DataPageV1 167 174 6 6.3 159.6 14.5X
+SQL Parquet MR: DataPageV2 169 178 6 6.2 161.0 14.4X
+SQL ORC Vectorized 36 45 8 29.3 34.2 67.6X
+SQL ORC MR 164 170 4 6.4 156.6 14.8X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Single Column Scan from 100 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 4195 4198 4 0.2 4000.7 1.0X
-SQL Json 10366 10415 69 0.1 9886.0 0.4X
-SQL Parquet Vectorized: DataPageV1 34 40 7 30.7 32.5 122.9X
-SQL Parquet Vectorized: DataPageV2 38 41 4 27.4 36.5 109.6X
-SQL Parquet MR: DataPageV1 169 178 6 6.2 160.8 24.9X
-SQL Parquet MR: DataPageV2 176 181 4 6.0 167.4 23.9X
-SQL ORC Vectorized 36 41 6 29.1 34.4 116.3X
-SQL ORC MR 140 146 8 7.5 133.4 30.0X
+SQL CSV 3988 4030 59 0.3 3802.9 1.0X
+SQL Json 10875 10928 75 0.1 10370.8 0.4X
+SQL Parquet Vectorized: DataPageV1 38 46 8 27.7 36.1 105.4X
+SQL Parquet Vectorized: DataPageV2 45 52 8 23.1 43.3 87.8X
+SQL Parquet MR: DataPageV1 175 194 8 6.0 167.3 22.7X
+SQL Parquet MR: DataPageV2 186 195 4 5.6 177.8 21.4X
+SQL ORC Vectorized 41 46 6 25.5 39.1 97.1X
+SQL ORC MR 156 169 5 6.7 148.8 25.6X
diff --git a/sql/core/benchmarks/DataSourceReadBenchmark-results.txt b/sql/core/benchmarks/DataSourceReadBenchmark-results.txt
index f8627fb101679..ad66e68428bcf 100644
--- a/sql/core/benchmarks/DataSourceReadBenchmark-results.txt
+++ b/sql/core/benchmarks/DataSourceReadBenchmark-results.txt
@@ -2,437 +2,437 @@
SQL Single Numeric Column Scan
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SQL Single BOOLEAN Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 12149 12272 173 1.3 772.4 1.0X
-SQL Json 8354 8445 129 1.9 531.1 1.5X
-SQL Json with UnsafeRow 9377 9393 22 1.7 596.2 1.3X
-SQL Parquet Vectorized: DataPageV1 101 113 8 155.1 6.4 119.8X
-SQL Parquet Vectorized: DataPageV2 103 112 7 152.8 6.5 118.0X
-SQL Parquet MR: DataPageV1 1767 1776 13 8.9 112.3 6.9X
-SQL Parquet MR: DataPageV2 1721 1730 12 9.1 109.4 7.1X
-SQL ORC Vectorized 136 143 5 115.5 8.7 89.2X
-SQL ORC MR 1485 1489 6 10.6 94.4 8.2X
-
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+SQL CSV 12682 12766 120 1.2 806.3 1.0X
+SQL Json 8232 8269 52 1.9 523.4 1.5X
+SQL Json with UnsafeRow 9077 9103 36 1.7 577.1 1.4X
+SQL Parquet Vectorized: DataPageV1 103 119 9 152.6 6.6 123.0X
+SQL Parquet Vectorized: DataPageV2 117 124 5 133.9 7.5 107.9X
+SQL Parquet MR: DataPageV1 1788 1799 15 8.8 113.7 7.1X
+SQL Parquet MR: DataPageV2 1730 1738 11 9.1 110.0 7.3X
+SQL ORC Vectorized 143 149 4 110.0 9.1 88.7X
+SQL ORC MR 1585 1599 19 9.9 100.8 8.0X
+
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Parquet Reader Single BOOLEAN Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-ParquetReader Vectorized: DataPageV1 87 88 1 180.4 5.5 1.0X
-ParquetReader Vectorized: DataPageV2 94 95 1 166.9 6.0 0.9X
-ParquetReader Vectorized -> Row: DataPageV1 73 74 1 215.3 4.6 1.2X
-ParquetReader Vectorized -> Row: DataPageV2 82 83 2 191.9 5.2 1.1X
+ParquetReader Vectorized: DataPageV1 95 96 1 165.7 6.0 1.0X
+ParquetReader Vectorized: DataPageV2 119 120 1 132.7 7.5 0.8X
+ParquetReader Vectorized -> Row: DataPageV1 75 76 1 208.9 4.8 1.3X
+ParquetReader Vectorized -> Row: DataPageV2 99 102 4 158.1 6.3 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SQL Single TINYINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 11552 11570 25 1.4 734.5 1.0X
-SQL Json 9556 9560 6 1.6 607.6 1.2X
-SQL Json with UnsafeRow 10151 10165 20 1.5 645.4 1.1X
-SQL Parquet Vectorized: DataPageV1 92 99 8 170.9 5.9 125.5X
-SQL Parquet Vectorized: DataPageV2 91 98 5 172.6 5.8 126.8X
-SQL Parquet MR: DataPageV1 1972 2005 47 8.0 125.4 5.9X
-SQL Parquet MR: DataPageV2 1895 1927 45 8.3 120.5 6.1X
-SQL ORC Vectorized 130 139 11 121.3 8.2 89.1X
-SQL ORC MR 1455 1462 11 10.8 92.5 7.9X
-
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+SQL CSV 12590 12760 241 1.2 800.4 1.0X
+SQL Json 9103 9139 51 1.7 578.8 1.4X
+SQL Json with UnsafeRow 9946 9973 37 1.6 632.4 1.3X
+SQL Parquet Vectorized: DataPageV1 92 101 7 171.2 5.8 137.1X
+SQL Parquet Vectorized: DataPageV2 91 98 5 173.0 5.8 138.5X
+SQL Parquet MR: DataPageV1 1855 1864 13 8.5 117.9 6.8X
+SQL Parquet MR: DataPageV2 1808 1812 5 8.7 115.0 7.0X
+SQL ORC Vectorized 115 122 6 136.4 7.3 109.2X
+SQL ORC MR 1459 1459 1 10.8 92.7 8.6X
+
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Parquet Reader Single TINYINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-ParquetReader Vectorized: DataPageV1 68 70 1 231.1 4.3 1.0X
-ParquetReader Vectorized: DataPageV2 68 69 1 230.3 4.3 1.0X
-ParquetReader Vectorized -> Row: DataPageV1 48 49 2 329.2 3.0 1.4X
-ParquetReader Vectorized -> Row: DataPageV2 48 49 1 329.1 3.0 1.4X
+ParquetReader Vectorized: DataPageV1 70 72 2 225.9 4.4 1.0X
+ParquetReader Vectorized: DataPageV2 70 71 1 224.9 4.4 1.0X
+ParquetReader Vectorized -> Row: DataPageV1 47 48 1 336.8 3.0 1.5X
+ParquetReader Vectorized -> Row: DataPageV2 47 48 1 335.4 3.0 1.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SQL Single SMALLINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 11987 11990 4 1.3 762.1 1.0X
-SQL Json 9896 9903 10 1.6 629.2 1.2X
-SQL Json with UnsafeRow 10451 10472 29 1.5 664.5 1.1X
-SQL Parquet Vectorized: DataPageV1 101 105 4 155.8 6.4 118.7X
-SQL Parquet Vectorized: DataPageV2 134 138 4 117.2 8.5 89.3X
-SQL Parquet MR: DataPageV1 2115 2150 50 7.4 134.5 5.7X
-SQL Parquet MR: DataPageV2 1958 1962 6 8.0 124.5 6.1X
-SQL ORC Vectorized 129 132 2 121.7 8.2 92.8X
-SQL ORC MR 1597 1610 18 9.8 101.5 7.5X
-
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+SQL CSV 13145 13165 28 1.2 835.7 1.0X
+SQL Json 9536 9550 20 1.6 606.3 1.4X
+SQL Json with UnsafeRow 10364 10401 53 1.5 658.9 1.3X
+SQL Parquet Vectorized: DataPageV1 109 114 3 143.9 6.9 120.3X
+SQL Parquet Vectorized: DataPageV2 138 141 3 113.7 8.8 95.0X
+SQL Parquet MR: DataPageV1 2015 2019 5 7.8 128.1 6.5X
+SQL Parquet MR: DataPageV2 1969 1977 11 8.0 125.2 6.7X
+SQL ORC Vectorized 127 131 3 123.8 8.1 103.4X
+SQL ORC MR 1673 1677 7 9.4 106.3 7.9X
+
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Parquet Reader Single SMALLINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-ParquetReader Vectorized: DataPageV1 139 140 1 113.2 8.8 1.0X
-ParquetReader Vectorized: DataPageV2 173 174 1 90.9 11.0 0.8X
-ParquetReader Vectorized -> Row: DataPageV1 136 139 2 115.3 8.7 1.0X
-ParquetReader Vectorized -> Row: DataPageV2 170 173 2 92.3 10.8 0.8X
+ParquetReader Vectorized: DataPageV1 141 142 1 111.9 8.9 1.0X
+ParquetReader Vectorized: DataPageV2 170 171 1 92.5 10.8 0.8X
+ParquetReader Vectorized -> Row: DataPageV1 139 140 1 113.4 8.8 1.0X
+ParquetReader Vectorized -> Row: DataPageV2 168 169 2 93.9 10.7 0.8X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SQL Single INT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 13126 13151 35 1.2 834.5 1.0X
-SQL Json 10020 10023 5 1.6 637.0 1.3X
-SQL Json with UnsafeRow 10792 10812 29 1.5 686.2 1.2X
-SQL Parquet Vectorized: DataPageV1 100 107 10 156.8 6.4 130.9X
-SQL Parquet Vectorized: DataPageV2 180 183 5 87.5 11.4 73.0X
-SQL Parquet MR: DataPageV1 2141 2270 183 7.3 136.1 6.1X
-SQL Parquet MR: DataPageV2 1978 2002 34 8.0 125.8 6.6X
-SQL ORC Vectorized 146 150 4 107.6 9.3 89.8X
-SQL ORC MR 1623 1630 10 9.7 103.2 8.1X
-
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+SQL CSV 14110 14121 15 1.1 897.1 1.0X
+SQL Json 9936 9949 19 1.6 631.7 1.4X
+SQL Json with UnsafeRow 10717 10718 1 1.5 681.4 1.3X
+SQL Parquet Vectorized: DataPageV1 108 112 3 145.7 6.9 130.7X
+SQL Parquet Vectorized: DataPageV2 178 182 4 88.3 11.3 79.3X
+SQL Parquet MR: DataPageV1 2086 2135 69 7.5 132.6 6.8X
+SQL Parquet MR: DataPageV2 2059 2070 17 7.6 130.9 6.9X
+SQL ORC Vectorized 146 149 3 107.6 9.3 96.5X
+SQL ORC MR 1668 1670 3 9.4 106.0 8.5X
+
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Parquet Reader Single INT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-ParquetReader Vectorized: DataPageV1 142 144 1 110.7 9.0 1.0X
-ParquetReader Vectorized: DataPageV2 216 218 2 72.8 13.7 0.7X
-ParquetReader Vectorized -> Row: DataPageV1 139 142 2 112.9 8.9 1.0X
-ParquetReader Vectorized -> Row: DataPageV2 214 215 2 73.5 13.6 0.7X
+ParquetReader Vectorized: DataPageV1 142 144 1 110.6 9.0 1.0X
+ParquetReader Vectorized: DataPageV2 212 214 1 74.2 13.5 0.7X
+ParquetReader Vectorized -> Row: DataPageV1 140 144 5 112.3 8.9 1.0X
+ParquetReader Vectorized -> Row: DataPageV2 210 212 1 74.8 13.4 0.7X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SQL Single BIGINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 12940 12949 13 1.2 822.7 1.0X
-SQL Json 10064 10080 22 1.6 639.9 1.3X
-SQL Json with UnsafeRow 10745 10775 43 1.5 683.1 1.2X
-SQL Parquet Vectorized: DataPageV1 286 289 3 55.0 18.2 45.2X
-SQL Parquet Vectorized: DataPageV2 175 178 3 90.0 11.1 74.0X
-SQL Parquet MR: DataPageV1 2462 2484 31 6.4 156.5 5.3X
-SQL Parquet MR: DataPageV2 1998 2017 27 7.9 127.0 6.5X
-SQL ORC Vectorized 132 137 4 118.8 8.4 97.8X
-SQL ORC MR 1673 1677 7 9.4 106.3 7.7X
-
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+SQL CSV 13768 13773 7 1.1 875.4 1.0X
+SQL Json 9915 9933 24 1.6 630.4 1.4X
+SQL Json with UnsafeRow 10669 10685 22 1.5 678.3 1.3X
+SQL Parquet Vectorized: DataPageV1 284 296 8 55.4 18.1 48.5X
+SQL Parquet Vectorized: DataPageV2 200 203 3 78.6 12.7 68.8X
+SQL Parquet MR: DataPageV1 2647 2676 41 5.9 168.3 5.2X
+SQL Parquet MR: DataPageV2 2116 2163 67 7.4 134.5 6.5X
+SQL ORC Vectorized 128 137 9 122.5 8.2 107.2X
+SQL ORC MR 1691 1702 15 9.3 107.5 8.1X
+
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Parquet Reader Single BIGINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-ParquetReader Vectorized: DataPageV1 315 322 11 49.9 20.0 1.0X
-ParquetReader Vectorized: DataPageV2 205 206 1 76.9 13.0 1.5X
-ParquetReader Vectorized -> Row: DataPageV1 332 334 2 47.4 21.1 0.9X
-ParquetReader Vectorized -> Row: DataPageV2 219 221 2 71.8 13.9 1.4X
+ParquetReader Vectorized: DataPageV1 320 322 1 49.1 20.4 1.0X
+ParquetReader Vectorized: DataPageV2 218 220 2 72.1 13.9 1.5X
+ParquetReader Vectorized -> Row: DataPageV1 329 332 2 47.8 20.9 1.0X
+ParquetReader Vectorized -> Row: DataPageV2 232 234 2 67.7 14.8 1.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SQL Single FLOAT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 13460 13468 11 1.2 855.8 1.0X
-SQL Json 11451 11489 54 1.4 728.0 1.2X
-SQL Json with UnsafeRow 12351 12372 29 1.3 785.2 1.1X
-SQL Parquet Vectorized: DataPageV1 82 85 3 190.9 5.2 163.3X
-SQL Parquet Vectorized: DataPageV2 81 84 3 193.8 5.2 165.8X
-SQL Parquet MR: DataPageV1 2027 2080 76 7.8 128.9 6.6X
-SQL Parquet MR: DataPageV2 1913 1922 13 8.2 121.6 7.0X
-SQL ORC Vectorized 242 246 5 65.1 15.4 55.7X
-SQL ORC MR 1788 1789 0 8.8 113.7 7.5X
-
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+SQL CSV 14385 14421 50 1.1 914.6 1.0X
+SQL Json 11200 11214 20 1.4 712.1 1.3X
+SQL Json with UnsafeRow 11987 12002 22 1.3 762.1 1.2X
+SQL Parquet Vectorized: DataPageV1 85 89 3 184.4 5.4 168.7X
+SQL Parquet Vectorized: DataPageV2 85 88 2 184.6 5.4 168.8X
+SQL Parquet MR: DataPageV1 2024 2024 1 7.8 128.7 7.1X
+SQL Parquet MR: DataPageV2 1935 1956 30 8.1 123.1 7.4X
+SQL ORC Vectorized 249 251 2 63.2 15.8 57.8X
+SQL ORC MR 1679 1690 15 9.4 106.8 8.6X
+
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Parquet Reader Single FLOAT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-ParquetReader Vectorized: DataPageV1 138 138 0 114.3 8.7 1.0X
-ParquetReader Vectorized: DataPageV2 134 136 1 117.0 8.5 1.0X
-ParquetReader Vectorized -> Row: DataPageV1 133 136 6 118.1 8.5 1.0X
-ParquetReader Vectorized -> Row: DataPageV2 133 136 3 118.1 8.5 1.0X
+ParquetReader Vectorized: DataPageV1 135 137 1 116.5 8.6 1.0X
+ParquetReader Vectorized: DataPageV2 136 139 5 115.8 8.6 1.0X
+ParquetReader Vectorized -> Row: DataPageV1 134 138 8 117.1 8.5 1.0X
+ParquetReader Vectorized -> Row: DataPageV2 134 139 4 117.0 8.5 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SQL Single DOUBLE Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 13669 13678 12 1.2 869.1 1.0X
-SQL Json 11696 11714 25 1.3 743.6 1.2X
-SQL Json with UnsafeRow 12529 12545 23 1.3 796.6 1.1X
-SQL Parquet Vectorized: DataPageV1 261 263 3 60.3 16.6 52.4X
-SQL Parquet Vectorized: DataPageV2 260 264 3 60.4 16.6 52.5X
-SQL Parquet MR: DataPageV1 2457 2468 16 6.4 156.2 5.6X
-SQL Parquet MR: DataPageV2 2334 2398 91 6.7 148.4 5.9X
-SQL ORC Vectorized 576 582 4 27.3 36.7 23.7X
-SQL ORC MR 2114 2121 10 7.4 134.4 6.5X
-
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+SQL CSV 14585 14594 13 1.1 927.3 1.0X
+SQL Json 11269 11294 35 1.4 716.5 1.3X
+SQL Json with UnsafeRow 12098 12126 40 1.3 769.2 1.2X
+SQL Parquet Vectorized: DataPageV1 265 268 3 59.4 16.8 55.1X
+SQL Parquet Vectorized: DataPageV2 264 266 2 59.6 16.8 55.2X
+SQL Parquet MR: DataPageV1 2599 2606 10 6.1 165.2 5.6X
+SQL Parquet MR: DataPageV2 2488 2490 3 6.3 158.2 5.9X
+SQL ORC Vectorized 579 582 4 27.2 36.8 25.2X
+SQL ORC MR 2117 2144 38 7.4 134.6 6.9X
+
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Parquet Reader Single DOUBLE Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-ParquetReader Vectorized: DataPageV1 314 315 1 50.2 19.9 1.0X
-ParquetReader Vectorized: DataPageV2 308 324 31 51.1 19.6 1.0X
-ParquetReader Vectorized -> Row: DataPageV1 334 335 1 47.1 21.2 0.9X
-ParquetReader Vectorized -> Row: DataPageV2 335 336 1 47.0 21.3 0.9X
+ParquetReader Vectorized: DataPageV1 315 317 1 50.0 20.0 1.0X
+ParquetReader Vectorized: DataPageV2 317 318 1 49.6 20.2 1.0X
+ParquetReader Vectorized -> Row: DataPageV1 334 336 3 47.1 21.2 0.9X
+ParquetReader Vectorized -> Row: DataPageV2 333 334 1 47.2 21.2 0.9X
================================================================================================
SQL Single Numeric Column Scan in Struct
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SQL Single TINYINT Column Scan in Struct: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------
-SQL ORC MR 1958 1972 21 8.0 124.5 1.0X
-SQL ORC Vectorized (Nested Column Disabled) 1985 1986 1 7.9 126.2 1.0X
-SQL ORC Vectorized (Nested Column Enabled) 127 133 6 123.8 8.1 15.4X
-SQL Parquet MR: DataPageV1 2395 2397 2 6.6 152.3 0.8X
-SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 2745 2754 12 5.7 174.5 0.7X
-SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 99 105 5 158.2 6.3 19.7X
-SQL Parquet MR: DataPageV2 2291 2291 1 6.9 145.6 0.9X
-SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2675 2681 9 5.9 170.1 0.7X
-SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 99 104 5 159.2 6.3 19.8X
-
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+SQL ORC MR 2048 2050 3 7.7 130.2 1.0X
+SQL ORC Vectorized (Nested Column Disabled) 2030 2053 33 7.7 129.1 1.0X
+SQL ORC Vectorized (Nested Column Enabled) 112 116 4 140.7 7.1 18.3X
+SQL Parquet MR: DataPageV1 2205 2207 4 7.1 140.2 0.9X
+SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 2760 2826 93 5.7 175.5 0.7X
+SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 95 99 5 166.0 6.0 21.6X
+SQL Parquet MR: DataPageV2 2145 2147 2 7.3 136.4 1.0X
+SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2587 2591 6 6.1 164.5 0.8X
+SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 94 99 3 166.9 6.0 21.7X
+
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SQL Single SMALLINT Column Scan in Struct: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------
-SQL ORC MR 2000 2006 8 7.9 127.2 1.0X
-SQL ORC Vectorized (Nested Column Disabled) 2011 2016 7 7.8 127.9 1.0X
-SQL ORC Vectorized (Nested Column Enabled) 284 290 7 55.3 18.1 7.0X
-SQL Parquet MR: DataPageV1 2499 2504 8 6.3 158.9 0.8X
-SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 3087 3089 3 5.1 196.3 0.6X
-SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 112 120 8 141.0 7.1 17.9X
-SQL Parquet MR: DataPageV2 2322 2323 2 6.8 147.6 0.9X
-SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2966 2969 5 5.3 188.6 0.7X
-SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 240 253 10 65.5 15.3 8.3X
-
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+SQL ORC MR 2057 2083 38 7.6 130.8 1.0X
+SQL ORC Vectorized (Nested Column Disabled) 2067 2072 7 7.6 131.4 1.0X
+SQL ORC Vectorized (Nested Column Enabled) 257 262 7 61.3 16.3 8.0X
+SQL Parquet MR: DataPageV1 2475 2480 7 6.4 157.3 0.8X
+SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 3189 3196 10 4.9 202.8 0.6X
+SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 125 129 4 125.9 7.9 16.5X
+SQL Parquet MR: DataPageV2 2370 2371 1 6.6 150.7 0.9X
+SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2888 2892 6 5.4 183.6 0.7X
+SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 244 247 2 64.4 15.5 8.4X
+
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SQL Single INT Column Scan in Struct: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------
-SQL ORC MR 2118 2196 110 7.4 134.6 1.0X
-SQL ORC Vectorized (Nested Column Disabled) 2129 2184 77 7.4 135.4 1.0X
-SQL ORC Vectorized (Nested Column Enabled) 293 304 9 53.8 18.6 7.2X
-SQL Parquet MR: DataPageV1 2483 2505 32 6.3 157.8 0.9X
-SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 3260 3265 6 4.8 207.3 0.6X
-SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 113 127 17 138.8 7.2 18.7X
-SQL Parquet MR: DataPageV2 2326 2343 23 6.8 147.9 0.9X
-SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2928 2944 23 5.4 186.2 0.7X
-SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 259 264 3 60.8 16.4 8.2X
-
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+SQL ORC MR 2069 2131 88 7.6 131.5 1.0X
+SQL ORC Vectorized (Nested Column Disabled) 2085 2115 43 7.5 132.5 1.0X
+SQL ORC Vectorized (Nested Column Enabled) 278 287 6 56.6 17.7 7.4X
+SQL Parquet MR: DataPageV1 2350 2355 7 6.7 149.4 0.9X
+SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 2822 2836 20 5.6 179.4 0.7X
+SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 115 118 3 136.4 7.3 17.9X
+SQL Parquet MR: DataPageV2 2285 2290 7 6.9 145.3 0.9X
+SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2970 2981 16 5.3 188.8 0.7X
+SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 257 260 3 61.2 16.3 8.0X
+
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SQL Single BIGINT Column Scan in Struct: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------
-SQL ORC MR 2122 2165 61 7.4 134.9 1.0X
-SQL ORC Vectorized (Nested Column Disabled) 2113 2115 3 7.4 134.4 1.0X
-SQL ORC Vectorized (Nested Column Enabled) 308 315 5 51.1 19.6 6.9X
-SQL Parquet MR: DataPageV1 2838 2865 38 5.5 180.4 0.7X
-SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 3238 3250 17 4.9 205.9 0.7X
-SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 303 307 5 51.9 19.3 7.0X
-SQL Parquet MR: DataPageV2 2315 2332 24 6.8 147.2 0.9X
-SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2685 2686 1 5.9 170.7 0.8X
-SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 187 197 8 84.2 11.9 11.4X
-
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+SQL ORC MR 2119 2146 39 7.4 134.7 1.0X
+SQL ORC Vectorized (Nested Column Disabled) 2143 2169 37 7.3 136.2 1.0X
+SQL ORC Vectorized (Nested Column Enabled) 280 286 7 56.1 17.8 7.6X
+SQL Parquet MR: DataPageV1 2785 2794 14 5.6 177.0 0.8X
+SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 3359 3363 6 4.7 213.5 0.6X
+SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 291 294 2 54.1 18.5 7.3X
+SQL Parquet MR: DataPageV2 2293 2321 39 6.9 145.8 0.9X
+SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2871 2877 8 5.5 182.5 0.7X
+SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 190 194 4 82.8 12.1 11.2X
+
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SQL Single FLOAT Column Scan in Struct: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------
-SQL ORC MR 2243 2255 17 7.0 142.6 1.0X
-SQL ORC Vectorized (Nested Column Disabled) 2229 2251 31 7.1 141.7 1.0X
-SQL ORC Vectorized (Nested Column Enabled) 379 395 11 41.5 24.1 5.9X
-SQL Parquet MR: DataPageV1 2388 2408 28 6.6 151.8 0.9X
-SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 2869 2881 17 5.5 182.4 0.8X
-SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 95 116 16 165.2 6.1 23.6X
-SQL Parquet MR: DataPageV2 2322 2333 16 6.8 147.6 1.0X
-SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2769 2797 40 5.7 176.0 0.8X
-SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 90 96 6 175.4 5.7 25.0X
-
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+SQL ORC MR 2262 2314 73 7.0 143.8 1.0X
+SQL ORC Vectorized (Nested Column Disabled) 2157 2185 40 7.3 137.1 1.0X
+SQL ORC Vectorized (Nested Column Enabled) 380 383 3 41.3 24.2 5.9X
+SQL Parquet MR: DataPageV1 2395 2400 7 6.6 152.2 0.9X
+SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 2865 2869 6 5.5 182.2 0.8X
+SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 94 100 6 167.9 6.0 24.2X
+SQL Parquet MR: DataPageV2 2343 2351 12 6.7 148.9 1.0X
+SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 2799 2807 11 5.6 177.9 0.8X
+SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 93 99 7 169.2 5.9 24.3X
+
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SQL Single DOUBLE Column Scan in Struct: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------
-SQL ORC MR 2570 2570 0 6.1 163.4 1.0X
-SQL ORC Vectorized (Nested Column Disabled) 2576 2581 6 6.1 163.8 1.0X
-SQL ORC Vectorized (Nested Column Enabled) 721 722 2 21.8 45.8 3.6X
-SQL Parquet MR: DataPageV1 2808 2831 34 5.6 178.5 0.9X
-SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 3286 3316 42 4.8 208.9 0.8X
-SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 283 296 12 55.5 18.0 9.1X
-SQL Parquet MR: DataPageV2 2642 2659 23 6.0 168.0 1.0X
-SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 3126 3129 5 5.0 198.7 0.8X
-SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 289 297 7 54.4 18.4 8.9X
+SQL ORC MR 2589 2613 34 6.1 164.6 1.0X
+SQL ORC Vectorized (Nested Column Disabled) 2600 2638 53 6.0 165.3 1.0X
+SQL ORC Vectorized (Nested Column Enabled) 723 735 13 21.8 46.0 3.6X
+SQL Parquet MR: DataPageV1 2782 2807 36 5.7 176.9 0.9X
+SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 3259 3266 11 4.8 207.2 0.8X
+SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 279 288 7 56.4 17.7 9.3X
+SQL Parquet MR: DataPageV2 2699 2720 31 5.8 171.6 1.0X
+SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 3174 3182 12 5.0 201.8 0.8X
+SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 275 284 7 57.1 17.5 9.4X
================================================================================================
SQL Nested Column Scan
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
SQL Nested Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------
-SQL ORC MR 12768 12973 132 0.1 12176.7 1.0X
-SQL ORC Vectorized (Nested Column Disabled) 12932 13289 327 0.1 12333.3 1.0X
-SQL ORC Vectorized (Nested Column Enabled) 7205 7270 60 0.1 6870.8 1.8X
-SQL Parquet MR: DataPageV1 9028 9123 63 0.1 8609.9 1.4X
-SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 9363 9406 38 0.1 8928.9 1.4X
-SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 5947 6003 50 0.2 5671.6 2.1X
-SQL Parquet MR: DataPageV2 9731 10064 276 0.1 9280.2 1.3X
-SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 9999 10015 14 0.1 9535.5 1.3X
-SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 5554 5581 16 0.2 5297.1 2.3X
+SQL ORC MR 13352 13556 177 0.1 12733.3 1.0X
+SQL ORC Vectorized (Nested Column Disabled) 13179 13338 148 0.1 12568.6 1.0X
+SQL ORC Vectorized (Nested Column Enabled) 7419 7472 53 0.1 7075.5 1.8X
+SQL Parquet MR: DataPageV1 8783 8858 49 0.1 8375.8 1.5X
+SQL Parquet Vectorized: DataPageV1 (Nested Column Disabled) 9105 9163 39 0.1 8683.3 1.5X
+SQL Parquet Vectorized: DataPageV1 (Nested Column Enabled) 5874 5929 36 0.2 5601.5 2.3X
+SQL Parquet MR: DataPageV2 9562 9661 78 0.1 9118.9 1.4X
+SQL Parquet Vectorized: DataPageV2 (Nested Column Disabled) 9851 9891 24 0.1 9394.4 1.4X
+SQL Parquet Vectorized: DataPageV2 (Nested Column Enabled) 5640 5684 27 0.2 5379.0 2.4X
================================================================================================
Int and String Scan
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Int and String Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 12435 12520 120 0.8 1185.9 1.0X
-SQL Json 10492 10539 67 1.0 1000.6 1.2X
-SQL Parquet Vectorized: DataPageV1 1788 1793 7 5.9 170.5 7.0X
-SQL Parquet Vectorized: DataPageV2 1918 1924 9 5.5 182.9 6.5X
-SQL Parquet MR: DataPageV1 4163 4169 8 2.5 397.0 3.0X
-SQL Parquet MR: DataPageV2 4092 4108 21 2.6 390.3 3.0X
-SQL ORC Vectorized 1843 1847 6 5.7 175.7 6.7X
-SQL ORC MR 3565 3573 11 2.9 340.0 3.5X
+SQL CSV 12583 12752 240 0.8 1200.0 1.0X
+SQL Json 10318 10364 64 1.0 984.0 1.2X
+SQL Parquet Vectorized: DataPageV1 1792 1802 15 5.9 170.9 7.0X
+SQL Parquet Vectorized: DataPageV2 1890 1897 10 5.5 180.3 6.7X
+SQL Parquet MR: DataPageV1 3919 3921 2 2.7 373.8 3.2X
+SQL Parquet MR: DataPageV2 3924 3945 30 2.7 374.2 3.2X
+SQL ORC Vectorized 1922 1928 9 5.5 183.3 6.5X
+SQL ORC MR 3550 3556 8 3.0 338.6 3.5X
================================================================================================
Repeated String Scan
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Repeated String: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 7117 7146 41 1.5 678.8 1.0X
-SQL Json 6813 6814 1 1.5 649.7 1.0X
-SQL Parquet Vectorized: DataPageV1 487 492 5 21.5 46.4 14.6X
-SQL Parquet Vectorized: DataPageV2 492 502 12 21.3 46.9 14.5X
-SQL Parquet MR: DataPageV1 1659 1667 12 6.3 158.2 4.3X
-SQL Parquet MR: DataPageV2 1613 1621 11 6.5 153.8 4.4X
-SQL ORC Vectorized 398 409 13 26.3 38.0 17.9X
-SQL ORC MR 1857 1860 5 5.6 177.1 3.8X
+SQL CSV 7707 7781 104 1.4 735.0 1.0X
+SQL Json 6650 6655 7 1.6 634.2 1.2X
+SQL Parquet Vectorized: DataPageV1 511 524 18 20.5 48.7 15.1X
+SQL Parquet Vectorized: DataPageV2 513 521 14 20.5 48.9 15.0X
+SQL Parquet MR: DataPageV1 1628 1674 66 6.4 155.2 4.7X
+SQL Parquet MR: DataPageV2 1576 1580 7 6.7 150.3 4.9X
+SQL ORC Vectorized 408 411 2 25.7 38.9 18.9X
+SQL ORC MR 1683 1683 0 6.2 160.5 4.6X
================================================================================================
Partitioned Table Scan
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Partitioned Table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------
-Data column - CSV 13634 13651 24 1.2 866.8 1.0X
-Data column - Json 10104 10109 7 1.6 642.4 1.3X
-Data column - Parquet Vectorized: DataPageV1 102 122 17 154.3 6.5 133.8X
-Data column - Parquet Vectorized: DataPageV2 236 252 17 66.8 15.0 57.9X
-Data column - Parquet MR: DataPageV1 2341 2361 28 6.7 148.9 5.8X
-Data column - Parquet MR: DataPageV2 2297 2311 20 6.8 146.0 5.9X
-Data column - ORC Vectorized 146 151 5 108.1 9.3 93.7X
-Data column - ORC MR 1848 1849 0 8.5 117.5 7.4X
-Partition column - CSV 3362 3435 103 4.7 213.8 4.1X
-Partition column - Json 8775 8801 37 1.8 557.9 1.6X
-Partition column - Parquet Vectorized: DataPageV1 31 34 3 512.8 2.0 444.5X
-Partition column - Parquet Vectorized: DataPageV2 30 34 4 522.3 1.9 452.8X
-Partition column - Parquet MR: DataPageV1 1205 1208 4 13.1 76.6 11.3X
-Partition column - Parquet MR: DataPageV2 1218 1240 31 12.9 77.4 11.2X
-Partition column - ORC Vectorized 29 32 3 549.4 1.8 476.3X
-Partition column - ORC MR 1255 1260 7 12.5 79.8 10.9X
-Both columns - CSV 13329 13349 28 1.2 847.4 1.0X
-Both columns - Json 10467 10476 13 1.5 665.5 1.3X
-Both columns - Parquet Vectorized: DataPageV1 121 140 14 129.6 7.7 112.4X
-Both columns - Parquet Vectorized: DataPageV2 250 275 17 63.0 15.9 54.6X
-Both columns - Parquet MR: DataPageV1 2295 2316 31 6.9 145.9 5.9X
-Both columns - Parquet MR: DataPageV2 2356 2378 30 6.7 149.8 5.8X
-Both columns - ORC Vectorized 178 187 16 88.5 11.3 76.7X
-Both columns - ORC MR 1893 1893 0 8.3 120.3 7.2X
+Data column - CSV 14303 14342 55 1.1 909.4 1.0X
+Data column - Json 9732 9830 139 1.6 618.8 1.5X
+Data column - Parquet Vectorized: DataPageV1 109 120 6 144.7 6.9 131.6X
+Data column - Parquet Vectorized: DataPageV2 264 275 8 59.7 16.8 54.3X
+Data column - Parquet MR: DataPageV1 2340 2346 9 6.7 148.8 6.1X
+Data column - Parquet MR: DataPageV2 2230 2238 11 7.1 141.8 6.4X
+Data column - ORC Vectorized 172 192 9 91.2 11.0 83.0X
+Data column - ORC MR 2021 2031 14 7.8 128.5 7.1X
+Partition column - CSV 3387 3413 36 4.6 215.4 4.2X
+Partition column - Json 8342 8348 9 1.9 530.4 1.7X
+Partition column - Parquet Vectorized: DataPageV1 33 36 3 470.0 2.1 427.4X
+Partition column - Parquet Vectorized: DataPageV2 34 37 4 466.6 2.1 424.3X
+Partition column - Parquet MR: DataPageV1 1204 1209 7 13.1 76.5 11.9X
+Partition column - Parquet MR: DataPageV2 1202 1210 11 13.1 76.4 11.9X
+Partition column - ORC Vectorized 36 38 3 438.3 2.3 398.6X
+Partition column - ORC MR 1348 1349 2 11.7 85.7 10.6X
+Both columns - CSV 14140 14184 62 1.1 899.0 1.0X
+Both columns - Json 10105 10120 21 1.6 642.4 1.4X
+Both columns - Parquet Vectorized: DataPageV1 140 155 17 112.2 8.9 102.0X
+Both columns - Parquet Vectorized: DataPageV2 302 330 21 52.1 19.2 47.4X
+Both columns - Parquet MR: DataPageV1 2385 2404 26 6.6 151.6 6.0X
+Both columns - Parquet MR: DataPageV2 2354 2374 28 6.7 149.6 6.1X
+Both columns - ORC Vectorized 208 244 21 75.5 13.2 68.7X
+Both columns - ORC MR 2162 2163 1 7.3 137.4 6.6X
================================================================================================
String with Nulls Scan
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
String with Nulls Scan (0.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 8636 8656 28 1.2 823.6 1.0X
-SQL Json 8982 9006 34 1.2 856.6 1.0X
-SQL Parquet Vectorized: DataPageV1 1302 1305 3 8.1 124.2 6.6X
-SQL Parquet Vectorized: DataPageV2 1342 1360 25 7.8 128.0 6.4X
-SQL Parquet MR: DataPageV1 3495 3511 22 3.0 333.3 2.5X
-SQL Parquet MR: DataPageV2 3660 3673 17 2.9 349.1 2.4X
-ParquetReader Vectorized: DataPageV1 859 862 3 12.2 81.9 10.1X
-ParquetReader Vectorized: DataPageV2 873 877 6 12.0 83.3 9.9X
-SQL ORC Vectorized 923 940 17 11.4 88.0 9.4X
-SQL ORC MR 2963 2966 4 3.5 282.6 2.9X
-
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+SQL CSV 9147 9239 130 1.1 872.4 1.0X
+SQL Json 9080 9083 4 1.2 865.9 1.0X
+SQL Parquet Vectorized: DataPageV1 1277 1299 31 8.2 121.8 7.2X
+SQL Parquet Vectorized: DataPageV2 1369 1387 26 7.7 130.6 6.7X
+SQL Parquet MR: DataPageV1 3487 3491 5 3.0 332.6 2.6X
+SQL Parquet MR: DataPageV2 3582 3583 2 2.9 341.6 2.6X
+ParquetReader Vectorized: DataPageV1 869 886 19 12.1 82.9 10.5X
+ParquetReader Vectorized: DataPageV2 899 903 4 11.7 85.7 10.2X
+SQL ORC Vectorized 971 988 15 10.8 92.6 9.4X
+SQL ORC MR 2933 2955 32 3.6 279.7 3.1X
+
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
String with Nulls Scan (50.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 6432 6443 15 1.6 613.4 1.0X
-SQL Json 7575 7586 15 1.4 722.4 0.8X
-SQL Parquet Vectorized: DataPageV1 867 885 15 12.1 82.7 7.4X
-SQL Parquet Vectorized: DataPageV2 927 946 22 11.3 88.4 6.9X
-SQL Parquet MR: DataPageV1 2864 2868 5 3.7 273.1 2.2X
-SQL Parquet MR: DataPageV2 3055 3055 0 3.4 291.3 2.1X
-ParquetReader Vectorized: DataPageV1 755 767 11 13.9 72.0 8.5X
-ParquetReader Vectorized: DataPageV2 793 801 7 13.2 75.6 8.1X
-SQL ORC Vectorized 999 1001 4 10.5 95.2 6.4X
-SQL ORC MR 2740 2743 4 3.8 261.3 2.3X
-
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+SQL CSV 6715 6773 83 1.6 640.4 1.0X
+SQL Json 7526 7531 6 1.4 717.8 0.9X
+SQL Parquet Vectorized: DataPageV1 859 875 20 12.2 81.9 7.8X
+SQL Parquet Vectorized: DataPageV2 950 958 7 11.0 90.6 7.1X
+SQL Parquet MR: DataPageV1 2656 2668 18 3.9 253.3 2.5X
+SQL Parquet MR: DataPageV2 2819 2828 13 3.7 268.8 2.4X
+ParquetReader Vectorized: DataPageV1 760 780 19 13.8 72.5 8.8X
+ParquetReader Vectorized: DataPageV2 823 830 12 12.7 78.5 8.2X
+SQL ORC Vectorized 1040 1040 0 10.1 99.2 6.5X
+SQL ORC MR 2782 2791 13 3.8 265.3 2.4X
+
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
String with Nulls Scan (95.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 4835 4848 18 2.2 461.1 1.0X
-SQL Json 5469 5472 5 1.9 521.5 0.9X
-SQL Parquet Vectorized: DataPageV1 166 169 2 63.0 15.9 29.1X
-SQL Parquet Vectorized: DataPageV2 183 184 2 57.3 17.5 26.4X
-SQL Parquet MR: DataPageV1 1775 1785 14 5.9 169.3 2.7X
-SQL Parquet MR: DataPageV2 1800 1815 21 5.8 171.7 2.7X
-ParquetReader Vectorized: DataPageV1 169 173 2 62.2 16.1 28.7X
-ParquetReader Vectorized: DataPageV2 183 187 2 57.4 17.4 26.5X
-SQL ORC Vectorized 304 308 2 34.5 29.0 15.9X
-SQL ORC MR 1535 1541 9 6.8 146.4 3.2X
+SQL CSV 4498 4509 16 2.3 429.0 1.0X
+SQL Json 5349 5355 8 2.0 510.1 0.8X
+SQL Parquet Vectorized: DataPageV1 167 172 2 63.0 15.9 27.0X
+SQL Parquet Vectorized: DataPageV2 189 190 1 55.5 18.0 23.8X
+SQL Parquet MR: DataPageV1 1658 1675 24 6.3 158.1 2.7X
+SQL Parquet MR: DataPageV2 1613 1640 39 6.5 153.8 2.8X
+ParquetReader Vectorized: DataPageV1 172 173 1 61.0 16.4 26.1X
+ParquetReader Vectorized: DataPageV2 191 194 3 54.9 18.2 23.5X
+SQL ORC Vectorized 331 336 8 31.7 31.5 13.6X
+SQL ORC MR 1634 1642 11 6.4 155.8 2.8X
================================================================================================
Single Column Scan From Wide Columns
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Single Column Scan from 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 1202 1210 11 0.9 1146.3 1.0X
-SQL Json 1734 1735 1 0.6 1654.0 0.7X
-SQL Parquet Vectorized: DataPageV1 24 27 3 43.9 22.8 50.3X
-SQL Parquet Vectorized: DataPageV2 32 34 3 32.7 30.5 37.5X
-SQL Parquet MR: DataPageV1 156 160 4 6.7 149.0 7.7X
-SQL Parquet MR: DataPageV2 153 156 2 6.9 145.9 7.9X
-SQL ORC Vectorized 29 33 4 36.0 27.8 41.3X
-SQL ORC MR 123 129 6 8.5 117.2 9.8X
-
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+SQL CSV 1267 1279 17 0.8 1208.4 1.0X
+SQL Json 1757 1762 6 0.6 1676.0 0.7X
+SQL Parquet Vectorized: DataPageV1 25 29 4 41.5 24.1 50.2X
+SQL Parquet Vectorized: DataPageV2 34 38 4 30.9 32.3 37.4X
+SQL Parquet MR: DataPageV1 158 163 4 6.6 150.6 8.0X
+SQL Parquet MR: DataPageV2 154 158 4 6.8 146.5 8.3X
+SQL ORC Vectorized 30 32 4 35.1 28.5 42.4X
+SQL ORC MR 130 132 3 8.1 123.9 9.8X
+
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Single Column Scan from 50 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 2618 2631 18 0.4 2497.0 1.0X
-SQL Json 6142 6146 6 0.2 5857.1 0.4X
-SQL Parquet Vectorized: DataPageV1 26 29 3 40.1 24.9 100.2X
-SQL Parquet Vectorized: DataPageV2 34 37 4 31.1 32.2 77.6X
-SQL Parquet MR: DataPageV1 161 165 5 6.5 153.4 16.3X
-SQL Parquet MR: DataPageV2 155 161 4 6.7 148.2 16.8X
-SQL ORC Vectorized 32 35 3 32.9 30.4 82.1X
-SQL ORC MR 125 129 3 8.4 119.5 20.9X
-
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+SQL CSV 2691 2740 70 0.4 2566.3 1.0X
+SQL Json 6037 6057 29 0.2 5757.4 0.4X
+SQL Parquet Vectorized: DataPageV1 28 30 3 38.0 26.3 97.5X
+SQL Parquet Vectorized: DataPageV2 37 40 4 28.3 35.3 72.6X
+SQL Parquet MR: DataPageV1 163 167 3 6.4 155.9 16.5X
+SQL Parquet MR: DataPageV2 158 163 6 6.6 150.7 17.0X
+SQL ORC Vectorized 34 37 3 31.3 32.0 80.3X
+SQL ORC MR 133 136 2 7.9 127.2 20.2X
+
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Single Column Scan from 100 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-SQL CSV 4450 4453 4 0.2 4244.0 1.0X
-SQL Json 11304 11358 77 0.1 10780.6 0.4X
-SQL Parquet Vectorized: DataPageV1 34 37 4 30.9 32.4 131.2X
-SQL Parquet Vectorized: DataPageV2 41 46 6 25.6 39.1 108.5X
-SQL Parquet MR: DataPageV1 169 177 6 6.2 160.7 26.4X
-SQL Parquet MR: DataPageV2 168 172 3 6.3 159.7 26.6X
-SQL ORC Vectorized 37 41 7 28.2 35.5 119.6X
-SQL ORC MR 129 135 4 8.1 123.1 34.5X
+SQL CSV 4417 4431 19 0.2 4212.5 1.0X
+SQL Json 11107 11201 134 0.1 10592.1 0.4X
+SQL Parquet Vectorized: DataPageV1 36 40 5 29.3 34.1 123.5X
+SQL Parquet Vectorized: DataPageV2 45 49 5 23.5 42.5 99.1X
+SQL Parquet MR: DataPageV1 176 180 2 5.9 168.1 25.1X
+SQL Parquet MR: DataPageV2 172 175 3 6.1 163.6 25.7X
+SQL ORC Vectorized 43 47 5 24.6 40.7 103.6X
+SQL ORC MR 146 149 3 7.2 139.3 30.2X
diff --git a/sql/core/benchmarks/DatasetBenchmark-jdk21-results.txt b/sql/core/benchmarks/DatasetBenchmark-jdk21-results.txt
index 6f45a55c40ec6..051577fa4a9f0 100644
--- a/sql/core/benchmarks/DatasetBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/DatasetBenchmark-jdk21-results.txt
@@ -2,45 +2,45 @@
Dataset Benchmark
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
back-to-back map long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-RDD 6520 6520 1 15.3 65.2 1.0X
-DataFrame 1192 1241 68 83.9 11.9 5.5X
-Dataset 1640 1705 92 61.0 16.4 4.0X
+RDD 6456 6516 85 15.5 64.6 1.0X
+DataFrame 1215 1262 67 82.3 12.1 5.3X
+Dataset 1722 1726 6 58.1 17.2 3.7X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
back-to-back map: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-RDD 7453 7459 9 13.4 74.5 1.0X
-DataFrame 2922 2940 25 34.2 29.2 2.6X
-Dataset 7021 7024 4 14.2 70.2 1.1X
+RDD 7533 7547 20 13.3 75.3 1.0X
+DataFrame 2802 2841 55 35.7 28.0 2.7X
+Dataset 7391 7397 8 13.5 73.9 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
back-to-back filter Long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-RDD 4428 4492 90 22.6 44.3 1.0X
-DataFrame 728 736 10 137.5 7.3 6.1X
-Dataset 1638 1654 23 61.1 16.4 2.7X
+RDD 4352 4379 38 23.0 43.5 1.0X
+DataFrame 714 730 20 140.1 7.1 6.1X
+Dataset 2404 2407 4 41.6 24.0 1.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
back-to-back filter: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-RDD 2086 2095 12 47.9 20.9 1.0X
-DataFrame 110 122 10 912.5 1.1 19.0X
-Dataset 2367 2398 45 42.3 23.7 0.9X
+RDD 2082 2116 47 48.0 20.8 1.0X
+DataFrame 112 125 16 896.6 1.1 18.7X
+Dataset 2342 2375 46 42.7 23.4 0.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
aggregate: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-RDD sum 1404 1408 5 71.2 14.0 1.0X
-DataFrame sum 57 70 10 1767.9 0.6 24.8X
-Dataset sum using Aggregator 1944 2023 111 51.4 19.4 0.7X
-Dataset complex Aggregator 4838 4881 60 20.7 48.4 0.3X
+RDD sum 1402 1412 14 71.3 14.0 1.0X
+DataFrame sum 68 83 11 1470.1 0.7 20.6X
+Dataset sum using Aggregator 1946 2009 89 51.4 19.5 0.7X
+Dataset complex Aggregator 5018 5103 119 19.9 50.2 0.3X
diff --git a/sql/core/benchmarks/DatasetBenchmark-results.txt b/sql/core/benchmarks/DatasetBenchmark-results.txt
index 856b2302069a0..5cdca4d5041b2 100644
--- a/sql/core/benchmarks/DatasetBenchmark-results.txt
+++ b/sql/core/benchmarks/DatasetBenchmark-results.txt
@@ -2,45 +2,45 @@
Dataset Benchmark
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
back-to-back map long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-RDD 5879 6176 420 17.0 58.8 1.0X
-DataFrame 1159 1189 43 86.3 11.6 5.1X
-Dataset 1617 1667 70 61.8 16.2 3.6X
+RDD 5736 5828 129 17.4 57.4 1.0X
+DataFrame 1164 1168 6 85.9 11.6 4.9X
+Dataset 1604 1701 137 62.4 16.0 3.6X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
back-to-back map: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-RDD 6934 6965 44 14.4 69.3 1.0X
-DataFrame 2836 2893 82 35.3 28.4 2.4X
-Dataset 6597 6669 102 15.2 66.0 1.1X
+RDD 6867 6977 155 14.6 68.7 1.0X
+DataFrame 2745 2782 52 36.4 27.5 2.5X
+Dataset 6475 6615 198 15.4 64.7 1.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
back-to-back filter Long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-RDD 4186 4227 57 23.9 41.9 1.0X
-DataFrame 671 703 53 148.9 6.7 6.2X
-Dataset 1598 1599 1 62.6 16.0 2.6X
+RDD 4072 4132 84 24.6 40.7 1.0X
+DataFrame 683 696 12 146.4 6.8 6.0X
+Dataset 1724 1763 55 58.0 17.2 2.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
back-to-back filter: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-RDD 2083 2087 6 48.0 20.8 1.0X
-DataFrame 111 124 11 903.6 1.1 18.8X
-Dataset 2453 2498 65 40.8 24.5 0.8X
+RDD 2035 2090 78 49.1 20.3 1.0X
+DataFrame 108 120 11 927.7 1.1 18.9X
+Dataset 2467 2477 14 40.5 24.7 0.8X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
aggregate: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-RDD sum 1412 1427 20 70.8 14.1 1.0X
-DataFrame sum 69 81 9 1447.3 0.7 20.4X
-Dataset sum using Aggregator 2107 2148 58 47.5 21.1 0.7X
-Dataset complex Aggregator 4949 5079 183 20.2 49.5 0.3X
+RDD sum 1402 1422 29 71.3 14.0 1.0X
+DataFrame sum 67 78 9 1499.3 0.7 21.0X
+Dataset sum using Aggregator 2099 2104 7 47.6 21.0 0.7X
+Dataset complex Aggregator 4956 5026 99 20.2 49.6 0.3X
diff --git a/sql/core/benchmarks/DateTimeBenchmark-jdk21-results.txt b/sql/core/benchmarks/DateTimeBenchmark-jdk21-results.txt
index be96ecb63d510..09f3b3dbba4c6 100644
--- a/sql/core/benchmarks/DateTimeBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/DateTimeBenchmark-jdk21-results.txt
@@ -2,460 +2,460 @@
datetime +/- interval
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
datetime +/- interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date + interval(m) 910 932 21 11.0 91.0 1.0X
-date + interval(m, d) 899 900 1 11.1 89.9 1.0X
-date + interval(m, d, ms) 3476 3482 8 2.9 347.6 0.3X
-date - interval(m) 826 830 4 12.1 82.6 1.1X
-date - interval(m, d) 886 888 2 11.3 88.6 1.0X
-date - interval(m, d, ms) 3498 3504 9 2.9 349.8 0.3X
-timestamp + interval(m) 1827 1831 6 5.5 182.7 0.5X
-timestamp + interval(m, d) 1902 1905 3 5.3 190.2 0.5X
-timestamp + interval(m, d, ms) 1749 1750 1 5.7 174.9 0.5X
-timestamp - interval(m) 1494 1496 2 6.7 149.4 0.6X
-timestamp - interval(m, d) 1579 1580 2 6.3 157.9 0.6X
-timestamp - interval(m, d, ms) 1737 1740 5 5.8 173.7 0.5X
+date + interval(m) 886 903 16 11.3 88.6 1.0X
+date + interval(m, d) 860 869 8 11.6 86.0 1.0X
+date + interval(m, d, ms) 3589 3600 16 2.8 358.9 0.2X
+date - interval(m) 835 837 2 12.0 83.5 1.1X
+date - interval(m, d) 896 900 5 11.2 89.6 1.0X
+date - interval(m, d, ms) 3614 3618 6 2.8 361.4 0.2X
+timestamp + interval(m) 1867 1883 23 5.4 186.7 0.5X
+timestamp + interval(m, d) 1922 1923 1 5.2 192.2 0.5X
+timestamp + interval(m, d, ms) 1711 1712 1 5.8 171.1 0.5X
+timestamp - interval(m) 1515 1516 2 6.6 151.5 0.6X
+timestamp - interval(m, d) 1561 1562 1 6.4 156.1 0.6X
+timestamp - interval(m, d, ms) 1704 1708 6 5.9 170.4 0.5X
================================================================================================
Extract components
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
cast to timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-cast to timestamp wholestage off 200 204 5 50.0 20.0 1.0X
-cast to timestamp wholestage on 208 214 4 48.2 20.8 1.0X
+cast to timestamp wholestage off 198 203 7 50.6 19.8 1.0X
+cast to timestamp wholestage on 213 218 3 46.9 21.3 0.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
year of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-year of timestamp wholestage off 641 645 6 15.6 64.1 1.0X
-year of timestamp wholestage on 630 635 7 15.9 63.0 1.0X
+year of timestamp wholestage off 628 630 3 15.9 62.8 1.0X
+year of timestamp wholestage on 633 639 8 15.8 63.3 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
quarter of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-quarter of timestamp wholestage off 674 677 5 14.8 67.4 1.0X
-quarter of timestamp wholestage on 673 676 2 14.9 67.3 1.0X
+quarter of timestamp wholestage off 673 674 1 14.9 67.3 1.0X
+quarter of timestamp wholestage on 669 675 6 14.9 66.9 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
month of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-month of timestamp wholestage off 650 655 8 15.4 65.0 1.0X
-month of timestamp wholestage on 637 641 4 15.7 63.7 1.0X
+month of timestamp wholestage off 647 648 1 15.5 64.7 1.0X
+month of timestamp wholestage on 646 650 3 15.5 64.6 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
weekofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-weekofyear of timestamp wholestage off 1006 1012 8 9.9 100.6 1.0X
-weekofyear of timestamp wholestage on 1054 1057 3 9.5 105.4 1.0X
+weekofyear of timestamp wholestage off 1037 1039 2 9.6 103.7 1.0X
+weekofyear of timestamp wholestage on 1120 1124 4 8.9 112.0 0.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
day of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-day of timestamp wholestage off 660 661 0 15.1 66.0 1.0X
-day of timestamp wholestage on 637 643 5 15.7 63.7 1.0X
+day of timestamp wholestage off 658 658 1 15.2 65.8 1.0X
+day of timestamp wholestage on 661 666 7 15.1 66.1 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
dayofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-dayofyear of timestamp wholestage off 697 699 3 14.4 69.7 1.0X
-dayofyear of timestamp wholestage on 689 693 3 14.5 68.9 1.0X
+dayofyear of timestamp wholestage off 679 680 3 14.7 67.9 1.0X
+dayofyear of timestamp wholestage on 683 693 12 14.6 68.3 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
dayofmonth of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-dayofmonth of timestamp wholestage off 684 687 5 14.6 68.4 1.0X
-dayofmonth of timestamp wholestage on 640 643 3 15.6 64.0 1.1X
+dayofmonth of timestamp wholestage off 673 678 7 14.9 67.3 1.0X
+dayofmonth of timestamp wholestage on 655 658 4 15.3 65.5 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
dayofweek of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-dayofweek of timestamp wholestage off 820 826 8 12.2 82.0 1.0X
-dayofweek of timestamp wholestage on 810 816 4 12.3 81.0 1.0X
+dayofweek of timestamp wholestage off 804 808 6 12.4 80.4 1.0X
+dayofweek of timestamp wholestage on 813 818 5 12.3 81.3 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
weekday of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-weekday of timestamp wholestage off 765 766 1 13.1 76.5 1.0X
-weekday of timestamp wholestage on 741 745 4 13.5 74.1 1.0X
+weekday of timestamp wholestage off 739 740 2 13.5 73.9 1.0X
+weekday of timestamp wholestage on 749 752 4 13.4 74.9 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
hour of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-hour of timestamp wholestage off 555 556 1 18.0 55.5 1.0X
-hour of timestamp wholestage on 573 576 4 17.5 57.3 1.0X
+hour of timestamp wholestage off 543 544 2 18.4 54.3 1.0X
+hour of timestamp wholestage on 558 570 22 17.9 55.8 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
minute of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-minute of timestamp wholestage off 559 559 1 17.9 55.9 1.0X
-minute of timestamp wholestage on 551 554 4 18.1 55.1 1.0X
+minute of timestamp wholestage off 544 546 3 18.4 54.4 1.0X
+minute of timestamp wholestage on 561 563 2 17.8 56.1 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
second of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-second of timestamp wholestage off 556 558 3 18.0 55.6 1.0X
-second of timestamp wholestage on 552 556 3 18.1 55.2 1.0X
+second of timestamp wholestage off 544 566 31 18.4 54.4 1.0X
+second of timestamp wholestage on 561 563 2 17.8 56.1 1.0X
================================================================================================
Current date and time
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
current_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-current_date wholestage off 196 197 3 51.1 19.6 1.0X
-current_date wholestage on 208 211 4 48.1 20.8 0.9X
+current_date wholestage off 176 176 1 56.8 17.6 1.0X
+current_date wholestage on 215 219 4 46.5 21.5 0.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
current_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-current_timestamp wholestage off 188 190 2 53.1 18.8 1.0X
-current_timestamp wholestage on 218 237 28 45.9 21.8 0.9X
+current_timestamp wholestage off 182 186 5 54.9 18.2 1.0X
+current_timestamp wholestage on 230 242 14 43.5 23.0 0.8X
================================================================================================
Date arithmetic
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
cast to date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-cast to date wholestage off 619 626 10 16.2 61.9 1.0X
-cast to date wholestage on 621 626 5 16.1 62.1 1.0X
+cast to date wholestage off 614 615 1 16.3 61.4 1.0X
+cast to date wholestage on 633 637 2 15.8 63.3 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
last_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-last_day wholestage off 684 685 2 14.6 68.4 1.0X
-last_day wholestage on 664 670 4 15.0 66.4 1.0X
+last_day wholestage off 673 674 2 14.9 67.3 1.0X
+last_day wholestage on 682 687 6 14.7 68.2 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
next_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-next_day wholestage off 653 656 5 15.3 65.3 1.0X
-next_day wholestage on 651 654 4 15.4 65.1 1.0X
+next_day wholestage off 637 647 15 15.7 63.7 1.0X
+next_day wholestage on 664 666 1 15.1 66.4 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_add: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_add wholestage off 590 592 2 16.9 59.0 1.0X
-date_add wholestage on 587 591 4 17.0 58.7 1.0X
+date_add wholestage off 590 591 2 17.0 59.0 1.0X
+date_add wholestage on 593 595 2 16.9 59.3 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_sub: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_sub wholestage off 599 599 1 16.7 59.9 1.0X
-date_sub wholestage on 588 590 1 17.0 58.8 1.0X
+date_sub wholestage off 585 587 3 17.1 58.5 1.0X
+date_sub wholestage on 590 595 5 16.9 59.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
add_months: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-add_months wholestage off 836 840 6 12.0 83.6 1.0X
-add_months wholestage on 812 815 4 12.3 81.2 1.0X
+add_months wholestage off 812 815 4 12.3 81.2 1.0X
+add_months wholestage on 823 824 1 12.1 82.3 1.0X
================================================================================================
Formatting dates
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
format date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-format date wholestage off 3197 3201 6 3.1 319.7 1.0X
-format date wholestage on 3169 3195 18 3.2 316.9 1.0X
+format date wholestage off 3196 3208 17 3.1 319.6 1.0X
+format date wholestage on 3256 3281 45 3.1 325.6 1.0X
================================================================================================
Formatting timestamps
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
from_unixtime: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-from_unixtime wholestage off 2512 2522 14 4.0 251.2 1.0X
-from_unixtime wholestage on 2577 2590 8 3.9 257.7 1.0X
+from_unixtime wholestage off 2538 2542 6 3.9 253.8 1.0X
+from_unixtime wholestage on 2543 2559 16 3.9 254.3 1.0X
================================================================================================
Convert timestamps
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
from_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-from_utc_timestamp wholestage off 673 674 1 14.9 67.3 1.0X
-from_utc_timestamp wholestage on 779 785 5 12.8 77.9 0.9X
+from_utc_timestamp wholestage off 704 705 2 14.2 70.4 1.0X
+from_utc_timestamp wholestage on 776 777 1 12.9 77.6 0.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
to_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-to_utc_timestamp wholestage off 794 795 0 12.6 79.4 1.0X
-to_utc_timestamp wholestage on 837 843 7 12.0 83.7 0.9X
+to_utc_timestamp wholestage off 774 774 1 12.9 77.4 1.0X
+to_utc_timestamp wholestage on 827 830 3 12.1 82.7 0.9X
================================================================================================
Intervals
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
cast interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-cast interval wholestage off 226 227 1 44.3 22.6 1.0X
-cast interval wholestage on 217 218 3 46.2 21.7 1.0X
+cast interval wholestage off 228 234 9 43.9 22.8 1.0X
+cast interval wholestage on 218 220 3 45.8 21.8 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
datediff: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-datediff wholestage off 996 1000 6 10.0 99.6 1.0X
-datediff wholestage on 1042 1046 6 9.6 104.2 1.0X
+datediff wholestage off 984 984 0 10.2 98.4 1.0X
+datediff wholestage on 1038 1039 1 9.6 103.8 0.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
months_between: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-months_between wholestage off 3012 3021 14 3.3 301.2 1.0X
-months_between wholestage on 3080 3088 12 3.2 308.0 1.0X
+months_between wholestage off 3204 3208 6 3.1 320.4 1.0X
+months_between wholestage on 3175 3182 7 3.1 317.5 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
window: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-window wholestage off 409 416 10 2.4 408.5 1.0X
-window wholestage on 654 692 24 1.5 654.2 0.6X
+window wholestage off 401 408 10 2.5 400.6 1.0X
+window wholestage on 613 642 18 1.6 612.9 0.7X
================================================================================================
Truncation
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_trunc YEAR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_trunc YEAR wholestage off 1544 1545 0 6.5 154.4 1.0X
-date_trunc YEAR wholestage on 1535 1539 4 6.5 153.5 1.0X
+date_trunc YEAR wholestage off 1674 1677 4 6.0 167.4 1.0X
+date_trunc YEAR wholestage on 1643 1646 3 6.1 164.3 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_trunc YYYY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_trunc YYYY wholestage off 1541 1547 9 6.5 154.1 1.0X
-date_trunc YYYY wholestage on 1534 1536 2 6.5 153.4 1.0X
+date_trunc YYYY wholestage off 1675 1676 3 6.0 167.5 1.0X
+date_trunc YYYY wholestage on 1642 1646 4 6.1 164.2 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_trunc YY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_trunc YY wholestage off 1544 1545 0 6.5 154.4 1.0X
-date_trunc YY wholestage on 1534 1540 6 6.5 153.4 1.0X
+date_trunc YY wholestage off 1677 1677 1 6.0 167.7 1.0X
+date_trunc YY wholestage on 1642 1645 4 6.1 164.2 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_trunc MON: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_trunc MON wholestage off 1592 1593 1 6.3 159.2 1.0X
-date_trunc MON wholestage on 1805 1816 9 5.5 180.5 0.9X
+date_trunc MON wholestage off 1700 1701 1 5.9 170.0 1.0X
+date_trunc MON wholestage on 1634 1636 2 6.1 163.4 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_trunc MONTH: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_trunc MONTH wholestage off 1591 1592 2 6.3 159.1 1.0X
-date_trunc MONTH wholestage on 1805 1813 7 5.5 180.5 0.9X
+date_trunc MONTH wholestage off 1705 1706 2 5.9 170.5 1.0X
+date_trunc MONTH wholestage on 1633 1636 3 6.1 163.3 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_trunc MM: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_trunc MM wholestage off 1587 1587 1 6.3 158.7 1.0X
-date_trunc MM wholestage on 1804 1812 5 5.5 180.4 0.9X
+date_trunc MM wholestage off 1702 1703 2 5.9 170.2 1.0X
+date_trunc MM wholestage on 1634 1636 3 6.1 163.4 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_trunc DAY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_trunc DAY wholestage off 1178 1179 1 8.5 117.8 1.0X
-date_trunc DAY wholestage on 1157 1160 2 8.6 115.7 1.0X
+date_trunc DAY wholestage off 1160 1162 3 8.6 116.0 1.0X
+date_trunc DAY wholestage on 1157 1159 1 8.6 115.7 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_trunc DD: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_trunc DD wholestage off 1183 1198 21 8.5 118.3 1.0X
-date_trunc DD wholestage on 1157 1167 19 8.6 115.7 1.0X
+date_trunc DD wholestage off 1187 1195 11 8.4 118.7 1.0X
+date_trunc DD wholestage on 1150 1153 3 8.7 115.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_trunc HOUR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_trunc HOUR wholestage off 1216 1223 9 8.2 121.6 1.0X
-date_trunc HOUR wholestage on 1236 1239 2 8.1 123.6 1.0X
+date_trunc HOUR wholestage off 1213 1215 3 8.2 121.3 1.0X
+date_trunc HOUR wholestage on 1230 1232 3 8.1 123.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_trunc MINUTE: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_trunc MINUTE wholestage off 1201 1202 1 8.3 120.1 1.0X
-date_trunc MINUTE wholestage on 1172 1174 2 8.5 117.2 1.0X
+date_trunc MINUTE wholestage off 1196 1197 2 8.4 119.6 1.0X
+date_trunc MINUTE wholestage on 1190 1192 2 8.4 119.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_trunc SECOND: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_trunc SECOND wholestage off 307 313 8 32.5 30.7 1.0X
-date_trunc SECOND wholestage on 288 293 5 34.7 28.8 1.1X
+date_trunc SECOND wholestage off 290 294 5 34.4 29.0 1.0X
+date_trunc SECOND wholestage on 264 268 4 37.8 26.4 1.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_trunc WEEK: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_trunc WEEK wholestage off 1484 1486 4 6.7 148.4 1.0X
-date_trunc WEEK wholestage on 1458 1462 4 6.9 145.8 1.0X
+date_trunc WEEK wholestage off 1677 1679 2 6.0 167.7 1.0X
+date_trunc WEEK wholestage on 1548 1550 2 6.5 154.8 1.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_trunc QUARTER: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_trunc QUARTER wholestage off 1859 1860 1 5.4 185.9 1.0X
-date_trunc QUARTER wholestage on 1892 1897 4 5.3 189.2 1.0X
+date_trunc QUARTER wholestage off 2013 2019 9 5.0 201.3 1.0X
+date_trunc QUARTER wholestage on 1933 1935 3 5.2 193.3 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
trunc year: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-trunc year wholestage off 804 805 1 12.4 80.4 1.0X
-trunc year wholestage on 780 785 5 12.8 78.0 1.0X
+trunc year wholestage off 792 793 1 12.6 79.2 1.0X
+trunc year wholestage on 765 770 3 13.1 76.5 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
trunc yyyy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-trunc yyyy wholestage off 800 800 1 12.5 80.0 1.0X
-trunc yyyy wholestage on 779 782 2 12.8 77.9 1.0X
+trunc yyyy wholestage off 793 793 1 12.6 79.3 1.0X
+trunc yyyy wholestage on 762 765 2 13.1 76.2 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
trunc yy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-trunc yy wholestage off 800 802 2 12.5 80.0 1.0X
-trunc yy wholestage on 780 782 2 12.8 78.0 1.0X
+trunc yy wholestage off 793 793 1 12.6 79.3 1.0X
+trunc yy wholestage on 763 765 2 13.1 76.3 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
trunc mon: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-trunc mon wholestage off 771 771 0 13.0 77.1 1.0X
-trunc mon wholestage on 752 756 3 13.3 75.2 1.0X
+trunc mon wholestage off 747 749 2 13.4 74.7 1.0X
+trunc mon wholestage on 731 733 2 13.7 73.1 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
trunc month: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-trunc month wholestage off 768 769 2 13.0 76.8 1.0X
-trunc month wholestage on 754 757 4 13.3 75.4 1.0X
+trunc month wholestage off 748 748 0 13.4 74.8 1.0X
+trunc month wholestage on 731 732 1 13.7 73.1 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
trunc mm: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-trunc mm wholestage off 768 770 3 13.0 76.8 1.0X
-trunc mm wholestage on 757 767 10 13.2 75.7 1.0X
+trunc mm wholestage off 749 749 1 13.4 74.9 1.0X
+trunc mm wholestage on 731 736 3 13.7 73.1 1.0X
================================================================================================
Parsing
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
to timestamp str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-to timestamp str wholestage off 106 106 0 9.5 105.7 1.0X
-to timestamp str wholestage on 100 103 4 10.0 100.3 1.1X
+to timestamp str wholestage off 104 106 2 9.6 104.2 1.0X
+to timestamp str wholestage on 97 100 3 10.3 97.4 1.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
to_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-to_timestamp wholestage off 663 670 9 1.5 663.0 1.0X
-to_timestamp wholestage on 665 666 1 1.5 665.0 1.0X
+to_timestamp wholestage off 667 677 14 1.5 667.3 1.0X
+to_timestamp wholestage on 666 668 1 1.5 666.3 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
to_unix_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-to_unix_timestamp wholestage off 673 680 10 1.5 672.7 1.0X
-to_unix_timestamp wholestage on 675 676 1 1.5 674.7 1.0X
+to_unix_timestamp wholestage off 678 680 2 1.5 678.4 1.0X
+to_unix_timestamp wholestage on 667 669 3 1.5 666.9 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
to date str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-to date str wholestage off 134 135 1 7.5 133.6 1.0X
-to date str wholestage on 133 135 2 7.5 132.7 1.0X
+to date str wholestage off 137 138 2 7.3 136.5 1.0X
+to date str wholestage on 129 131 2 7.8 128.9 1.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
to_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-to_date wholestage off 650 652 2 1.5 650.3 1.0X
-to_date wholestage on 643 647 4 1.6 642.9 1.0X
+to_date wholestage off 693 694 1 1.4 693.3 1.0X
+to_date wholestage on 690 693 2 1.4 690.1 1.0X
================================================================================================
Conversion from/to external types
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
To/from Java's date-time: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-From java.sql.Date 290 292 2 17.3 57.9 1.0X
-From java.time.LocalDate 231 233 2 21.7 46.2 1.3X
-Collect java.sql.Date 1074 1256 160 4.7 214.7 0.3X
-Collect java.time.LocalDate 907 988 138 5.5 181.4 0.3X
-From java.sql.Timestamp 253 256 4 19.7 50.7 1.1X
-From java.time.Instant 204 216 12 24.6 40.7 1.4X
-Collect longs 802 980 168 6.2 160.3 0.4X
-Collect java.sql.Timestamp 1147 1173 32 4.4 229.5 0.3X
-Collect java.time.Instant 1064 1093 46 4.7 212.8 0.3X
-java.sql.Date to Hive string 4131 4176 54 1.2 826.2 0.1X
-java.time.LocalDate to Hive string 3118 3176 57 1.6 623.5 0.1X
-java.sql.Timestamp to Hive string 6324 6506 159 0.8 1264.8 0.0X
-java.time.Instant to Hive string 4122 4159 33 1.2 824.3 0.1X
+From java.sql.Date 270 271 1 18.5 54.0 1.0X
+From java.time.LocalDate 222 223 0 22.5 44.5 1.2X
+Collect java.sql.Date 1212 1267 83 4.1 242.3 0.2X
+Collect java.time.LocalDate 848 981 212 5.9 169.6 0.3X
+From java.sql.Timestamp 201 202 1 24.8 40.3 1.3X
+From java.time.Instant 172 174 2 29.1 34.4 1.6X
+Collect longs 900 1019 150 5.6 179.9 0.3X
+Collect java.sql.Timestamp 1137 1235 91 4.4 227.4 0.2X
+Collect java.time.Instant 799 1074 285 6.3 159.9 0.3X
+java.sql.Date to Hive string 4066 4209 128 1.2 813.2 0.1X
+java.time.LocalDate to Hive string 3288 3408 146 1.5 657.6 0.1X
+java.sql.Timestamp to Hive string 6486 6588 90 0.8 1297.2 0.0X
+java.time.Instant to Hive string 4388 4434 74 1.1 877.5 0.1X
diff --git a/sql/core/benchmarks/DateTimeBenchmark-results.txt b/sql/core/benchmarks/DateTimeBenchmark-results.txt
index dcb4e3c45e26a..34bd57334d646 100644
--- a/sql/core/benchmarks/DateTimeBenchmark-results.txt
+++ b/sql/core/benchmarks/DateTimeBenchmark-results.txt
@@ -2,460 +2,460 @@
datetime +/- interval
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
datetime +/- interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date + interval(m) 1015 1028 19 9.9 101.5 1.0X
-date + interval(m, d) 988 993 7 10.1 98.8 1.0X
-date + interval(m, d, ms) 3626 3649 32 2.8 362.6 0.3X
-date - interval(m) 981 993 15 10.2 98.1 1.0X
-date - interval(m, d) 1009 1020 17 9.9 100.9 1.0X
-date - interval(m, d, ms) 3654 3658 6 2.7 365.4 0.3X
-timestamp + interval(m) 1782 1786 6 5.6 178.2 0.6X
-timestamp + interval(m, d) 1826 1835 13 5.5 182.6 0.6X
-timestamp + interval(m, d, ms) 1970 1971 2 5.1 197.0 0.5X
-timestamp - interval(m) 1717 1731 19 5.8 171.7 0.6X
-timestamp - interval(m, d) 1791 1793 3 5.6 179.1 0.6X
-timestamp - interval(m, d, ms) 1962 1965 4 5.1 196.2 0.5X
+date + interval(m) 976 985 8 10.2 97.6 1.0X
+date + interval(m, d) 995 1005 10 10.1 99.5 1.0X
+date + interval(m, d, ms) 3730 3736 8 2.7 373.0 0.3X
+date - interval(m) 966 970 4 10.4 96.6 1.0X
+date - interval(m, d) 978 994 26 10.2 97.8 1.0X
+date - interval(m, d, ms) 3761 3773 17 2.7 376.1 0.3X
+timestamp + interval(m) 1897 1919 30 5.3 189.7 0.5X
+timestamp + interval(m, d) 1952 1954 4 5.1 195.2 0.5X
+timestamp + interval(m, d, ms) 2181 2182 3 4.6 218.1 0.4X
+timestamp - interval(m) 1946 1946 1 5.1 194.6 0.5X
+timestamp - interval(m, d) 2018 2022 5 5.0 201.8 0.5X
+timestamp - interval(m, d, ms) 2181 2185 6 4.6 218.1 0.4X
================================================================================================
Extract components
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
cast to timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-cast to timestamp wholestage off 193 198 6 51.7 19.3 1.0X
-cast to timestamp wholestage on 209 217 9 47.8 20.9 0.9X
+cast to timestamp wholestage off 195 198 4 51.3 19.5 1.0X
+cast to timestamp wholestage on 208 215 7 48.0 20.8 0.9X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
year of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-year of timestamp wholestage off 775 778 5 12.9 77.5 1.0X
-year of timestamp wholestage on 772 777 5 13.0 77.2 1.0X
+year of timestamp wholestage off 775 783 11 12.9 77.5 1.0X
+year of timestamp wholestage on 775 780 5 12.9 77.5 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
quarter of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-quarter of timestamp wholestage off 791 793 2 12.6 79.1 1.0X
-quarter of timestamp wholestage on 782 788 7 12.8 78.2 1.0X
+quarter of timestamp wholestage off 793 794 2 12.6 79.3 1.0X
+quarter of timestamp wholestage on 803 807 5 12.5 80.3 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
month of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-month of timestamp wholestage off 769 770 1 13.0 76.9 1.0X
-month of timestamp wholestage on 771 775 3 13.0 77.1 1.0X
+month of timestamp wholestage off 771 773 3 13.0 77.1 1.0X
+month of timestamp wholestage on 780 784 3 12.8 78.0 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
weekofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-weekofyear of timestamp wholestage off 1029 1032 5 9.7 102.9 1.0X
-weekofyear of timestamp wholestage on 1207 1210 4 8.3 120.7 0.9X
+weekofyear of timestamp wholestage off 1135 1141 8 8.8 113.5 1.0X
+weekofyear of timestamp wholestage on 1240 1242 1 8.1 124.0 0.9X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
day of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-day of timestamp wholestage off 780 782 2 12.8 78.0 1.0X
-day of timestamp wholestage on 779 786 11 12.8 77.9 1.0X
+day of timestamp wholestage off 766 775 13 13.1 76.6 1.0X
+day of timestamp wholestage on 778 790 16 12.8 77.8 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
dayofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-dayofyear of timestamp wholestage off 805 807 2 12.4 80.5 1.0X
-dayofyear of timestamp wholestage on 816 820 3 12.3 81.6 1.0X
+dayofyear of timestamp wholestage off 806 807 2 12.4 80.6 1.0X
+dayofyear of timestamp wholestage on 819 823 6 12.2 81.9 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
dayofmonth of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-dayofmonth of timestamp wholestage off 781 783 3 12.8 78.1 1.0X
-dayofmonth of timestamp wholestage on 773 777 3 12.9 77.3 1.0X
+dayofmonth of timestamp wholestage off 780 786 8 12.8 78.0 1.0X
+dayofmonth of timestamp wholestage on 782 787 7 12.8 78.2 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
dayofweek of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-dayofweek of timestamp wholestage off 929 930 2 10.8 92.9 1.0X
-dayofweek of timestamp wholestage on 912 917 5 11.0 91.2 1.0X
+dayofweek of timestamp wholestage off 914 916 2 10.9 91.4 1.0X
+dayofweek of timestamp wholestage on 924 930 7 10.8 92.4 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
weekday of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-weekday of timestamp wholestage off 896 896 0 11.2 89.6 1.0X
-weekday of timestamp wholestage on 871 871 1 11.5 87.1 1.0X
+weekday of timestamp wholestage off 877 880 5 11.4 87.7 1.0X
+weekday of timestamp wholestage on 892 894 3 11.2 89.2 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
hour of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-hour of timestamp wholestage off 611 611 0 16.4 61.1 1.0X
-hour of timestamp wholestage on 607 614 5 16.5 60.7 1.0X
+hour of timestamp wholestage off 605 607 2 16.5 60.5 1.0X
+hour of timestamp wholestage on 625 627 2 16.0 62.5 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
minute of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-minute of timestamp wholestage off 606 607 1 16.5 60.6 1.0X
-minute of timestamp wholestage on 606 615 10 16.5 60.6 1.0X
+minute of timestamp wholestage off 604 606 2 16.6 60.4 1.0X
+minute of timestamp wholestage on 625 631 11 16.0 62.5 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
second of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-second of timestamp wholestage off 604 605 2 16.6 60.4 1.0X
-second of timestamp wholestage on 612 616 4 16.3 61.2 1.0X
+second of timestamp wholestage off 605 608 4 16.5 60.5 1.0X
+second of timestamp wholestage on 624 627 3 16.0 62.4 1.0X
================================================================================================
Current date and time
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
current_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-current_date wholestage off 182 184 2 54.9 18.2 1.0X
-current_date wholestage on 208 215 7 48.2 20.8 0.9X
+current_date wholestage off 182 183 1 54.8 18.2 1.0X
+current_date wholestage on 220 228 12 45.5 22.0 0.8X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
current_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-current_timestamp wholestage off 190 192 2 52.6 19.0 1.0X
-current_timestamp wholestage on 225 239 18 44.3 22.5 0.8X
+current_timestamp wholestage off 192 195 4 52.0 19.2 1.0X
+current_timestamp wholestage on 217 249 42 46.1 21.7 0.9X
================================================================================================
Date arithmetic
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
cast to date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-cast to date wholestage off 667 668 3 15.0 66.7 1.0X
-cast to date wholestage on 678 683 6 14.7 67.8 1.0X
+cast to date wholestage off 666 668 2 15.0 66.6 1.0X
+cast to date wholestage on 678 684 5 14.7 67.8 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
last_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-last_day wholestage off 786 786 0 12.7 78.6 1.0X
-last_day wholestage on 777 780 3 12.9 77.7 1.0X
+last_day wholestage off 782 783 1 12.8 78.2 1.0X
+last_day wholestage on 787 789 2 12.7 78.7 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
next_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
next_day wholestage off 699 700 1 14.3 69.9 1.0X
-next_day wholestage on 705 707 2 14.2 70.5 1.0X
+next_day wholestage on 704 708 3 14.2 70.4 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_add: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_add wholestage off 653 655 2 15.3 65.3 1.0X
-date_add wholestage on 647 658 23 15.5 64.7 1.0X
+date_add wholestage off 651 651 1 15.4 65.1 1.0X
+date_add wholestage on 657 669 19 15.2 65.7 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_sub: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_sub wholestage off 658 659 2 15.2 65.8 1.0X
-date_sub wholestage on 652 660 5 15.3 65.2 1.0X
+date_sub wholestage off 650 651 2 15.4 65.0 1.0X
+date_sub wholestage on 658 663 4 15.2 65.8 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
add_months: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-add_months wholestage off 932 932 0 10.7 93.2 1.0X
-add_months wholestage on 925 930 6 10.8 92.5 1.0X
+add_months wholestage off 923 925 2 10.8 92.3 1.0X
+add_months wholestage on 934 939 6 10.7 93.4 1.0X
================================================================================================
Formatting dates
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
format date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-format date wholestage off 3595 3654 83 2.8 359.5 1.0X
-format date wholestage on 3531 3545 9 2.8 353.1 1.0X
+format date wholestage off 3458 3482 34 2.9 345.8 1.0X
+format date wholestage on 3353 3362 10 3.0 335.3 1.0X
================================================================================================
Formatting timestamps
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
from_unixtime: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-from_unixtime wholestage off 3760 3761 2 2.7 376.0 1.0X
-from_unixtime wholestage on 3836 3852 13 2.6 383.6 1.0X
+from_unixtime wholestage off 3487 3514 38 2.9 348.7 1.0X
+from_unixtime wholestage on 3425 3441 18 2.9 342.5 1.0X
================================================================================================
Convert timestamps
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
from_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-from_utc_timestamp wholestage off 743 745 2 13.5 74.3 1.0X
-from_utc_timestamp wholestage on 869 873 4 11.5 86.9 0.9X
+from_utc_timestamp wholestage off 734 736 3 13.6 73.4 1.0X
+from_utc_timestamp wholestage on 841 851 14 11.9 84.1 0.9X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
to_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-to_utc_timestamp wholestage off 1081 1082 2 9.3 108.1 1.0X
-to_utc_timestamp wholestage on 1079 1087 5 9.3 107.9 1.0X
+to_utc_timestamp wholestage off 1115 1115 0 9.0 111.5 1.0X
+to_utc_timestamp wholestage on 1147 1155 13 8.7 114.7 1.0X
================================================================================================
Intervals
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
cast interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-cast interval wholestage off 217 219 3 46.1 21.7 1.0X
-cast interval wholestage on 195 200 5 51.3 19.5 1.1X
+cast interval wholestage off 220 222 2 45.5 22.0 1.0X
+cast interval wholestage on 219 223 3 45.6 21.9 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
datediff: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-datediff wholestage off 1119 1123 6 8.9 111.9 1.0X
-datediff wholestage on 1162 1166 3 8.6 116.2 1.0X
+datediff wholestage off 1116 1117 1 9.0 111.6 1.0X
+datediff wholestage on 1165 1173 7 8.6 116.5 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
months_between: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-months_between wholestage off 3399 3409 14 2.9 339.9 1.0X
-months_between wholestage on 3477 3485 8 2.9 347.7 1.0X
+months_between wholestage off 3358 3359 1 3.0 335.8 1.0X
+months_between wholestage on 3361 3371 18 3.0 336.1 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
window: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-window wholestage off 404 409 8 2.5 403.6 1.0X
-window wholestage on 630 652 20 1.6 629.7 0.6X
+window wholestage off 419 428 13 2.4 419.3 1.0X
+window wholestage on 643 661 16 1.6 642.7 0.7X
================================================================================================
Truncation
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_trunc YEAR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_trunc YEAR wholestage off 1811 1814 4 5.5 181.1 1.0X
-date_trunc YEAR wholestage on 1797 1802 4 5.6 179.7 1.0X
+date_trunc YEAR wholestage off 1752 1763 15 5.7 175.2 1.0X
+date_trunc YEAR wholestage on 1726 1730 4 5.8 172.6 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_trunc YYYY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_trunc YYYY wholestage off 1806 1810 6 5.5 180.6 1.0X
-date_trunc YYYY wholestage on 1798 1809 19 5.6 179.8 1.0X
+date_trunc YYYY wholestage off 1774 1777 3 5.6 177.4 1.0X
+date_trunc YYYY wholestage on 1727 1730 3 5.8 172.7 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_trunc YY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_trunc YY wholestage off 1814 1815 1 5.5 181.4 1.0X
-date_trunc YY wholestage on 1800 1804 3 5.6 180.0 1.0X
+date_trunc YY wholestage off 1756 1764 12 5.7 175.6 1.0X
+date_trunc YY wholestage on 1723 1729 4 5.8 172.3 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_trunc MON: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_trunc MON wholestage off 1843 1846 4 5.4 184.3 1.0X
-date_trunc MON wholestage on 1799 1805 12 5.6 179.9 1.0X
+date_trunc MON wholestage off 1735 1736 1 5.8 173.5 1.0X
+date_trunc MON wholestage on 1758 1760 2 5.7 175.8 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_trunc MONTH: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_trunc MONTH wholestage off 1841 1842 1 5.4 184.1 1.0X
-date_trunc MONTH wholestage on 1795 1800 4 5.6 179.5 1.0X
+date_trunc MONTH wholestage off 1741 1743 3 5.7 174.1 1.0X
+date_trunc MONTH wholestage on 1759 1772 21 5.7 175.9 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_trunc MM: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_trunc MM wholestage off 1838 1842 6 5.4 183.8 1.0X
-date_trunc MM wholestage on 1796 1805 7 5.6 179.6 1.0X
+date_trunc MM wholestage off 1729 1731 3 5.8 172.9 1.0X
+date_trunc MM wholestage on 1758 1762 4 5.7 175.8 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_trunc DAY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_trunc DAY wholestage off 1309 1313 6 7.6 130.9 1.0X
-date_trunc DAY wholestage on 1302 1308 6 7.7 130.2 1.0X
+date_trunc DAY wholestage off 1348 1348 0 7.4 134.8 1.0X
+date_trunc DAY wholestage on 1308 1313 5 7.6 130.8 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_trunc DD: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_trunc DD wholestage off 1315 1316 1 7.6 131.5 1.0X
-date_trunc DD wholestage on 1302 1304 3 7.7 130.2 1.0X
+date_trunc DD wholestage off 1348 1351 4 7.4 134.8 1.0X
+date_trunc DD wholestage on 1307 1308 1 7.7 130.7 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_trunc HOUR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_trunc HOUR wholestage off 1287 1289 2 7.8 128.7 1.0X
-date_trunc HOUR wholestage on 1261 1265 5 7.9 126.1 1.0X
+date_trunc HOUR wholestage off 1327 1334 9 7.5 132.7 1.0X
+date_trunc HOUR wholestage on 1280 1281 2 7.8 128.0 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_trunc MINUTE: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_trunc MINUTE wholestage off 1316 1318 4 7.6 131.6 1.0X
-date_trunc MINUTE wholestage on 1280 1283 2 7.8 128.0 1.0X
+date_trunc MINUTE wholestage off 1348 1349 1 7.4 134.8 1.0X
+date_trunc MINUTE wholestage on 1305 1310 7 7.7 130.5 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_trunc SECOND: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_trunc SECOND wholestage off 315 315 0 31.8 31.5 1.0X
-date_trunc SECOND wholestage on 271 274 2 36.9 27.1 1.2X
+date_trunc SECOND wholestage off 325 326 1 30.7 32.5 1.0X
+date_trunc SECOND wholestage on 276 278 2 36.2 27.6 1.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_trunc WEEK: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_trunc WEEK wholestage off 1687 1690 4 5.9 168.7 1.0X
-date_trunc WEEK wholestage on 1694 1700 4 5.9 169.4 1.0X
+date_trunc WEEK wholestage off 1616 1632 23 6.2 161.6 1.0X
+date_trunc WEEK wholestage on 1625 1629 7 6.2 162.5 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
date_trunc QUARTER: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-date_trunc QUARTER wholestage off 2405 2405 1 4.2 240.5 1.0X
-date_trunc QUARTER wholestage on 2328 2343 25 4.3 232.8 1.0X
+date_trunc QUARTER wholestage off 2073 2073 1 4.8 207.3 1.0X
+date_trunc QUARTER wholestage on 2077 2094 24 4.8 207.7 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
trunc year: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-trunc year wholestage off 919 920 1 10.9 91.9 1.0X
-trunc year wholestage on 892 897 4 11.2 89.2 1.0X
+trunc year wholestage off 933 937 5 10.7 93.3 1.0X
+trunc year wholestage on 898 903 5 11.1 89.8 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
trunc yyyy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-trunc yyyy wholestage off 920 920 0 10.9 92.0 1.0X
-trunc yyyy wholestage on 889 896 7 11.2 88.9 1.0X
+trunc yyyy wholestage off 932 934 3 10.7 93.2 1.0X
+trunc yyyy wholestage on 901 909 10 11.1 90.1 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
trunc yy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-trunc yy wholestage off 918 920 3 10.9 91.8 1.0X
-trunc yy wholestage on 894 896 4 11.2 89.4 1.0X
+trunc yy wholestage off 938 942 6 10.7 93.8 1.0X
+trunc yy wholestage on 897 898 1 11.1 89.7 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
trunc mon: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-trunc mon wholestage off 898 898 0 11.1 89.8 1.0X
-trunc mon wholestage on 862 865 3 11.6 86.2 1.0X
+trunc mon wholestage off 906 913 9 11.0 90.6 1.0X
+trunc mon wholestage on 883 893 15 11.3 88.3 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
trunc month: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-trunc month wholestage off 891 893 3 11.2 89.1 1.0X
-trunc month wholestage on 861 863 1 11.6 86.1 1.0X
+trunc month wholestage off 907 907 1 11.0 90.7 1.0X
+trunc month wholestage on 883 884 1 11.3 88.3 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
trunc mm: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-trunc mm wholestage off 895 896 2 11.2 89.5 1.0X
-trunc mm wholestage on 861 864 4 11.6 86.1 1.0X
+trunc mm wholestage off 904 908 6 11.1 90.4 1.0X
+trunc mm wholestage on 881 893 21 11.4 88.1 1.0X
================================================================================================
Parsing
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
to timestamp str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-to timestamp str wholestage off 115 116 1 8.7 115.2 1.0X
-to timestamp str wholestage on 99 104 6 10.1 99.4 1.2X
+to timestamp str wholestage off 106 107 2 9.4 106.1 1.0X
+to timestamp str wholestage on 99 100 1 10.1 98.8 1.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
to_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-to_timestamp wholestage off 744 745 2 1.3 743.6 1.0X
-to_timestamp wholestage on 745 748 3 1.3 744.7 1.0X
+to_timestamp wholestage off 737 737 0 1.4 736.8 1.0X
+to_timestamp wholestage on 747 748 1 1.3 746.6 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
to_unix_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-to_unix_timestamp wholestage off 748 748 0 1.3 748.2 1.0X
-to_unix_timestamp wholestage on 761 766 9 1.3 760.8 1.0X
+to_unix_timestamp wholestage off 750 762 16 1.3 750.3 1.0X
+to_unix_timestamp wholestage on 758 760 2 1.3 758.0 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
to date str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-to date str wholestage off 135 137 2 7.4 135.4 1.0X
-to date str wholestage on 136 140 5 7.4 135.9 1.0X
+to date str wholestage off 134 136 4 7.5 133.7 1.0X
+to date str wholestage on 130 131 1 7.7 129.6 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
to_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-to_date wholestage off 654 655 2 1.5 653.6 1.0X
-to_date wholestage on 649 652 3 1.5 649.5 1.0X
+to_date wholestage off 647 649 2 1.5 647.3 1.0X
+to_date wholestage on 632 634 1 1.6 632.3 1.0X
================================================================================================
Conversion from/to external types
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
To/from Java's date-time: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-From java.sql.Date 279 285 7 17.9 55.8 1.0X
-From java.time.LocalDate 242 242 1 20.7 48.3 1.2X
-Collect java.sql.Date 1131 1375 220 4.4 226.1 0.2X
-Collect java.time.LocalDate 984 1083 115 5.1 196.8 0.3X
-From java.sql.Timestamp 225 236 13 22.2 45.0 1.2X
-From java.time.Instant 179 182 5 28.0 35.7 1.6X
-Collect longs 809 914 181 6.2 161.8 0.3X
-Collect java.sql.Timestamp 865 1112 222 5.8 173.1 0.3X
-Collect java.time.Instant 834 976 152 6.0 166.7 0.3X
-java.sql.Date to Hive string 3819 3954 176 1.3 763.9 0.1X
-java.time.LocalDate to Hive string 3414 3543 114 1.5 682.8 0.1X
-java.sql.Timestamp to Hive string 6507 6599 134 0.8 1301.4 0.0X
-java.time.Instant to Hive string 5115 5206 124 1.0 1022.9 0.1X
+From java.sql.Date 281 283 2 17.8 56.3 1.0X
+From java.time.LocalDate 242 244 2 20.6 48.5 1.2X
+Collect java.sql.Date 1221 1342 111 4.1 244.1 0.2X
+Collect java.time.LocalDate 964 1080 105 5.2 192.9 0.3X
+From java.sql.Timestamp 227 231 7 22.1 45.3 1.2X
+From java.time.Instant 183 187 5 27.3 36.6 1.5X
+Collect longs 891 983 84 5.6 178.3 0.3X
+Collect java.sql.Timestamp 944 1202 225 5.3 188.8 0.3X
+Collect java.time.Instant 966 1046 126 5.2 193.2 0.3X
+java.sql.Date to Hive string 3882 4034 137 1.3 776.3 0.1X
+java.time.LocalDate to Hive string 3308 3348 67 1.5 661.7 0.1X
+java.sql.Timestamp to Hive string 6361 6505 149 0.8 1272.2 0.0X
+java.time.Instant to Hive string 5138 5301 170 1.0 1027.6 0.1X
diff --git a/sql/core/benchmarks/DateTimeRebaseBenchmark-jdk21-results.txt b/sql/core/benchmarks/DateTimeRebaseBenchmark-jdk21-results.txt
index dd76ea3d6e1d7..ffb629892fed2 100644
--- a/sql/core/benchmarks/DateTimeRebaseBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/DateTimeRebaseBenchmark-jdk21-results.txt
@@ -2,153 +2,153 @@
Rebasing dates/timestamps in Parquet datasource
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Save DATE to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-after 1582, noop 10311 10311 0 9.7 103.1 1.0X
-before 1582, noop 6160 6160 0 16.2 61.6 1.7X
-after 1582, rebase EXCEPTION 18563 18563 0 5.4 185.6 0.6X
-after 1582, rebase LEGACY 18638 18638 0 5.4 186.4 0.6X
-after 1582, rebase CORRECTED 18928 18928 0 5.3 189.3 0.5X
-before 1582, rebase LEGACY 14605 14605 0 6.8 146.1 0.7X
-before 1582, rebase CORRECTED 14392 14392 0 6.9 143.9 0.7X
-
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+after 1582, noop 10212 10212 0 9.8 102.1 1.0X
+before 1582, noop 6105 6105 0 16.4 61.1 1.7X
+after 1582, rebase EXCEPTION 19278 19278 0 5.2 192.8 0.5X
+after 1582, rebase LEGACY 18977 18977 0 5.3 189.8 0.5X
+after 1582, rebase CORRECTED 19081 19081 0 5.2 190.8 0.5X
+before 1582, rebase LEGACY 15225 15225 0 6.6 152.3 0.7X
+before 1582, rebase CORRECTED 15244 15244 0 6.6 152.4 0.7X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Load DATE from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-after 1582, vec off, rebase EXCEPTION 11852 11893 45 8.4 118.5 1.0X
-after 1582, vec off, rebase LEGACY 12029 12065 31 8.3 120.3 1.0X
-after 1582, vec off, rebase CORRECTED 11961 12060 127 8.4 119.6 1.0X
-after 1582, vec on, rebase EXCEPTION 2393 2404 11 41.8 23.9 5.0X
-after 1582, vec on, rebase LEGACY 2502 2516 15 40.0 25.0 4.7X
-after 1582, vec on, rebase CORRECTED 2467 2513 40 40.5 24.7 4.8X
-before 1582, vec off, rebase LEGACY 12216 12263 45 8.2 122.2 1.0X
-before 1582, vec off, rebase CORRECTED 11957 11986 29 8.4 119.6 1.0X
-before 1582, vec on, rebase LEGACY 2851 2880 25 35.1 28.5 4.2X
-before 1582, vec on, rebase CORRECTED 2446 2460 23 40.9 24.5 4.8X
-
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+after 1582, vec off, rebase EXCEPTION 11007 11051 39 9.1 110.1 1.0X
+after 1582, vec off, rebase LEGACY 11022 11031 8 9.1 110.2 1.0X
+after 1582, vec off, rebase CORRECTED 10983 11022 46 9.1 109.8 1.0X
+after 1582, vec on, rebase EXCEPTION 2383 2388 7 42.0 23.8 4.6X
+after 1582, vec on, rebase LEGACY 2466 2472 11 40.5 24.7 4.5X
+after 1582, vec on, rebase CORRECTED 2438 2444 6 41.0 24.4 4.5X
+before 1582, vec off, rebase LEGACY 11465 11546 130 8.7 114.7 1.0X
+before 1582, vec off, rebase CORRECTED 11249 11284 33 8.9 112.5 1.0X
+before 1582, vec on, rebase LEGACY 2769 2786 16 36.1 27.7 4.0X
+before 1582, vec on, rebase CORRECTED 2441 2451 13 41.0 24.4 4.5X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Save TIMESTAMP_INT96 to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-after 1900, noop 2362 2362 0 42.3 23.6 1.0X
+after 1900, noop 2337 2337 0 42.8 23.4 1.0X
before 1900, noop 2361 2361 0 42.4 23.6 1.0X
-after 1900, rebase EXCEPTION 12537 12537 0 8.0 125.4 0.2X
-after 1900, rebase LEGACY 12241 12241 0 8.2 122.4 0.2X
-after 1900, rebase CORRECTED 12272 12272 0 8.1 122.7 0.2X
-before 1900, rebase LEGACY 15654 15654 0 6.4 156.5 0.2X
-before 1900, rebase CORRECTED 13597 13597 0 7.4 136.0 0.2X
+after 1900, rebase EXCEPTION 12186 12186 0 8.2 121.9 0.2X
+after 1900, rebase LEGACY 12195 12195 0 8.2 121.9 0.2X
+after 1900, rebase CORRECTED 12358 12358 0 8.1 123.6 0.2X
+before 1900, rebase LEGACY 14091 14091 0 7.1 140.9 0.2X
+before 1900, rebase CORRECTED 12391 12391 0 8.1 123.9 0.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Load TIMESTAMP_INT96 from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-after 1900, vec off, rebase EXCEPTION 16938 17063 112 5.9 169.4 1.0X
-after 1900, vec off, rebase LEGACY 16645 16650 7 6.0 166.5 1.0X
-after 1900, vec off, rebase CORRECTED 16494 16508 12 6.1 164.9 1.0X
-after 1900, vec on, rebase EXCEPTION 4026 4046 21 24.8 40.3 4.2X
-after 1900, vec on, rebase LEGACY 4046 4055 10 24.7 40.5 4.2X
-after 1900, vec on, rebase CORRECTED 4041 4060 17 24.7 40.4 4.2X
-before 1900, vec off, rebase LEGACY 18686 18818 114 5.4 186.9 0.9X
-before 1900, vec off, rebase CORRECTED 16266 16320 50 6.1 162.7 1.0X
-before 1900, vec on, rebase LEGACY 6178 6194 23 16.2 61.8 2.7X
-before 1900, vec on, rebase CORRECTED 4059 4073 16 24.6 40.6 4.2X
-
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+after 1900, vec off, rebase EXCEPTION 16129 16171 37 6.2 161.3 1.0X
+after 1900, vec off, rebase LEGACY 16787 16840 46 6.0 167.9 1.0X
+after 1900, vec off, rebase CORRECTED 15982 16015 37 6.3 159.8 1.0X
+after 1900, vec on, rebase EXCEPTION 4027 4052 33 24.8 40.3 4.0X
+after 1900, vec on, rebase LEGACY 4079 4086 9 24.5 40.8 4.0X
+after 1900, vec on, rebase CORRECTED 4081 4104 24 24.5 40.8 4.0X
+before 1900, vec off, rebase LEGACY 18237 18291 89 5.5 182.4 0.9X
+before 1900, vec off, rebase CORRECTED 15917 15987 73 6.3 159.2 1.0X
+before 1900, vec on, rebase LEGACY 6133 6141 12 16.3 61.3 2.6X
+before 1900, vec on, rebase CORRECTED 4062 4072 13 24.6 40.6 4.0X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Save TIMESTAMP_MICROS to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-after 1900, noop 2404 2404 0 41.6 24.0 1.0X
-before 1900, noop 2349 2349 0 42.6 23.5 1.0X
-after 1900, rebase EXCEPTION 11876 11876 0 8.4 118.8 0.2X
-after 1900, rebase LEGACY 11243 11243 0 8.9 112.4 0.2X
-after 1900, rebase CORRECTED 11348 11348 0 8.8 113.5 0.2X
-before 1900, rebase LEGACY 13254 13254 0 7.5 132.5 0.2X
-before 1900, rebase CORRECTED 11536 11536 0 8.7 115.4 0.2X
-
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+after 1900, noop 2548 2548 0 39.2 25.5 1.0X
+before 1900, noop 2352 2352 0 42.5 23.5 1.1X
+after 1900, rebase EXCEPTION 11992 11992 0 8.3 119.9 0.2X
+after 1900, rebase LEGACY 11708 11708 0 8.5 117.1 0.2X
+after 1900, rebase CORRECTED 11784 11784 0 8.5 117.8 0.2X
+before 1900, rebase LEGACY 13537 13537 0 7.4 135.4 0.2X
+before 1900, rebase CORRECTED 11748 11748 0 8.5 117.5 0.2X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Load TIMESTAMP_MICROS from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-after 1900, vec off, rebase EXCEPTION 14674 14716 54 6.8 146.7 1.0X
-after 1900, vec off, rebase LEGACY 14633 14706 74 6.8 146.3 1.0X
-after 1900, vec off, rebase CORRECTED 14704 14765 74 6.8 147.0 1.0X
-after 1900, vec on, rebase EXCEPTION 3772 3791 31 26.5 37.7 3.9X
-after 1900, vec on, rebase LEGACY 3744 3790 45 26.7 37.4 3.9X
-after 1900, vec on, rebase CORRECTED 3707 3766 75 27.0 37.1 4.0X
-before 1900, vec off, rebase LEGACY 16749 16837 86 6.0 167.5 0.9X
-before 1900, vec off, rebase CORRECTED 14649 14709 80 6.8 146.5 1.0X
-before 1900, vec on, rebase LEGACY 5535 5548 12 18.1 55.3 2.7X
-before 1900, vec on, rebase CORRECTED 3721 3747 37 26.9 37.2 3.9X
-
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+after 1900, vec off, rebase EXCEPTION 14859 14953 87 6.7 148.6 1.0X
+after 1900, vec off, rebase LEGACY 14898 14952 56 6.7 149.0 1.0X
+after 1900, vec off, rebase CORRECTED 14825 14868 38 6.7 148.2 1.0X
+after 1900, vec on, rebase EXCEPTION 3771 3783 10 26.5 37.7 3.9X
+after 1900, vec on, rebase LEGACY 3768 3800 35 26.5 37.7 3.9X
+after 1900, vec on, rebase CORRECTED 3747 3759 10 26.7 37.5 4.0X
+before 1900, vec off, rebase LEGACY 17281 17347 71 5.8 172.8 0.9X
+before 1900, vec off, rebase CORRECTED 14931 14986 72 6.7 149.3 1.0X
+before 1900, vec on, rebase LEGACY 5572 5609 38 17.9 55.7 2.7X
+before 1900, vec on, rebase CORRECTED 3710 3757 50 27.0 37.1 4.0X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Save TIMESTAMP_MILLIS to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-after 1900, noop 2380 2380 0 42.0 23.8 1.0X
-before 1900, noop 2375 2375 0 42.1 23.7 1.0X
-after 1900, rebase EXCEPTION 11461 11461 0 8.7 114.6 0.2X
-after 1900, rebase LEGACY 11387 11387 0 8.8 113.9 0.2X
-after 1900, rebase CORRECTED 11302 11302 0 8.8 113.0 0.2X
-before 1900, rebase LEGACY 14100 14100 0 7.1 141.0 0.2X
-before 1900, rebase CORRECTED 11411 11411 0 8.8 114.1 0.2X
-
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+after 1900, noop 2359 2359 0 42.4 23.6 1.0X
+before 1900, noop 2433 2433 0 41.1 24.3 1.0X
+after 1900, rebase EXCEPTION 10998 10998 0 9.1 110.0 0.2X
+after 1900, rebase LEGACY 11011 11011 0 9.1 110.1 0.2X
+after 1900, rebase CORRECTED 11328 11328 0 8.8 113.3 0.2X
+before 1900, rebase LEGACY 12980 12980 0 7.7 129.8 0.2X
+before 1900, rebase CORRECTED 10966 10966 0 9.1 109.7 0.2X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Load TIMESTAMP_MILLIS from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-after 1900, vec off, rebase EXCEPTION 14312 14337 24 7.0 143.1 1.0X
-after 1900, vec off, rebase LEGACY 14327 14406 73 7.0 143.3 1.0X
-after 1900, vec off, rebase CORRECTED 14244 14289 61 7.0 142.4 1.0X
-after 1900, vec on, rebase EXCEPTION 4862 4916 49 20.6 48.6 2.9X
-after 1900, vec on, rebase LEGACY 4469 4498 36 22.4 44.7 3.2X
-after 1900, vec on, rebase CORRECTED 4858 4869 14 20.6 48.6 2.9X
-before 1900, vec off, rebase LEGACY 16578 16652 66 6.0 165.8 0.9X
-before 1900, vec off, rebase CORRECTED 14240 14331 82 7.0 142.4 1.0X
-before 1900, vec on, rebase LEGACY 6015 6044 48 16.6 60.1 2.4X
-before 1900, vec on, rebase CORRECTED 4873 4898 27 20.5 48.7 2.9X
+after 1900, vec off, rebase EXCEPTION 14432 14453 20 6.9 144.3 1.0X
+after 1900, vec off, rebase LEGACY 14708 14761 57 6.8 147.1 1.0X
+after 1900, vec off, rebase CORRECTED 14689 14744 70 6.8 146.9 1.0X
+after 1900, vec on, rebase EXCEPTION 4891 4915 22 20.4 48.9 3.0X
+after 1900, vec on, rebase LEGACY 4353 4392 40 23.0 43.5 3.3X
+after 1900, vec on, rebase CORRECTED 4906 4934 26 20.4 49.1 2.9X
+before 1900, vec off, rebase LEGACY 17095 17242 217 5.8 170.9 0.8X
+before 1900, vec off, rebase CORRECTED 14633 14651 27 6.8 146.3 1.0X
+before 1900, vec on, rebase LEGACY 6068 6069 1 16.5 60.7 2.4X
+before 1900, vec on, rebase CORRECTED 4910 4934 36 20.4 49.1 2.9X
================================================================================================
Rebasing dates/timestamps in ORC datasource
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Save DATE to ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-after 1582, noop 11787 11787 0 8.5 117.9 1.0X
-before 1582, noop 5513 5513 0 18.1 55.1 2.1X
-after 1582 16137 16137 0 6.2 161.4 0.7X
-before 1582 9680 9680 0 10.3 96.8 1.2X
+after 1582, noop 9915 9915 0 10.1 99.2 1.0X
+before 1582, noop 5846 5846 0 17.1 58.5 1.7X
+after 1582 13925 13925 0 7.2 139.2 0.7X
+before 1582 10596 10596 0 9.4 106.0 0.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Load DATE from ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-after 1582, vec off 8458 8500 37 11.8 84.6 1.0X
-after 1582, vec on 2411 2437 23 41.5 24.1 3.5X
-before 1582, vec off 8607 8658 73 11.6 86.1 1.0X
-before 1582, vec on 2557 2586 45 39.1 25.6 3.3X
+after 1582, vec off 8802 8811 11 11.4 88.0 1.0X
+after 1582, vec on 2410 2434 21 41.5 24.1 3.7X
+before 1582, vec off 8919 8930 10 11.2 89.2 1.0X
+before 1582, vec on 2553 2566 21 39.2 25.5 3.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Save TIMESTAMP to ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-after 1900, noop 2371 2371 0 42.2 23.7 1.0X
-before 1900, noop 2365 2365 0 42.3 23.6 1.0X
-after 1900 9499 9499 0 10.5 95.0 0.2X
-before 1900 12115 12115 0 8.3 121.1 0.2X
+after 1900, noop 2296 2296 0 43.6 23.0 1.0X
+before 1900, noop 2316 2316 0 43.2 23.2 1.0X
+after 1900 11515 11515 0 8.7 115.1 0.2X
+before 1900 12345 12345 0 8.1 123.5 0.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Load TIMESTAMP from ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-after 1900, vec off 9880 9897 19 10.1 98.8 1.0X
-after 1900, vec on 3891 3908 18 25.7 38.9 2.5X
-before 1900, vec off 11614 11656 42 8.6 116.1 0.9X
-before 1900, vec on 5409 5441 29 18.5 54.1 1.8X
+after 1900, vec off 10546 10548 3 9.5 105.5 1.0X
+after 1900, vec on 4153 4159 10 24.1 41.5 2.5X
+before 1900, vec off 12789 12815 38 7.8 127.9 0.8X
+before 1900, vec on 5646 5648 2 17.7 56.5 1.9X
diff --git a/sql/core/benchmarks/DateTimeRebaseBenchmark-results.txt b/sql/core/benchmarks/DateTimeRebaseBenchmark-results.txt
index f96863864365d..8d9e830e163fe 100644
--- a/sql/core/benchmarks/DateTimeRebaseBenchmark-results.txt
+++ b/sql/core/benchmarks/DateTimeRebaseBenchmark-results.txt
@@ -2,153 +2,153 @@
Rebasing dates/timestamps in Parquet datasource
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Save DATE to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-after 1582, noop 12334 12334 0 8.1 123.3 1.0X
-before 1582, noop 6783 6783 0 14.7 67.8 1.8X
-after 1582, rebase EXCEPTION 21193 21193 0 4.7 211.9 0.6X
-after 1582, rebase LEGACY 21767 21767 0 4.6 217.7 0.6X
-after 1582, rebase CORRECTED 21968 21968 0 4.6 219.7 0.6X
-before 1582, rebase LEGACY 17878 17878 0 5.6 178.8 0.7X
-before 1582, rebase CORRECTED 17372 17372 0 5.8 173.7 0.7X
-
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+after 1582, noop 11639 11639 0 8.6 116.4 1.0X
+before 1582, noop 6668 6668 0 15.0 66.7 1.7X
+after 1582, rebase EXCEPTION 20766 20766 0 4.8 207.7 0.6X
+after 1582, rebase LEGACY 20737 20737 0 4.8 207.4 0.6X
+after 1582, rebase CORRECTED 20877 20877 0 4.8 208.8 0.6X
+before 1582, rebase LEGACY 16082 16082 0 6.2 160.8 0.7X
+before 1582, rebase CORRECTED 16149 16149 0 6.2 161.5 0.7X
+
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Load DATE from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-after 1582, vec off, rebase EXCEPTION 11178 11279 174 8.9 111.8 1.0X
-after 1582, vec off, rebase LEGACY 11579 11620 44 8.6 115.8 1.0X
-after 1582, vec off, rebase CORRECTED 11618 11659 37 8.6 116.2 1.0X
-after 1582, vec on, rebase EXCEPTION 2549 2561 17 39.2 25.5 4.4X
-after 1582, vec on, rebase LEGACY 2558 2573 14 39.1 25.6 4.4X
-after 1582, vec on, rebase CORRECTED 2507 2540 30 39.9 25.1 4.5X
-before 1582, vec off, rebase LEGACY 11480 11530 79 8.7 114.8 1.0X
-before 1582, vec off, rebase CORRECTED 11377 11392 15 8.8 113.8 1.0X
-before 1582, vec on, rebase LEGACY 2850 2866 23 35.1 28.5 3.9X
-before 1582, vec on, rebase CORRECTED 2504 2529 21 39.9 25.0 4.5X
-
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+after 1582, vec off, rebase EXCEPTION 11234 11278 45 8.9 112.3 1.0X
+after 1582, vec off, rebase LEGACY 11417 11445 31 8.8 114.2 1.0X
+after 1582, vec off, rebase CORRECTED 11277 11301 22 8.9 112.8 1.0X
+after 1582, vec on, rebase EXCEPTION 2499 2522 21 40.0 25.0 4.5X
+after 1582, vec on, rebase LEGACY 2529 2557 29 39.5 25.3 4.4X
+after 1582, vec on, rebase CORRECTED 2452 2503 49 40.8 24.5 4.6X
+before 1582, vec off, rebase LEGACY 11636 11638 4 8.6 116.4 1.0X
+before 1582, vec off, rebase CORRECTED 11413 11433 33 8.8 114.1 1.0X
+before 1582, vec on, rebase LEGACY 2856 2942 134 35.0 28.6 3.9X
+before 1582, vec on, rebase CORRECTED 2423 2432 8 41.3 24.2 4.6X
+
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Save TIMESTAMP_INT96 to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-after 1900, noop 2239 2239 0 44.7 22.4 1.0X
-before 1900, noop 2258 2258 0 44.3 22.6 1.0X
-after 1900, rebase EXCEPTION 13699 13699 0 7.3 137.0 0.2X
-after 1900, rebase LEGACY 13692 13692 0 7.3 136.9 0.2X
-after 1900, rebase CORRECTED 13661 13661 0 7.3 136.6 0.2X
-before 1900, rebase LEGACY 15321 15321 0 6.5 153.2 0.1X
-before 1900, rebase CORRECTED 13646 13646 0 7.3 136.5 0.2X
-
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+after 1900, noop 2256 2256 0 44.3 22.6 1.0X
+before 1900, noop 2318 2318 0 43.1 23.2 1.0X
+after 1900, rebase EXCEPTION 13444 13444 0 7.4 134.4 0.2X
+after 1900, rebase LEGACY 13419 13419 0 7.5 134.2 0.2X
+after 1900, rebase CORRECTED 13231 13231 0 7.6 132.3 0.2X
+before 1900, rebase LEGACY 15176 15176 0 6.6 151.8 0.1X
+before 1900, rebase CORRECTED 13494 13494 0 7.4 134.9 0.2X
+
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Load TIMESTAMP_INT96 from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-after 1900, vec off, rebase EXCEPTION 14596 14627 27 6.9 146.0 1.0X
-after 1900, vec off, rebase LEGACY 15157 15202 49 6.6 151.6 1.0X
-after 1900, vec off, rebase CORRECTED 14936 14972 33 6.7 149.4 1.0X
-after 1900, vec on, rebase EXCEPTION 5412 5429 18 18.5 54.1 2.7X
-after 1900, vec on, rebase LEGACY 5372 5427 48 18.6 53.7 2.7X
-after 1900, vec on, rebase CORRECTED 5406 5413 9 18.5 54.1 2.7X
-before 1900, vec off, rebase LEGACY 17556 17637 82 5.7 175.6 0.8X
-before 1900, vec off, rebase CORRECTED 15475 15502 26 6.5 154.7 0.9X
-before 1900, vec on, rebase LEGACY 7054 7080 23 14.2 70.5 2.1X
-before 1900, vec on, rebase CORRECTED 5369 5377 8 18.6 53.7 2.7X
-
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+after 1900, vec off, rebase EXCEPTION 14887 14927 50 6.7 148.9 1.0X
+after 1900, vec off, rebase LEGACY 14789 14811 22 6.8 147.9 1.0X
+after 1900, vec off, rebase CORRECTED 14792 14829 40 6.8 147.9 1.0X
+after 1900, vec on, rebase EXCEPTION 4113 4146 33 24.3 41.1 3.6X
+after 1900, vec on, rebase LEGACY 5453 5483 26 18.3 54.5 2.7X
+after 1900, vec on, rebase CORRECTED 4103 4146 41 24.4 41.0 3.6X
+before 1900, vec off, rebase LEGACY 17155 17183 29 5.8 171.5 0.9X
+before 1900, vec off, rebase CORRECTED 14714 14733 20 6.8 147.1 1.0X
+before 1900, vec on, rebase LEGACY 7105 7129 21 14.1 71.1 2.1X
+before 1900, vec on, rebase CORRECTED 4113 4128 23 24.3 41.1 3.6X
+
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Save TIMESTAMP_MICROS to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-after 1900, noop 2260 2260 0 44.2 22.6 1.0X
-before 1900, noop 2253 2253 0 44.4 22.5 1.0X
-after 1900, rebase EXCEPTION 12513 12513 0 8.0 125.1 0.2X
-after 1900, rebase LEGACY 12690 12690 0 7.9 126.9 0.2X
-after 1900, rebase CORRECTED 13009 13009 0 7.7 130.1 0.2X
-before 1900, rebase LEGACY 14412 14412 0 6.9 144.1 0.2X
-before 1900, rebase CORRECTED 12546 12546 0 8.0 125.5 0.2X
-
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+after 1900, noop 2316 2316 0 43.2 23.2 1.0X
+before 1900, noop 2341 2341 0 42.7 23.4 1.0X
+after 1900, rebase EXCEPTION 12220 12220 0 8.2 122.2 0.2X
+after 1900, rebase LEGACY 12031 12031 0 8.3 120.3 0.2X
+after 1900, rebase CORRECTED 12199 12199 0 8.2 122.0 0.2X
+before 1900, rebase LEGACY 14011 14011 0 7.1 140.1 0.2X
+before 1900, rebase CORRECTED 12239 12239 0 8.2 122.4 0.2X
+
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Load TIMESTAMP_MICROS from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-after 1900, vec off, rebase EXCEPTION 15222 15269 49 6.6 152.2 1.0X
-after 1900, vec off, rebase LEGACY 15154 15189 34 6.6 151.5 1.0X
-after 1900, vec off, rebase CORRECTED 15391 15426 31 6.5 153.9 1.0X
-after 1900, vec on, rebase EXCEPTION 3770 3790 17 26.5 37.7 4.0X
-after 1900, vec on, rebase LEGACY 3814 3869 71 26.2 38.1 4.0X
-after 1900, vec on, rebase CORRECTED 3784 3798 15 26.4 37.8 4.0X
-before 1900, vec off, rebase LEGACY 17591 17636 55 5.7 175.9 0.9X
-before 1900, vec off, rebase CORRECTED 15408 15436 45 6.5 154.1 1.0X
-before 1900, vec on, rebase LEGACY 5604 5629 29 17.8 56.0 2.7X
-before 1900, vec on, rebase CORRECTED 3784 3790 7 26.4 37.8 4.0X
-
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+after 1900, vec off, rebase EXCEPTION 15111 15142 33 6.6 151.1 1.0X
+after 1900, vec off, rebase LEGACY 15282 15396 169 6.5 152.8 1.0X
+after 1900, vec off, rebase CORRECTED 15273 15320 54 6.5 152.7 1.0X
+after 1900, vec on, rebase EXCEPTION 3843 3850 9 26.0 38.4 3.9X
+after 1900, vec on, rebase LEGACY 3862 3882 28 25.9 38.6 3.9X
+after 1900, vec on, rebase CORRECTED 3836 3864 26 26.1 38.4 3.9X
+before 1900, vec off, rebase LEGACY 17575 17590 13 5.7 175.7 0.9X
+before 1900, vec off, rebase CORRECTED 15328 15341 14 6.5 153.3 1.0X
+before 1900, vec on, rebase LEGACY 5662 5674 12 17.7 56.6 2.7X
+before 1900, vec on, rebase CORRECTED 3807 3821 12 26.3 38.1 4.0X
+
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Save TIMESTAMP_MILLIS to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-after 1900, noop 2303 2303 0 43.4 23.0 1.0X
+after 1900, noop 2262 2262 0 44.2 22.6 1.0X
before 1900, noop 2283 2283 0 43.8 22.8 1.0X
-after 1900, rebase EXCEPTION 11819 11819 0 8.5 118.2 0.2X
-after 1900, rebase LEGACY 11878 11878 0 8.4 118.8 0.2X
-after 1900, rebase CORRECTED 11760 11760 0 8.5 117.6 0.2X
-before 1900, rebase LEGACY 13634 13634 0 7.3 136.3 0.2X
-before 1900, rebase CORRECTED 11533 11533 0 8.7 115.3 0.2X
+after 1900, rebase EXCEPTION 11581 11581 0 8.6 115.8 0.2X
+after 1900, rebase LEGACY 11524 11524 0 8.7 115.2 0.2X
+after 1900, rebase CORRECTED 11721 11721 0 8.5 117.2 0.2X
+before 1900, rebase LEGACY 13792 13792 0 7.3 137.9 0.2X
+before 1900, rebase CORRECTED 11664 11664 0 8.6 116.6 0.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Load TIMESTAMP_MILLIS from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-after 1900, vec off, rebase EXCEPTION 15683 15716 56 6.4 156.8 1.0X
-after 1900, vec off, rebase LEGACY 15669 15708 57 6.4 156.7 1.0X
-after 1900, vec off, rebase CORRECTED 15790 15834 71 6.3 157.9 1.0X
-after 1900, vec on, rebase EXCEPTION 3936 3947 12 25.4 39.4 4.0X
-after 1900, vec on, rebase LEGACY 4508 4534 31 22.2 45.1 3.5X
-after 1900, vec on, rebase CORRECTED 4006 4009 4 25.0 40.1 3.9X
-before 1900, vec off, rebase LEGACY 17911 17944 47 5.6 179.1 0.9X
-before 1900, vec off, rebase CORRECTED 15705 15751 48 6.4 157.1 1.0X
-before 1900, vec on, rebase LEGACY 6250 6260 18 16.0 62.5 2.5X
-before 1900, vec on, rebase CORRECTED 3929 3976 42 25.5 39.3 4.0X
+after 1900, vec off, rebase EXCEPTION 15276 15313 38 6.5 152.8 1.0X
+after 1900, vec off, rebase LEGACY 15279 15323 44 6.5 152.8 1.0X
+after 1900, vec off, rebase CORRECTED 15412 15448 31 6.5 154.1 1.0X
+after 1900, vec on, rebase EXCEPTION 4031 4046 21 24.8 40.3 3.8X
+after 1900, vec on, rebase LEGACY 4618 4629 17 21.7 46.2 3.3X
+after 1900, vec on, rebase CORRECTED 4044 4058 13 24.7 40.4 3.8X
+before 1900, vec off, rebase LEGACY 17341 17397 57 5.8 173.4 0.9X
+before 1900, vec off, rebase CORRECTED 15353 15399 45 6.5 153.5 1.0X
+before 1900, vec on, rebase LEGACY 6287 6292 9 15.9 62.9 2.4X
+before 1900, vec on, rebase CORRECTED 4043 4074 28 24.7 40.4 3.8X
================================================================================================
Rebasing dates/timestamps in ORC datasource
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Save DATE to ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-after 1582, noop 12491 12491 0 8.0 124.9 1.0X
-before 1582, noop 6804 6804 0 14.7 68.0 1.8X
-after 1582 16422 16422 0 6.1 164.2 0.8X
-before 1582 11024 11024 0 9.1 110.2 1.1X
+after 1582, noop 11666 11666 0 8.6 116.7 1.0X
+before 1582, noop 6686 6686 0 15.0 66.9 1.7X
+after 1582 15631 15631 0 6.4 156.3 0.7X
+before 1582 10773 10773 0 9.3 107.7 1.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Load DATE from ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-after 1582, vec off 8591 8595 6 11.6 85.9 1.0X
-after 1582, vec on 2555 2578 31 39.1 25.5 3.4X
-before 1582, vec off 8445 8491 52 11.8 84.5 1.0X
-before 1582, vec on 2706 2715 11 37.0 27.1 3.2X
+after 1582, vec off 8409 8415 10 11.9 84.1 1.0X
+after 1582, vec on 2552 2574 30 39.2 25.5 3.3X
+before 1582, vec off 8456 8533 83 11.8 84.6 1.0X
+before 1582, vec on 2623 2638 14 38.1 26.2 3.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Save TIMESTAMP to ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-after 1900, noop 2177 2177 0 45.9 21.8 1.0X
-before 1900, noop 2180 2180 0 45.9 21.8 1.0X
-after 1900 8990 8990 0 11.1 89.9 0.2X
-before 1900 11043 11043 0 9.1 110.4 0.2X
+after 1900, noop 2185 2185 0 45.8 21.9 1.0X
+before 1900, noop 2263 2263 0 44.2 22.6 1.0X
+after 1900 9439 9439 0 10.6 94.4 0.2X
+before 1900 11613 11613 0 8.6 116.1 0.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Load TIMESTAMP from ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-after 1900, vec off 10558 10576 29 9.5 105.6 1.0X
-after 1900, vec on 4070 4109 41 24.6 40.7 2.6X
-before 1900, vec off 11951 11986 41 8.4 119.5 0.9X
-before 1900, vec on 5539 5547 10 18.1 55.4 1.9X
+after 1900, vec off 9771 9811 53 10.2 97.7 1.0X
+after 1900, vec on 4046 4062 14 24.7 40.5 2.4X
+before 1900, vec off 11436 11477 57 8.7 114.4 0.9X
+before 1900, vec on 5690 5701 10 17.6 56.9 1.7X
diff --git a/sql/core/benchmarks/EncodeBenchmark-jdk21-results.txt b/sql/core/benchmarks/EncodeBenchmark-jdk21-results.txt
index 6fa07836f177c..d0b3598712655 100644
--- a/sql/core/benchmarks/EncodeBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/EncodeBenchmark-jdk21-results.txt
@@ -1,8 +1,8 @@
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
encode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-UTF-32 48699 48707 12 0.2 4869.9 1.0X
-UTF-16 60472 60483 15 0.2 6047.2 0.8X
-UTF-8 31641 31688 66 0.3 3164.1 1.5X
+UTF-32 58475 58556 115 0.2 5847.5 1.0X
+UTF-16 60460 60596 193 0.2 6046.0 1.0X
+UTF-8 32910 32911 1 0.3 3291.0 1.8X
diff --git a/sql/core/benchmarks/EncodeBenchmark-results.txt b/sql/core/benchmarks/EncodeBenchmark-results.txt
index 5df736af1bc5e..58424b316eec5 100644
--- a/sql/core/benchmarks/EncodeBenchmark-results.txt
+++ b/sql/core/benchmarks/EncodeBenchmark-results.txt
@@ -1,8 +1,8 @@
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
encode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-UTF-32 33507 33538 44 0.3 3350.7 1.0X
-UTF-16 52085 52137 74 0.2 5208.5 0.6X
-UTF-8 30150 30156 9 0.3 3015.0 1.1X
+UTF-32 56295 56403 153 0.2 5629.5 1.0X
+UTF-16 50644 50653 13 0.2 5064.4 1.1X
+UTF-8 30599 30619 28 0.3 3059.9 1.8X
diff --git a/sql/core/benchmarks/ExternalAppendOnlyUnsafeRowArrayBenchmark-jdk21-results.txt b/sql/core/benchmarks/ExternalAppendOnlyUnsafeRowArrayBenchmark-jdk21-results.txt
index 2aa91c041f04a..8d16f6cb9c5cb 100644
--- a/sql/core/benchmarks/ExternalAppendOnlyUnsafeRowArrayBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/ExternalAppendOnlyUnsafeRowArrayBenchmark-jdk21-results.txt
@@ -2,44 +2,44 @@
WITHOUT SPILL
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Array with 100000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ArrayBuffer 2676 2677 2 38.3 26.1 1.0X
-ExternalAppendOnlyUnsafeRowArray 3586 3598 17 28.6 35.0 0.7X
+ArrayBuffer 2797 2810 17 36.6 27.3 1.0X
+ExternalAppendOnlyUnsafeRowArray 3838 3842 6 26.7 37.5 0.7X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Array with 1000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ArrayBuffer 5541 5543 2 47.3 21.1 1.0X
-ExternalAppendOnlyUnsafeRowArray 12412 12447 50 21.1 47.3 0.4X
+ArrayBuffer 5501 5508 10 47.7 21.0 1.0X
+ExternalAppendOnlyUnsafeRowArray 12829 12879 70 20.4 48.9 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Array with 30000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ArrayBuffer 11108 11111 4 44.2 22.6 1.0X
-ExternalAppendOnlyUnsafeRowArray 18620 18663 61 26.4 37.9 0.6X
+ArrayBuffer 10905 11000 135 45.1 22.2 1.0X
+ExternalAppendOnlyUnsafeRowArray 18327 18373 64 26.8 37.3 0.6X
================================================================================================
WITH SPILL
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Spilling with 1000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-UnsafeExternalSorter 8438 8501 88 31.1 32.2 1.0X
-ExternalAppendOnlyUnsafeRowArray 6672 6673 0 39.3 25.5 1.3X
+UnsafeExternalSorter 8819 8837 27 29.7 33.6 1.0X
+ExternalAppendOnlyUnsafeRowArray 7137 7140 4 36.7 27.2 1.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Spilling with 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-UnsafeExternalSorter 5 5 0 32.8 30.5 1.0X
-ExternalAppendOnlyUnsafeRowArray 4 4 0 38.6 25.9 1.2X
+UnsafeExternalSorter 5 5 0 33.6 29.8 1.0X
+ExternalAppendOnlyUnsafeRowArray 4 4 0 38.3 26.1 1.1X
diff --git a/sql/core/benchmarks/ExternalAppendOnlyUnsafeRowArrayBenchmark-results.txt b/sql/core/benchmarks/ExternalAppendOnlyUnsafeRowArrayBenchmark-results.txt
index a8fc3fc7e0395..6cc5f3af4581d 100644
--- a/sql/core/benchmarks/ExternalAppendOnlyUnsafeRowArrayBenchmark-results.txt
+++ b/sql/core/benchmarks/ExternalAppendOnlyUnsafeRowArrayBenchmark-results.txt
@@ -2,44 +2,44 @@
WITHOUT SPILL
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Array with 100000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ArrayBuffer 2513 2516 3 40.7 24.5 1.0X
-ExternalAppendOnlyUnsafeRowArray 3627 3635 12 28.2 35.4 0.7X
+ArrayBuffer 2536 2538 2 40.4 24.8 1.0X
+ExternalAppendOnlyUnsafeRowArray 3472 3488 22 29.5 33.9 0.7X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Array with 1000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ArrayBuffer 5460 5468 11 48.0 20.8 1.0X
-ExternalAppendOnlyUnsafeRowArray 12333 12352 28 21.3 47.0 0.4X
+ArrayBuffer 5217 5224 10 50.2 19.9 1.0X
+ExternalAppendOnlyUnsafeRowArray 13015 13025 15 20.1 49.6 0.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Array with 30000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-ArrayBuffer 10277 10283 9 47.8 20.9 1.0X
-ExternalAppendOnlyUnsafeRowArray 17958 17977 26 27.4 36.5 0.6X
+ArrayBuffer 9950 9973 33 49.4 20.2 1.0X
+ExternalAppendOnlyUnsafeRowArray 17598 17606 13 27.9 35.8 0.6X
================================================================================================
WITH SPILL
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Spilling with 1000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-UnsafeExternalSorter 8492 8495 4 30.9 32.4 1.0X
-ExternalAppendOnlyUnsafeRowArray 6176 6181 7 42.4 23.6 1.4X
+UnsafeExternalSorter 8485 8532 67 30.9 32.4 1.0X
+ExternalAppendOnlyUnsafeRowArray 6400 6405 7 41.0 24.4 1.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Spilling with 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-UnsafeExternalSorter 5 5 0 34.2 29.2 1.0X
-ExternalAppendOnlyUnsafeRowArray 4 4 0 42.0 23.8 1.2X
+UnsafeExternalSorter 5 5 0 32.5 30.8 1.0X
+ExternalAppendOnlyUnsafeRowArray 4 4 0 38.7 25.8 1.2X
diff --git a/sql/core/benchmarks/ExtractBenchmark-jdk21-results.txt b/sql/core/benchmarks/ExtractBenchmark-jdk21-results.txt
index 84b5eda46cc46..a22cc3228194d 100644
--- a/sql/core/benchmarks/ExtractBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/ExtractBenchmark-jdk21-results.txt
@@ -1,104 +1,104 @@
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Invoke extract for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-cast to timestamp 260 264 4 38.4 26.0 1.0X
-YEAR of timestamp 669 672 3 14.9 66.9 0.4X
-YEAROFWEEK of timestamp 642 643 1 15.6 64.2 0.4X
-QUARTER of timestamp 643 647 3 15.6 64.3 0.4X
-MONTH of timestamp 567 574 7 17.6 56.7 0.5X
-WEEK of timestamp 864 865 2 11.6 86.4 0.3X
-DAY of timestamp 557 562 8 18.0 55.7 0.5X
-DAYOFWEEK of timestamp 742 756 21 13.5 74.2 0.4X
-DOW of timestamp 743 754 18 13.5 74.3 0.4X
-DOW_ISO of timestamp 681 684 3 14.7 68.1 0.4X
-DAYOFWEEK_ISO of timestamp 681 685 7 14.7 68.1 0.4X
-DOY of timestamp 597 603 8 16.7 59.7 0.4X
-HOUR of timestamp 465 468 4 21.5 46.5 0.6X
-MINUTE of timestamp 467 472 6 21.4 46.7 0.6X
-SECOND of timestamp 529 530 2 18.9 52.9 0.5X
+cast to timestamp 255 265 9 39.2 25.5 1.0X
+YEAR of timestamp 693 695 2 14.4 69.3 0.4X
+YEAROFWEEK of timestamp 650 651 2 15.4 65.0 0.4X
+QUARTER of timestamp 649 654 5 15.4 64.9 0.4X
+MONTH of timestamp 571 574 4 17.5 57.1 0.4X
+WEEK of timestamp 892 899 11 11.2 89.2 0.3X
+DAY of timestamp 570 571 1 17.5 57.0 0.4X
+DAYOFWEEK of timestamp 756 758 2 13.2 75.6 0.3X
+DOW of timestamp 755 759 5 13.3 75.5 0.3X
+DOW_ISO of timestamp 694 699 6 14.4 69.4 0.4X
+DAYOFWEEK_ISO of timestamp 697 699 2 14.3 69.7 0.4X
+DOY of timestamp 610 612 2 16.4 61.0 0.4X
+HOUR of timestamp 474 476 4 21.1 47.4 0.5X
+MINUTE of timestamp 472 477 7 21.2 47.2 0.5X
+SECOND of timestamp 538 541 3 18.6 53.8 0.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Invoke date_part for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-cast to timestamp 236 243 8 42.4 23.6 1.0X
-YEAR of timestamp 529 533 6 18.9 52.9 0.4X
-YEAROFWEEK of timestamp 601 609 7 16.6 60.1 0.4X
-QUARTER of timestamp 623 629 8 16.0 62.3 0.4X
-MONTH of timestamp 538 542 3 18.6 53.8 0.4X
-WEEK of timestamp 855 859 4 11.7 85.5 0.3X
-DAY of timestamp 555 557 2 18.0 55.5 0.4X
-DAYOFWEEK of timestamp 737 739 2 13.6 73.7 0.3X
-DOW of timestamp 737 739 2 13.6 73.7 0.3X
-DOW_ISO of timestamp 673 676 3 14.9 67.3 0.4X
-DAYOFWEEK_ISO of timestamp 680 689 11 14.7 68.0 0.3X
-DOY of timestamp 589 591 2 17.0 58.9 0.4X
-HOUR of timestamp 460 468 12 21.7 46.0 0.5X
-MINUTE of timestamp 464 465 2 21.6 46.4 0.5X
-SECOND of timestamp 530 541 9 18.9 53.0 0.4X
+cast to timestamp 235 237 4 42.6 23.5 1.0X
+YEAR of timestamp 542 545 4 18.4 54.2 0.4X
+YEAROFWEEK of timestamp 627 632 8 15.9 62.7 0.4X
+QUARTER of timestamp 639 640 1 15.6 63.9 0.4X
+MONTH of timestamp 561 562 2 17.8 56.1 0.4X
+WEEK of timestamp 885 887 2 11.3 88.5 0.3X
+DAY of timestamp 556 564 12 18.0 55.6 0.4X
+DAYOFWEEK of timestamp 750 754 4 13.3 75.0 0.3X
+DOW of timestamp 750 752 2 13.3 75.0 0.3X
+DOW_ISO of timestamp 686 692 6 14.6 68.6 0.3X
+DAYOFWEEK_ISO of timestamp 688 694 5 14.5 68.8 0.3X
+DOY of timestamp 605 607 2 16.5 60.5 0.4X
+HOUR of timestamp 469 471 1 21.3 46.9 0.5X
+MINUTE of timestamp 471 475 6 21.3 47.1 0.5X
+SECOND of timestamp 534 539 4 18.7 53.4 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Invoke extract for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-cast to date 512 518 7 19.5 51.2 1.0X
-YEAR of date 531 533 2 18.8 53.1 1.0X
-YEAROFWEEK of date 595 596 1 16.8 59.5 0.9X
-QUARTER of date 615 615 1 16.3 61.5 0.8X
-MONTH of date 538 539 1 18.6 53.8 1.0X
-WEEK of date 858 862 4 11.7 85.8 0.6X
-DAY of date 552 553 2 18.1 55.2 0.9X
-DAYOFWEEK of date 740 741 1 13.5 74.0 0.7X
-DOW of date 738 739 2 13.6 73.8 0.7X
-DOW_ISO of date 697 701 6 14.4 69.7 0.7X
-DAYOFWEEK_ISO of date 673 679 5 14.9 67.3 0.8X
-DOY of date 589 597 11 17.0 58.9 0.9X
-HOUR of date 978 989 15 10.2 97.8 0.5X
-MINUTE of date 976 988 13 10.2 97.6 0.5X
-SECOND of date 1035 1041 10 9.7 103.5 0.5X
+cast to date 529 530 1 18.9 52.9 1.0X
+YEAR of date 532 536 4 18.8 53.2 1.0X
+YEAROFWEEK of date 628 629 1 15.9 62.8 0.8X
+QUARTER of date 634 636 2 15.8 63.4 0.8X
+MONTH of date 561 565 5 17.8 56.1 0.9X
+WEEK of date 886 889 3 11.3 88.6 0.6X
+DAY of date 560 563 5 17.9 56.0 0.9X
+DAYOFWEEK of date 756 759 3 13.2 75.6 0.7X
+DOW of date 750 758 9 13.3 75.0 0.7X
+DOW_ISO of date 689 694 4 14.5 68.9 0.8X
+DAYOFWEEK_ISO of date 686 688 2 14.6 68.6 0.8X
+DOY of date 604 605 1 16.6 60.4 0.9X
+HOUR of date 985 995 12 10.2 98.5 0.5X
+MINUTE of date 987 988 1 10.1 98.7 0.5X
+SECOND of date 1036 1041 6 9.7 103.6 0.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Invoke date_part for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-cast to date 511 520 11 19.6 51.1 1.0X
-YEAR of date 531 532 1 18.8 53.1 1.0X
-YEAROFWEEK of date 597 599 3 16.7 59.7 0.9X
-QUARTER of date 616 618 3 16.2 61.6 0.8X
-MONTH of date 534 536 1 18.7 53.4 1.0X
-WEEK of date 853 856 3 11.7 85.3 0.6X
-DAY of date 550 551 2 18.2 55.0 0.9X
-DAYOFWEEK of date 734 759 42 13.6 73.4 0.7X
-DOW of date 734 735 2 13.6 73.4 0.7X
-DOW_ISO of date 673 675 2 14.9 67.3 0.8X
-DAYOFWEEK_ISO of date 672 674 3 14.9 67.2 0.8X
-DOY of date 588 594 8 17.0 58.8 0.9X
-HOUR of date 983 983 1 10.2 98.3 0.5X
-MINUTE of date 981 982 1 10.2 98.1 0.5X
-SECOND of date 1032 1034 2 9.7 103.2 0.5X
+cast to date 525 541 25 19.0 52.5 1.0X
+YEAR of date 533 535 3 18.8 53.3 1.0X
+YEAROFWEEK of date 627 628 1 15.9 62.7 0.8X
+QUARTER of date 631 635 5 15.8 63.1 0.8X
+MONTH of date 556 559 2 18.0 55.6 0.9X
+WEEK of date 885 885 1 11.3 88.5 0.6X
+DAY of date 557 558 1 18.0 55.7 0.9X
+DAYOFWEEK of date 748 749 1 13.4 74.8 0.7X
+DOW of date 748 749 2 13.4 74.8 0.7X
+DOW_ISO of date 687 687 1 14.6 68.7 0.8X
+DAYOFWEEK_ISO of date 685 687 2 14.6 68.5 0.8X
+DOY of date 602 604 2 16.6 60.2 0.9X
+HOUR of date 987 989 2 10.1 98.7 0.5X
+MINUTE of date 985 992 8 10.1 98.5 0.5X
+SECOND of date 1035 1037 3 9.7 103.5 0.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Invoke extract for interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-cast to interval 717 719 2 13.9 71.7 1.0X
-YEAR of interval 713 715 2 14.0 71.3 1.0X
-MONTH of interval 733 736 3 13.6 73.3 1.0X
-DAY of interval 709 717 12 14.1 70.9 1.0X
-HOUR of interval 721 722 2 13.9 72.1 1.0X
-MINUTE of interval 721 727 8 13.9 72.1 1.0X
-SECOND of interval 769 772 4 13.0 76.9 0.9X
+cast to interval 743 746 3 13.5 74.3 1.0X
+YEAR of interval 718 720 2 13.9 71.8 1.0X
+MONTH of interval 724 726 2 13.8 72.4 1.0X
+DAY of interval 723 726 5 13.8 72.3 1.0X
+HOUR of interval 732 733 1 13.7 73.2 1.0X
+MINUTE of interval 735 736 1 13.6 73.5 1.0X
+SECOND of interval 779 784 5 12.8 77.9 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Invoke date_part for interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-cast to interval 720 721 1 13.9 72.0 1.0X
-YEAR of interval 711 741 53 14.1 71.1 1.0X
-MONTH of interval 739 742 4 13.5 73.9 1.0X
-DAY of interval 708 710 2 14.1 70.8 1.0X
-HOUR of interval 723 730 11 13.8 72.3 1.0X
-MINUTE of interval 723 726 2 13.8 72.3 1.0X
-SECOND of interval 779 781 2 12.8 77.9 0.9X
+cast to interval 742 744 2 13.5 74.2 1.0X
+YEAR of interval 719 720 1 13.9 71.9 1.0X
+MONTH of interval 725 727 2 13.8 72.5 1.0X
+DAY of interval 722 725 2 13.8 72.2 1.0X
+HOUR of interval 732 736 4 13.7 73.2 1.0X
+MINUTE of interval 733 743 14 13.6 73.3 1.0X
+SECOND of interval 783 791 13 12.8 78.3 0.9X
diff --git a/sql/core/benchmarks/ExtractBenchmark-results.txt b/sql/core/benchmarks/ExtractBenchmark-results.txt
index 5c850be75f60d..96179820ea6f1 100644
--- a/sql/core/benchmarks/ExtractBenchmark-results.txt
+++ b/sql/core/benchmarks/ExtractBenchmark-results.txt
@@ -1,104 +1,104 @@
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Invoke extract for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-cast to timestamp 246 257 10 40.6 24.6 1.0X
-YEAR of timestamp 703 706 3 14.2 70.3 0.4X
-YEAROFWEEK of timestamp 767 780 12 13.0 76.7 0.3X
-QUARTER of timestamp 730 731 1 13.7 73.0 0.3X
-MONTH of timestamp 703 707 5 14.2 70.3 0.4X
-WEEK of timestamp 990 993 3 10.1 99.0 0.2X
-DAY of timestamp 694 700 6 14.4 69.4 0.4X
-DAYOFWEEK of timestamp 843 845 2 11.9 84.3 0.3X
-DOW of timestamp 841 845 5 11.9 84.1 0.3X
-DOW_ISO of timestamp 795 799 4 12.6 79.5 0.3X
-DAYOFWEEK_ISO of timestamp 800 803 5 12.5 80.0 0.3X
-DOY of timestamp 712 715 3 14.0 71.2 0.3X
-HOUR of timestamp 506 513 10 19.8 50.6 0.5X
-MINUTE of timestamp 504 508 7 19.8 50.4 0.5X
-SECOND of timestamp 603 604 2 16.6 60.3 0.4X
+cast to timestamp 255 276 27 39.3 25.5 1.0X
+YEAR of timestamp 695 701 8 14.4 69.5 0.4X
+YEAROFWEEK of timestamp 759 760 1 13.2 75.9 0.3X
+QUARTER of timestamp 700 717 23 14.3 70.0 0.4X
+MONTH of timestamp 682 685 4 14.7 68.2 0.4X
+WEEK of timestamp 988 989 2 10.1 98.8 0.3X
+DAY of timestamp 680 693 22 14.7 68.0 0.4X
+DAYOFWEEK of timestamp 828 833 5 12.1 82.8 0.3X
+DOW of timestamp 827 830 4 12.1 82.7 0.3X
+DOW_ISO of timestamp 803 806 2 12.4 80.3 0.3X
+DAYOFWEEK_ISO of timestamp 784 785 1 12.8 78.4 0.3X
+DOY of timestamp 702 704 3 14.2 70.2 0.4X
+HOUR of timestamp 538 540 2 18.6 53.8 0.5X
+MINUTE of timestamp 538 540 2 18.6 53.8 0.5X
+SECOND of timestamp 644 649 6 15.5 64.4 0.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Invoke date_part for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-cast to timestamp 200 203 3 50.0 20.0 1.0X
-YEAR of timestamp 663 669 5 15.1 66.3 0.3X
-YEAROFWEEK of timestamp 715 718 3 14.0 71.5 0.3X
-QUARTER of timestamp 686 692 10 14.6 68.6 0.3X
-MONTH of timestamp 667 670 3 15.0 66.7 0.3X
-WEEK of timestamp 951 956 5 10.5 95.1 0.2X
-DAY of timestamp 667 681 15 15.0 66.7 0.3X
-DAYOFWEEK of timestamp 808 813 8 12.4 80.8 0.2X
-DOW of timestamp 809 812 3 12.4 80.9 0.2X
-DOW_ISO of timestamp 765 766 1 13.1 76.5 0.3X
-DAYOFWEEK_ISO of timestamp 762 763 3 13.1 76.2 0.3X
-DOY of timestamp 704 709 6 14.2 70.4 0.3X
-HOUR of timestamp 504 507 3 19.8 50.4 0.4X
-MINUTE of timestamp 509 511 2 19.6 50.9 0.4X
-SECOND of timestamp 598 600 3 16.7 59.8 0.3X
+cast to timestamp 233 234 1 43.0 23.3 1.0X
+YEAR of timestamp 666 670 4 15.0 66.6 0.3X
+YEAROFWEEK of timestamp 735 739 4 13.6 73.5 0.3X
+QUARTER of timestamp 688 695 11 14.5 68.8 0.3X
+MONTH of timestamp 670 676 6 14.9 67.0 0.3X
+WEEK of timestamp 980 981 1 10.2 98.0 0.2X
+DAY of timestamp 670 674 5 14.9 67.0 0.3X
+DAYOFWEEK of timestamp 825 835 11 12.1 82.5 0.3X
+DOW of timestamp 821 827 10 12.2 82.1 0.3X
+DOW_ISO of timestamp 781 782 1 12.8 78.1 0.3X
+DAYOFWEEK_ISO of timestamp 780 788 9 12.8 78.0 0.3X
+DOY of timestamp 705 711 9 14.2 70.5 0.3X
+HOUR of timestamp 536 541 5 18.7 53.6 0.4X
+MINUTE of timestamp 546 549 4 18.3 54.6 0.4X
+SECOND of timestamp 644 658 23 15.5 64.4 0.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Invoke extract for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-cast to date 572 575 3 17.5 57.2 1.0X
-YEAR of date 660 664 4 15.1 66.0 0.9X
-YEAROFWEEK of date 714 718 4 14.0 71.4 0.8X
-QUARTER of date 683 698 21 14.7 68.3 0.8X
-MONTH of date 666 674 9 15.0 66.6 0.9X
-WEEK of date 953 955 3 10.5 95.3 0.6X
-DAY of date 673 676 4 14.9 67.3 0.9X
-DAYOFWEEK of date 810 815 8 12.3 81.0 0.7X
-DOW of date 807 810 3 12.4 80.7 0.7X
-DOW_ISO of date 766 769 3 13.1 76.6 0.7X
-DAYOFWEEK_ISO of date 765 766 1 13.1 76.5 0.7X
-DOY of date 704 705 1 14.2 70.4 0.8X
-HOUR of date 1118 1121 4 8.9 111.8 0.5X
-MINUTE of date 1129 1130 2 8.9 112.9 0.5X
-SECOND of date 1281 1287 9 7.8 128.1 0.4X
+cast to date 585 586 1 17.1 58.5 1.0X
+YEAR of date 670 674 4 14.9 67.0 0.9X
+YEAROFWEEK of date 738 749 11 13.6 73.8 0.8X
+QUARTER of date 701 701 0 14.3 70.1 0.8X
+MONTH of date 667 673 9 15.0 66.7 0.9X
+WEEK of date 974 978 5 10.3 97.4 0.6X
+DAY of date 669 674 7 14.9 66.9 0.9X
+DAYOFWEEK of date 825 837 20 12.1 82.5 0.7X
+DOW of date 821 824 4 12.2 82.1 0.7X
+DOW_ISO of date 776 777 1 12.9 77.6 0.8X
+DAYOFWEEK_ISO of date 775 776 1 12.9 77.5 0.8X
+DOY of date 697 701 6 14.3 69.7 0.8X
+HOUR of date 1129 1130 1 8.9 112.9 0.5X
+MINUTE of date 1131 1141 17 8.8 113.1 0.5X
+SECOND of date 1255 1257 3 8.0 125.5 0.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Invoke date_part for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-cast to date 570 573 3 17.5 57.0 1.0X
-YEAR of date 659 659 1 15.2 65.9 0.9X
-YEAROFWEEK of date 713 716 4 14.0 71.3 0.8X
-QUARTER of date 683 684 1 14.6 68.3 0.8X
-MONTH of date 663 666 3 15.1 66.3 0.9X
-WEEK of date 947 950 2 10.6 94.7 0.6X
-DAY of date 665 668 3 15.0 66.5 0.9X
-DAYOFWEEK of date 807 808 1 12.4 80.7 0.7X
-DOW of date 805 809 3 12.4 80.5 0.7X
-DOW_ISO of date 758 762 4 13.2 75.8 0.8X
-DAYOFWEEK_ISO of date 761 763 3 13.1 76.1 0.7X
-DOY of date 701 705 4 14.3 70.1 0.8X
-HOUR of date 1119 1129 16 8.9 111.9 0.5X
-MINUTE of date 1121 1128 6 8.9 112.1 0.5X
-SECOND of date 1274 1284 9 7.8 127.4 0.4X
+cast to date 582 583 2 17.2 58.2 1.0X
+YEAR of date 663 663 1 15.1 66.3 0.9X
+YEAROFWEEK of date 731 735 3 13.7 73.1 0.8X
+QUARTER of date 682 688 9 14.7 68.2 0.9X
+MONTH of date 667 669 2 15.0 66.7 0.9X
+WEEK of date 975 980 6 10.3 97.5 0.6X
+DAY of date 665 672 6 15.0 66.5 0.9X
+DAYOFWEEK of date 822 824 3 12.2 82.2 0.7X
+DOW of date 818 822 5 12.2 81.8 0.7X
+DOW_ISO of date 776 787 18 12.9 77.6 0.7X
+DAYOFWEEK_ISO of date 771 784 19 13.0 77.1 0.8X
+DOY of date 697 701 6 14.4 69.7 0.8X
+HOUR of date 1133 1137 6 8.8 113.3 0.5X
+MINUTE of date 1137 1138 1 8.8 113.7 0.5X
+SECOND of date 1245 1247 1 8.0 124.5 0.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Invoke extract for interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-cast to interval 786 791 5 12.7 78.6 1.0X
-YEAR of interval 786 791 4 12.7 78.6 1.0X
-MONTH of interval 796 799 3 12.6 79.6 1.0X
-DAY of interval 788 790 2 12.7 78.8 1.0X
-HOUR of interval 791 794 3 12.6 79.1 1.0X
-MINUTE of interval 797 799 3 12.5 79.7 1.0X
-SECOND of interval 885 886 2 11.3 88.5 0.9X
+cast to interval 811 824 13 12.3 81.1 1.0X
+YEAR of interval 796 799 3 12.6 79.6 1.0X
+MONTH of interval 801 803 4 12.5 80.1 1.0X
+DAY of interval 801 807 8 12.5 80.1 1.0X
+HOUR of interval 803 806 4 12.5 80.3 1.0X
+MINUTE of interval 808 809 2 12.4 80.8 1.0X
+SECOND of interval 911 919 9 11.0 91.1 0.9X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Invoke date_part for interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-cast to interval 791 794 2 12.6 79.1 1.0X
-YEAR of interval 789 805 15 12.7 78.9 1.0X
-MONTH of interval 781 787 5 12.8 78.1 1.0X
-DAY of interval 791 794 2 12.6 79.1 1.0X
-HOUR of interval 798 804 11 12.5 79.8 1.0X
-MINUTE of interval 795 798 3 12.6 79.5 1.0X
-SECOND of interval 881 883 2 11.3 88.1 0.9X
+cast to interval 816 819 3 12.3 81.6 1.0X
+YEAR of interval 805 808 2 12.4 80.5 1.0X
+MONTH of interval 801 802 1 12.5 80.1 1.0X
+DAY of interval 797 814 16 12.5 79.7 1.0X
+HOUR of interval 806 807 1 12.4 80.6 1.0X
+MINUTE of interval 804 815 15 12.4 80.4 1.0X
+SECOND of interval 912 924 12 11.0 91.2 0.9X
diff --git a/sql/core/benchmarks/FilterPushdownBenchmark-jdk21-results.txt b/sql/core/benchmarks/FilterPushdownBenchmark-jdk21-results.txt
index 8128fae5e350f..c00d771725f8a 100644
--- a/sql/core/benchmarks/FilterPushdownBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/FilterPushdownBenchmark-jdk21-results.txt
@@ -2,733 +2,733 @@
Pushdown for many distinct value case
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 0 string row (value IS NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6419 6495 69 2.5 408.1 1.0X
-Parquet Vectorized (Pushdown) 365 389 16 43.1 23.2 17.6X
-Native ORC Vectorized 5170 5230 67 3.0 328.7 1.2X
-Native ORC Vectorized (Pushdown) 309 324 13 50.9 19.6 20.8X
+Parquet Vectorized 6457 6511 42 2.4 410.5 1.0X
+Parquet Vectorized (Pushdown) 331 362 18 47.5 21.0 19.5X
+Native ORC Vectorized 5199 5291 68 3.0 330.5 1.2X
+Native ORC Vectorized (Pushdown) 318 330 8 49.5 20.2 20.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 0 string row ('7864320' < value < '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6431 6441 7 2.4 408.9 1.0X
-Parquet Vectorized (Pushdown) 324 334 9 48.5 20.6 19.8X
-Native ORC Vectorized 5259 5289 32 3.0 334.4 1.2X
-Native ORC Vectorized (Pushdown) 315 325 10 50.0 20.0 20.4X
+Parquet Vectorized 6315 6337 18 2.5 401.5 1.0X
+Parquet Vectorized (Pushdown) 327 335 6 48.1 20.8 19.3X
+Native ORC Vectorized 5307 5322 17 3.0 337.4 1.2X
+Native ORC Vectorized (Pushdown) 325 347 28 48.3 20.7 19.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 string row (value = '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6492 6515 17 2.4 412.8 1.0X
-Parquet Vectorized (Pushdown) 309 326 14 50.9 19.7 21.0X
-Native ORC Vectorized 5394 5411 26 2.9 342.9 1.2X
-Native ORC Vectorized (Pushdown) 299 316 12 52.7 19.0 21.7X
+Parquet Vectorized 6387 6422 28 2.5 406.1 1.0X
+Parquet Vectorized (Pushdown) 324 330 5 48.6 20.6 19.7X
+Native ORC Vectorized 5314 5324 7 3.0 337.9 1.2X
+Native ORC Vectorized (Pushdown) 320 327 6 49.2 20.3 20.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 string row (value <=> '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6495 6531 31 2.4 412.9 1.0X
-Parquet Vectorized (Pushdown) 304 313 10 51.8 19.3 21.4X
-Native ORC Vectorized 5285 5300 14 3.0 336.0 1.2X
-Native ORC Vectorized (Pushdown) 292 305 9 53.8 18.6 22.2X
+Parquet Vectorized 6328 6336 7 2.5 402.3 1.0X
+Parquet Vectorized (Pushdown) 317 320 2 49.6 20.1 20.0X
+Native ORC Vectorized 5292 5307 14 3.0 336.4 1.2X
+Native ORC Vectorized (Pushdown) 315 318 3 50.0 20.0 20.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 string row ('7864320' <= value <= '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6421 6460 39 2.4 408.2 1.0X
-Parquet Vectorized (Pushdown) 303 316 9 51.8 19.3 21.2X
-Native ORC Vectorized 5286 5308 16 3.0 336.0 1.2X
-Native ORC Vectorized (Pushdown) 298 304 6 52.8 19.0 21.5X
+Parquet Vectorized 6317 6332 15 2.5 401.6 1.0X
+Parquet Vectorized (Pushdown) 317 323 6 49.6 20.1 19.9X
+Native ORC Vectorized 5298 5316 12 3.0 336.8 1.2X
+Native ORC Vectorized (Pushdown) 317 328 11 49.6 20.2 19.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select all string rows (value IS NOT NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 14207 14265 44 1.1 903.3 1.0X
-Parquet Vectorized (Pushdown) 14166 14227 38 1.1 900.7 1.0X
-Native ORC Vectorized 12799 12822 29 1.2 813.7 1.1X
-Native ORC Vectorized (Pushdown) 12860 12881 15 1.2 817.6 1.1X
+Parquet Vectorized 13607 13761 92 1.2 865.1 1.0X
+Parquet Vectorized (Pushdown) 13872 13912 58 1.1 881.9 1.0X
+Native ORC Vectorized 12865 12905 33 1.2 817.9 1.1X
+Native ORC Vectorized (Pushdown) 12967 13013 37 1.2 824.4 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 0 int row (value IS NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6233 6274 34 2.5 396.3 1.0X
-Parquet Vectorized (Pushdown) 296 324 23 53.2 18.8 21.1X
-Native ORC Vectorized 4775 4801 25 3.3 303.6 1.3X
-Native ORC Vectorized (Pushdown) 294 303 8 53.5 18.7 21.2X
+Parquet Vectorized 6151 6175 19 2.6 391.1 1.0X
+Parquet Vectorized (Pushdown) 292 313 19 53.8 18.6 21.1X
+Native ORC Vectorized 4759 4811 39 3.3 302.6 1.3X
+Native ORC Vectorized (Pushdown) 287 310 12 54.8 18.2 21.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 0 int row (7864320 < value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6109 6131 22 2.6 388.4 1.0X
-Parquet Vectorized (Pushdown) 316 324 6 49.8 20.1 19.3X
-Native ORC Vectorized 4766 4781 13 3.3 303.0 1.3X
-Native ORC Vectorized (Pushdown) 297 305 7 53.0 18.9 20.6X
+Parquet Vectorized 6072 6079 5 2.6 386.1 1.0X
+Parquet Vectorized (Pushdown) 297 304 5 52.9 18.9 20.4X
+Native ORC Vectorized 4752 4766 9 3.3 302.1 1.3X
+Native ORC Vectorized (Pushdown) 287 298 10 54.7 18.3 21.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 int row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6211 6238 26 2.5 394.9 1.0X
-Parquet Vectorized (Pushdown) 304 315 9 51.8 19.3 20.5X
-Native ORC Vectorized 4824 4849 22 3.3 306.7 1.3X
-Native ORC Vectorized (Pushdown) 278 286 7 56.6 17.7 22.4X
+Parquet Vectorized 6134 6144 7 2.6 390.0 1.0X
+Parquet Vectorized (Pushdown) 291 299 5 54.1 18.5 21.1X
+Native ORC Vectorized 4773 4832 34 3.3 303.5 1.3X
+Native ORC Vectorized (Pushdown) 288 294 4 54.6 18.3 21.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 int row (value <=> 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6153 6156 4 2.6 391.2 1.0X
-Parquet Vectorized (Pushdown) 295 301 4 53.4 18.7 20.9X
-Native ORC Vectorized 4864 4881 10 3.2 309.3 1.3X
-Native ORC Vectorized (Pushdown) 277 289 9 56.8 17.6 22.2X
+Parquet Vectorized 6130 6144 20 2.6 389.7 1.0X
+Parquet Vectorized (Pushdown) 288 293 4 54.7 18.3 21.3X
+Native ORC Vectorized 4769 4810 23 3.3 303.2 1.3X
+Native ORC Vectorized (Pushdown) 286 292 5 55.0 18.2 21.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 int row (7864320 <= value <= 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6164 6181 14 2.6 391.9 1.0X
-Parquet Vectorized (Pushdown) 299 302 4 52.7 19.0 20.6X
-Native ORC Vectorized 4846 4871 24 3.2 308.1 1.3X
-Native ORC Vectorized (Pushdown) 277 289 8 56.7 17.6 22.2X
+Parquet Vectorized 6118 6142 22 2.6 388.9 1.0X
+Parquet Vectorized (Pushdown) 290 294 3 54.2 18.4 21.1X
+Native ORC Vectorized 4826 4830 2 3.3 306.9 1.3X
+Native ORC Vectorized (Pushdown) 288 294 4 54.7 18.3 21.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 int row (7864319 < value < 7864321): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6138 6163 16 2.6 390.2 1.0X
-Parquet Vectorized (Pushdown) 291 300 5 54.0 18.5 21.1X
-Native ORC Vectorized 4833 4849 14 3.3 307.3 1.3X
-Native ORC Vectorized (Pushdown) 274 282 6 57.5 17.4 22.4X
+Parquet Vectorized 6113 6125 17 2.6 388.6 1.0X
+Parquet Vectorized (Pushdown) 288 293 4 54.7 18.3 21.3X
+Native ORC Vectorized 4806 4820 21 3.3 305.5 1.3X
+Native ORC Vectorized (Pushdown) 283 290 5 55.5 18.0 21.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 10% int rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6789 6817 29 2.3 431.6 1.0X
-Parquet Vectorized (Pushdown) 1547 1564 17 10.2 98.3 4.4X
-Native ORC Vectorized 5444 5466 17 2.9 346.1 1.2X
-Native ORC Vectorized (Pushdown) 1388 1394 8 11.3 88.2 4.9X
+Parquet Vectorized 6781 6812 29 2.3 431.1 1.0X
+Parquet Vectorized (Pushdown) 1545 1550 6 10.2 98.2 4.4X
+Native ORC Vectorized 5477 5482 4 2.9 348.2 1.2X
+Native ORC Vectorized (Pushdown) 1426 1434 7 11.0 90.6 4.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 50% int rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 9167 9207 37 1.7 582.8 1.0X
-Parquet Vectorized (Pushdown) 6291 6306 12 2.5 400.0 1.5X
-Native ORC Vectorized 7845 7882 46 2.0 498.8 1.2X
-Native ORC Vectorized (Pushdown) 5718 5729 11 2.8 363.5 1.6X
+Parquet Vectorized 9288 9304 12 1.7 590.5 1.0X
+Parquet Vectorized (Pushdown) 6431 6456 17 2.4 408.9 1.4X
+Native ORC Vectorized 8090 8131 60 1.9 514.3 1.1X
+Native ORC Vectorized (Pushdown) 5847 5861 11 2.7 371.8 1.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 90% int rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 11661 11680 16 1.3 741.4 1.0X
-Parquet Vectorized (Pushdown) 11121 11142 23 1.4 707.0 1.0X
-Native ORC Vectorized 10366 10402 38 1.5 659.1 1.1X
-Native ORC Vectorized (Pushdown) 9982 10009 28 1.6 634.6 1.2X
+Parquet Vectorized 11939 11962 20 1.3 759.0 1.0X
+Parquet Vectorized (Pushdown) 11431 11452 14 1.4 726.8 1.0X
+Native ORC Vectorized 10627 10646 17 1.5 675.6 1.1X
+Native ORC Vectorized (Pushdown) 10242 10282 53 1.5 651.2 1.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select all int rows (value IS NOT NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 12163 12182 18 1.3 773.3 1.0X
-Parquet Vectorized (Pushdown) 12196 12224 20 1.3 775.4 1.0X
-Native ORC Vectorized 10954 10971 26 1.4 696.5 1.1X
-Native ORC Vectorized (Pushdown) 11012 11035 19 1.4 700.2 1.1X
+Parquet Vectorized 12465 12473 6 1.3 792.5 1.0X
+Parquet Vectorized (Pushdown) 12514 12541 21 1.3 795.6 1.0X
+Native ORC Vectorized 11133 11145 10 1.4 707.8 1.1X
+Native ORC Vectorized (Pushdown) 11245 11275 22 1.4 715.0 1.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select all int rows (value > -1): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 12326 12343 12 1.3 783.7 1.0X
-Parquet Vectorized (Pushdown) 12321 12343 19 1.3 783.3 1.0X
-Native ORC Vectorized 10831 10850 27 1.5 688.6 1.1X
-Native ORC Vectorized (Pushdown) 10937 10953 12 1.4 695.3 1.1X
+Parquet Vectorized 12470 12495 26 1.3 792.8 1.0X
+Parquet Vectorized (Pushdown) 12543 12569 17 1.3 797.5 1.0X
+Native ORC Vectorized 11253 11274 12 1.4 715.5 1.1X
+Native ORC Vectorized (Pushdown) 11339 11371 20 1.4 720.9 1.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select all int rows (value != -1): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 12060 12089 20 1.3 766.7 1.0X
-Parquet Vectorized (Pushdown) 12142 12186 43 1.3 772.0 1.0X
-Native ORC Vectorized 10799 10834 24 1.5 686.6 1.1X
-Native ORC Vectorized (Pushdown) 10903 10952 41 1.4 693.2 1.1X
+Parquet Vectorized 12512 12556 28 1.3 795.5 1.0X
+Parquet Vectorized (Pushdown) 12589 12618 28 1.2 800.4 1.0X
+Native ORC Vectorized 11292 11305 24 1.4 718.0 1.1X
+Native ORC Vectorized (Pushdown) 11355 11384 31 1.4 721.9 1.1X
================================================================================================
Pushdown for few distinct value case (use dictionary encoding)
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 0 distinct string row (value IS NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 5716 5771 45 2.8 363.4 1.0X
-Parquet Vectorized (Pushdown) 245 256 11 64.1 15.6 23.3X
-Native ORC Vectorized 6171 6213 43 2.5 392.3 0.9X
-Native ORC Vectorized (Pushdown) 906 913 6 17.4 57.6 6.3X
+Parquet Vectorized 5514 5533 19 2.9 350.5 1.0X
+Parquet Vectorized (Pushdown) 240 265 22 65.6 15.2 23.0X
+Native ORC Vectorized 6365 6386 19 2.5 404.7 0.9X
+Native ORC Vectorized (Pushdown) 944 955 9 16.7 60.0 5.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 0 distinct string row ('100' < value < '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 5665 5686 22 2.8 360.2 1.0X
-Parquet Vectorized (Pushdown) 246 254 7 63.9 15.7 23.0X
-Native ORC Vectorized 6401 6409 15 2.5 407.0 0.9X
-Native ORC Vectorized (Pushdown) 908 921 15 17.3 57.7 6.2X
+Parquet Vectorized 5579 5590 9 2.8 354.7 1.0X
+Parquet Vectorized (Pushdown) 239 246 10 65.9 15.2 23.4X
+Native ORC Vectorized 6575 6586 9 2.4 418.0 0.8X
+Native ORC Vectorized (Pushdown) 947 954 5 16.6 60.2 5.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 distinct string row (value = '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 5621 5633 12 2.8 357.3 1.0X
-Parquet Vectorized (Pushdown) 310 317 7 50.8 19.7 18.1X
-Native ORC Vectorized 6390 6402 10 2.5 406.2 0.9X
-Native ORC Vectorized (Pushdown) 949 955 6 16.6 60.3 5.9X
+Parquet Vectorized 5554 5560 5 2.8 353.1 1.0X
+Parquet Vectorized (Pushdown) 284 290 6 55.3 18.1 19.5X
+Native ORC Vectorized 6614 6626 12 2.4 420.5 0.8X
+Native ORC Vectorized (Pushdown) 996 1013 22 15.8 63.3 5.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 distinct string row (value <=> '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 5605 5620 18 2.8 356.4 1.0X
-Parquet Vectorized (Pushdown) 288 292 3 54.6 18.3 19.4X
-Native ORC Vectorized 6377 6406 26 2.5 405.4 0.9X
-Native ORC Vectorized (Pushdown) 947 954 6 16.6 60.2 5.9X
+Parquet Vectorized 5500 5524 21 2.9 349.7 1.0X
+Parquet Vectorized (Pushdown) 282 286 2 55.7 18.0 19.5X
+Native ORC Vectorized 6609 6637 18 2.4 420.2 0.8X
+Native ORC Vectorized (Pushdown) 1000 1008 10 15.7 63.6 5.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 distinct string row ('100' <= value <= '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 5673 5705 27 2.8 360.7 1.0X
-Parquet Vectorized (Pushdown) 294 303 10 53.5 18.7 19.3X
-Native ORC Vectorized 6458 6481 27 2.4 410.6 0.9X
-Native ORC Vectorized (Pushdown) 958 962 5 16.4 60.9 5.9X
+Parquet Vectorized 5574 5599 24 2.8 354.4 1.0X
+Parquet Vectorized (Pushdown) 285 293 8 55.1 18.1 19.5X
+Native ORC Vectorized 6654 6668 15 2.4 423.0 0.8X
+Native ORC Vectorized (Pushdown) 999 1005 7 15.7 63.5 5.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select all distinct string rows (value IS NOT NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 13823 13841 17 1.1 878.8 1.0X
-Parquet Vectorized (Pushdown) 13836 13862 22 1.1 879.6 1.0X
-Native ORC Vectorized 14784 14837 79 1.1 939.9 0.9X
-Native ORC Vectorized (Pushdown) 14990 15048 52 1.0 953.1 0.9X
+Parquet Vectorized 13703 13741 26 1.1 871.2 1.0X
+Parquet Vectorized (Pushdown) 13739 13772 24 1.1 873.5 1.0X
+Native ORC Vectorized 15511 15562 38 1.0 986.2 0.9X
+Native ORC Vectorized (Pushdown) 15773 15841 122 1.0 1002.8 0.9X
================================================================================================
Pushdown benchmark for StringStartsWith
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
StringStartsWith filter: (value like '10%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6910 6946 24 2.3 439.3 1.0X
-Parquet Vectorized (Pushdown) 922 938 14 17.1 58.6 7.5X
-Native ORC Vectorized 5528 5546 14 2.8 351.4 1.3X
-Native ORC Vectorized (Pushdown) 5609 5625 10 2.8 356.6 1.2X
+Parquet Vectorized 6657 6673 11 2.4 423.2 1.0X
+Parquet Vectorized (Pushdown) 922 929 4 17.1 58.6 7.2X
+Native ORC Vectorized 5672 5736 58 2.8 360.6 1.2X
+Native ORC Vectorized (Pushdown) 5742 5761 21 2.7 365.1 1.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
StringStartsWith filter: (value like '1000%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6554 6567 11 2.4 416.7 1.0X
-Parquet Vectorized (Pushdown) 302 310 7 52.1 19.2 21.7X
-Native ORC Vectorized 5339 5360 17 2.9 339.4 1.2X
-Native ORC Vectorized (Pushdown) 5436 5460 20 2.9 345.6 1.2X
+Parquet Vectorized 6434 6445 16 2.4 409.1 1.0X
+Parquet Vectorized (Pushdown) 278 282 4 56.7 17.7 23.2X
+Native ORC Vectorized 5484 5500 25 2.9 348.7 1.2X
+Native ORC Vectorized (Pushdown) 5556 5579 17 2.8 353.2 1.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
StringStartsWith filter: (value like '786432%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6549 6556 6 2.4 416.4 1.0X
-Parquet Vectorized (Pushdown) 296 303 5 53.2 18.8 22.1X
-Native ORC Vectorized 5348 5378 20 2.9 340.0 1.2X
-Native ORC Vectorized (Pushdown) 5441 5449 8 2.9 345.9 1.2X
+Parquet Vectorized 6419 6434 15 2.5 408.1 1.0X
+Parquet Vectorized (Pushdown) 268 276 7 58.7 17.0 23.9X
+Native ORC Vectorized 5454 5480 27 2.9 346.7 1.2X
+Native ORC Vectorized (Pushdown) 5541 5561 16 2.8 352.3 1.2X
================================================================================================
Pushdown benchmark for StringEndsWith
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
StringEndsWith filter: (value like '%10'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 5813 5950 171 2.7 369.6 1.0X
-Parquet Vectorized (Pushdown) 403 455 56 39.0 25.6 14.4X
-Native ORC Vectorized 6523 6559 24 2.4 414.7 0.9X
-Native ORC Vectorized (Pushdown) 6692 6727 30 2.4 425.5 0.9X
+Parquet Vectorized 5559 5580 31 2.8 353.5 1.0X
+Parquet Vectorized (Pushdown) 359 379 21 43.8 22.8 15.5X
+Native ORC Vectorized 6650 6690 58 2.4 422.8 0.8X
+Native ORC Vectorized (Pushdown) 6873 6889 19 2.3 437.0 0.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
StringEndsWith filter: (value like '%1000'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 5552 5564 9 2.8 353.0 1.0X
-Parquet Vectorized (Pushdown) 284 304 19 55.5 18.0 19.6X
-Native ORC Vectorized 6441 6457 16 2.4 409.5 0.9X
-Native ORC Vectorized (Pushdown) 6687 6714 17 2.4 425.1 0.8X
+Parquet Vectorized 5438 5451 12 2.9 345.7 1.0X
+Parquet Vectorized (Pushdown) 267 271 4 59.0 17.0 20.4X
+Native ORC Vectorized 6587 6605 19 2.4 418.8 0.8X
+Native ORC Vectorized (Pushdown) 6784 6809 21 2.3 431.3 0.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
StringEndsWith filter: (value like '%786432'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 5525 5541 21 2.8 351.3 1.0X
-Parquet Vectorized (Pushdown) 276 290 7 56.9 17.6 20.0X
-Native ORC Vectorized 6437 6455 17 2.4 409.2 0.9X
-Native ORC Vectorized (Pushdown) 6634 6658 21 2.4 421.8 0.8X
+Parquet Vectorized 5445 5460 10 2.9 346.2 1.0X
+Parquet Vectorized (Pushdown) 269 279 14 58.6 17.1 20.3X
+Native ORC Vectorized 6625 6641 10 2.4 421.2 0.8X
+Native ORC Vectorized (Pushdown) 6854 6869 12 2.3 435.8 0.8X
================================================================================================
Pushdown benchmark for StringContains
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
StringContains filter: (value like '%10%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 5809 5972 92 2.7 369.3 1.0X
-Parquet Vectorized (Pushdown) 832 846 14 18.9 52.9 7.0X
-Native ORC Vectorized 6491 6536 47 2.4 412.7 0.9X
-Native ORC Vectorized (Pushdown) 6711 6733 27 2.3 426.7 0.9X
+Parquet Vectorized 5895 5923 22 2.7 374.8 1.0X
+Parquet Vectorized (Pushdown) 821 837 13 19.2 52.2 7.2X
+Native ORC Vectorized 6951 7011 79 2.3 442.0 0.8X
+Native ORC Vectorized (Pushdown) 7168 7197 25 2.2 455.7 0.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
StringContains filter: (value like '%1000%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 5538 5551 15 2.8 352.1 1.0X
-Parquet Vectorized (Pushdown) 280 291 8 56.1 17.8 19.8X
-Native ORC Vectorized 6255 6270 12 2.5 397.7 0.9X
-Native ORC Vectorized (Pushdown) 6481 6505 23 2.4 412.1 0.9X
+Parquet Vectorized 5503 5520 16 2.9 349.8 1.0X
+Parquet Vectorized (Pushdown) 267 273 3 58.8 17.0 20.6X
+Native ORC Vectorized 6657 6667 13 2.4 423.2 0.8X
+Native ORC Vectorized (Pushdown) 6874 6888 12 2.3 437.0 0.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
StringContains filter: (value like '%786432%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 5541 5547 6 2.8 352.3 1.0X
-Parquet Vectorized (Pushdown) 282 287 4 55.7 17.9 19.6X
-Native ORC Vectorized 6246 6271 21 2.5 397.1 0.9X
-Native ORC Vectorized (Pushdown) 6461 6475 10 2.4 410.8 0.9X
+Parquet Vectorized 5503 5517 13 2.9 349.9 1.0X
+Parquet Vectorized (Pushdown) 270 275 3 58.2 17.2 20.4X
+Native ORC Vectorized 6659 6681 22 2.4 423.3 0.8X
+Native ORC Vectorized (Pushdown) 6865 6898 22 2.3 436.5 0.8X
================================================================================================
Pushdown benchmark for decimal
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 decimal(9, 2) row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 2439 2480 64 6.5 155.0 1.0X
-Parquet Vectorized (Pushdown) 72 88 17 219.8 4.5 34.1X
-Native ORC Vectorized 3246 3265 25 4.8 206.4 0.8X
-Native ORC Vectorized (Pushdown) 58 63 6 272.9 3.7 42.3X
+Parquet Vectorized 2335 2352 24 6.7 148.4 1.0X
+Parquet Vectorized (Pushdown) 70 81 5 223.8 4.5 33.2X
+Native ORC Vectorized 3294 3304 18 4.8 209.4 0.7X
+Native ORC Vectorized (Pushdown) 61 70 5 257.9 3.9 38.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 10% decimal(9, 2) rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 3793 3802 8 4.1 241.2 1.0X
-Parquet Vectorized (Pushdown) 1887 1907 20 8.3 120.0 2.0X
-Native ORC Vectorized 4669 4712 44 3.4 296.8 0.8X
-Native ORC Vectorized (Pushdown) 2076 2103 21 7.6 132.0 1.8X
+Parquet Vectorized 3704 3728 29 4.2 235.5 1.0X
+Parquet Vectorized (Pushdown) 1851 1863 7 8.5 117.7 2.0X
+Native ORC Vectorized 4887 4906 11 3.2 310.7 0.8X
+Native ORC Vectorized (Pushdown) 2234 2240 4 7.0 142.1 1.7X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 50% decimal(9, 2) rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 8246 8261 17 1.9 524.3 1.0X
-Parquet Vectorized (Pushdown) 7933 7958 17 2.0 504.4 1.0X
-Native ORC Vectorized 9316 9341 19 1.7 592.3 0.9X
-Native ORC Vectorized (Pushdown) 8905 8938 27 1.8 566.1 0.9X
+Parquet Vectorized 8080 8103 29 1.9 513.7 1.0X
+Parquet Vectorized (Pushdown) 7803 7823 12 2.0 496.1 1.0X
+Native ORC Vectorized 10009 10022 11 1.6 636.4 0.8X
+Native ORC Vectorized (Pushdown) 9572 9599 39 1.6 608.5 0.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 90% decimal(9, 2) rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 9503 9576 69 1.7 604.2 1.0X
-Parquet Vectorized (Pushdown) 9497 9514 14 1.7 603.8 1.0X
-Native ORC Vectorized 10621 10667 50 1.5 675.2 0.9X
-Native ORC Vectorized (Pushdown) 10603 10654 36 1.5 674.1 0.9X
+Parquet Vectorized 9337 9366 32 1.7 593.6 1.0X
+Parquet Vectorized (Pushdown) 9353 9366 10 1.7 594.6 1.0X
+Native ORC Vectorized 11264 11287 18 1.4 716.1 0.8X
+Native ORC Vectorized (Pushdown) 11287 11309 16 1.4 717.6 0.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 decimal(18, 2) row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 2641 2666 30 6.0 167.9 1.0X
-Parquet Vectorized (Pushdown) 72 78 10 217.5 4.6 36.5X
-Native ORC Vectorized 3288 3324 37 4.8 209.0 0.8X
-Native ORC Vectorized (Pushdown) 55 65 7 286.3 3.5 48.1X
+Parquet Vectorized 2529 2539 7 6.2 160.8 1.0X
+Parquet Vectorized (Pushdown) 71 82 7 221.4 4.5 35.6X
+Native ORC Vectorized 3300 3321 26 4.8 209.8 0.8X
+Native ORC Vectorized (Pushdown) 59 66 5 267.7 3.7 43.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 10% decimal(18, 2) rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 3314 3376 74 4.7 210.7 1.0X
-Parquet Vectorized (Pushdown) 1045 1054 8 15.1 66.4 3.2X
-Native ORC Vectorized 4024 4048 36 3.9 255.9 0.8X
-Native ORC Vectorized (Pushdown) 1090 1102 12 14.4 69.3 3.0X
+Parquet Vectorized 3232 3257 36 4.9 205.5 1.0X
+Parquet Vectorized (Pushdown) 1028 1031 3 15.3 65.4 3.1X
+Native ORC Vectorized 4080 4090 11 3.9 259.4 0.8X
+Native ORC Vectorized (Pushdown) 1166 1172 7 13.5 74.1 2.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 50% decimal(18, 2) rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6161 6172 10 2.6 391.7 1.0X
-Parquet Vectorized (Pushdown) 4923 4957 51 3.2 313.0 1.3X
-Native ORC Vectorized 6961 6979 19 2.3 442.6 0.9X
-Native ORC Vectorized (Pushdown) 5280 5301 18 3.0 335.7 1.2X
+Parquet Vectorized 6104 6112 8 2.6 388.1 1.0X
+Parquet Vectorized (Pushdown) 4888 4905 18 3.2 310.8 1.2X
+Native ORC Vectorized 7257 7276 11 2.2 461.4 0.8X
+Native ORC Vectorized (Pushdown) 5610 5620 7 2.8 356.7 1.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 90% decimal(18, 2) rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 9004 9028 28 1.7 572.5 1.0X
-Parquet Vectorized (Pushdown) 8775 8790 11 1.8 557.9 1.0X
-Native ORC Vectorized 9794 9807 15 1.6 622.7 0.9X
-Native ORC Vectorized (Pushdown) 9490 9506 12 1.7 603.4 0.9X
+Parquet Vectorized 8832 8872 41 1.8 561.5 1.0X
+Parquet Vectorized (Pushdown) 8620 8628 7 1.8 548.0 1.0X
+Native ORC Vectorized 10412 10427 20 1.5 662.0 0.8X
+Native ORC Vectorized (Pushdown) 10104 10139 37 1.6 642.4 0.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 decimal(38, 2) row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 3780 3801 16 4.2 240.3 1.0X
-Parquet Vectorized (Pushdown) 79 98 22 200.0 5.0 48.1X
-Native ORC Vectorized 3288 3300 9 4.8 209.1 1.1X
-Native ORC Vectorized (Pushdown) 54 59 5 288.6 3.5 69.4X
+Parquet Vectorized 3829 3847 13 4.1 243.4 1.0X
+Parquet Vectorized (Pushdown) 75 84 5 210.2 4.8 51.2X
+Native ORC Vectorized 3304 3316 24 4.8 210.0 1.2X
+Native ORC Vectorized (Pushdown) 57 65 4 276.5 3.6 67.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 10% decimal(38, 2) rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 4732 4780 66 3.3 300.8 1.0X
-Parquet Vectorized (Pushdown) 1392 1404 8 11.3 88.5 3.4X
-Native ORC Vectorized 4156 4165 10 3.8 264.2 1.1X
-Native ORC Vectorized (Pushdown) 1234 1241 5 12.7 78.5 3.8X
+Parquet Vectorized 4667 4699 40 3.4 296.7 1.0X
+Parquet Vectorized (Pushdown) 1339 1344 4 11.7 85.1 3.5X
+Native ORC Vectorized 4215 4227 11 3.7 268.0 1.1X
+Native ORC Vectorized (Pushdown) 1286 1289 3 12.2 81.8 3.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 50% decimal(38, 2) rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 8479 8513 43 1.9 539.1 1.0X
-Parquet Vectorized (Pushdown) 6654 6669 23 2.4 423.0 1.3X
-Native ORC Vectorized 7628 7634 5 2.1 485.0 1.1X
-Native ORC Vectorized (Pushdown) 5957 5981 37 2.6 378.8 1.4X
+Parquet Vectorized 8329 8343 16 1.9 529.6 1.0X
+Parquet Vectorized (Pushdown) 6485 6499 11 2.4 412.3 1.3X
+Native ORC Vectorized 7903 7918 15 2.0 502.5 1.1X
+Native ORC Vectorized (Pushdown) 6234 6256 12 2.5 396.4 1.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 90% decimal(38, 2) rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 12128 12143 13 1.3 771.1 1.0X
-Parquet Vectorized (Pushdown) 11813 11820 11 1.3 751.0 1.0X
-Native ORC Vectorized 11005 11012 6 1.4 699.7 1.1X
-Native ORC Vectorized (Pushdown) 10691 10711 15 1.5 679.7 1.1X
+Parquet Vectorized 11910 11926 16 1.3 757.2 1.0X
+Parquet Vectorized (Pushdown) 11574 11583 9 1.4 735.9 1.0X
+Native ORC Vectorized 11496 11534 67 1.4 730.9 1.0X
+Native ORC Vectorized (Pushdown) 11211 11250 33 1.4 712.8 1.1X
================================================================================================
Pushdown benchmark for InSet -> InFilters
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
InSet -> InFilters (values count: 5, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6541 6622 78 2.4 415.9 1.0X
-Parquet Vectorized (Pushdown) 396 432 41 39.7 25.2 16.5X
-Native ORC Vectorized 4724 4750 20 3.3 300.3 1.4X
-Native ORC Vectorized (Pushdown) 325 350 29 48.4 20.7 20.1X
+Parquet Vectorized 6234 6271 28 2.5 396.3 1.0X
+Parquet Vectorized (Pushdown) 297 317 16 52.9 18.9 21.0X
+Native ORC Vectorized 4805 4847 38 3.3 305.5 1.3X
+Native ORC Vectorized (Pushdown) 315 320 5 49.9 20.0 19.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
InSet -> InFilters (values count: 5, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6231 6249 17 2.5 396.2 1.0X
-Parquet Vectorized (Pushdown) 303 316 8 51.9 19.3 20.6X
-Native ORC Vectorized 4716 4729 12 3.3 299.8 1.3X
-Native ORC Vectorized (Pushdown) 296 307 6 53.1 18.8 21.0X
+Parquet Vectorized 6125 6153 29 2.6 389.4 1.0X
+Parquet Vectorized (Pushdown) 283 289 3 55.6 18.0 21.6X
+Native ORC Vectorized 4766 4789 19 3.3 303.0 1.3X
+Native ORC Vectorized (Pushdown) 296 306 10 53.2 18.8 20.7X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
InSet -> InFilters (values count: 5, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6193 6206 14 2.5 393.7 1.0X
-Parquet Vectorized (Pushdown) 299 304 4 52.6 19.0 20.7X
-Native ORC Vectorized 4700 4736 32 3.3 298.8 1.3X
-Native ORC Vectorized (Pushdown) 292 309 19 53.9 18.5 21.2X
+Parquet Vectorized 6131 6144 19 2.6 389.8 1.0X
+Parquet Vectorized (Pushdown) 285 291 4 55.2 18.1 21.5X
+Native ORC Vectorized 4765 4783 14 3.3 302.9 1.3X
+Native ORC Vectorized (Pushdown) 298 302 3 52.9 18.9 20.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
InSet -> InFilters (values count: 10, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6192 6229 26 2.5 393.7 1.0X
-Parquet Vectorized (Pushdown) 314 347 63 50.1 20.0 19.7X
-Native ORC Vectorized 4746 4757 23 3.3 301.7 1.3X
-Native ORC Vectorized (Pushdown) 307 315 6 51.2 19.5 20.2X
+Parquet Vectorized 6124 6142 15 2.6 389.3 1.0X
+Parquet Vectorized (Pushdown) 300 304 3 52.5 19.1 20.4X
+Native ORC Vectorized 4780 4786 9 3.3 303.9 1.3X
+Native ORC Vectorized (Pushdown) 310 315 3 50.7 19.7 19.7X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
InSet -> InFilters (values count: 10, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6195 6204 9 2.5 393.9 1.0X
-Parquet Vectorized (Pushdown) 318 323 6 49.5 20.2 19.5X
-Native ORC Vectorized 4724 4790 96 3.3 300.4 1.3X
-Native ORC Vectorized (Pushdown) 305 311 8 51.5 19.4 20.3X
+Parquet Vectorized 6146 6183 36 2.6 390.8 1.0X
+Parquet Vectorized (Pushdown) 298 302 3 52.8 18.9 20.6X
+Native ORC Vectorized 4768 4790 15 3.3 303.1 1.3X
+Native ORC Vectorized (Pushdown) 308 313 2 51.0 19.6 19.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
InSet -> InFilters (values count: 10, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6215 6232 16 2.5 395.1 1.0X
-Parquet Vectorized (Pushdown) 310 317 7 50.7 19.7 20.0X
-Native ORC Vectorized 4738 4748 14 3.3 301.2 1.3X
-Native ORC Vectorized (Pushdown) 304 310 8 51.7 19.4 20.4X
+Parquet Vectorized 6120 6151 29 2.6 389.1 1.0X
+Parquet Vectorized (Pushdown) 297 302 4 52.9 18.9 20.6X
+Native ORC Vectorized 4746 4767 14 3.3 301.7 1.3X
+Native ORC Vectorized (Pushdown) 308 314 5 51.0 19.6 19.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
InSet -> InFilters (values count: 50, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6390 6421 23 2.5 406.3 1.0X
-Parquet Vectorized (Pushdown) 892 901 9 17.6 56.7 7.2X
-Native ORC Vectorized 4915 4928 12 3.2 312.5 1.3X
-Native ORC Vectorized (Pushdown) 413 421 7 38.1 26.2 15.5X
+Parquet Vectorized 6324 6337 15 2.5 402.1 1.0X
+Parquet Vectorized (Pushdown) 880 886 4 17.9 56.0 7.2X
+Native ORC Vectorized 4958 4972 10 3.2 315.2 1.3X
+Native ORC Vectorized (Pushdown) 412 416 2 38.2 26.2 15.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
InSet -> InFilters (values count: 50, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6392 6396 4 2.5 406.4 1.0X
-Parquet Vectorized (Pushdown) 3302 3315 10 4.8 209.9 1.9X
-Native ORC Vectorized 4916 4925 7 3.2 312.6 1.3X
-Native ORC Vectorized (Pushdown) 431 433 2 36.5 27.4 14.8X
+Parquet Vectorized 6319 6338 18 2.5 401.8 1.0X
+Parquet Vectorized (Pushdown) 3192 3199 8 4.9 202.9 2.0X
+Native ORC Vectorized 4954 4981 18 3.2 315.0 1.3X
+Native ORC Vectorized (Pushdown) 422 429 5 37.3 26.8 15.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
InSet -> InFilters (values count: 50, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6384 6399 11 2.5 405.9 1.0X
-Parquet Vectorized (Pushdown) 5735 5748 10 2.7 364.6 1.1X
-Native ORC Vectorized 4930 4937 6 3.2 313.5 1.3X
-Native ORC Vectorized (Pushdown) 425 429 3 37.0 27.0 15.0X
+Parquet Vectorized 6314 6337 18 2.5 401.4 1.0X
+Parquet Vectorized (Pushdown) 5356 5400 29 2.9 340.5 1.2X
+Native ORC Vectorized 4983 4994 17 3.2 316.8 1.3X
+Native ORC Vectorized (Pushdown) 438 444 3 35.9 27.9 14.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
InSet -> InFilters (values count: 100, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6355 6365 12 2.5 404.1 1.0X
-Parquet Vectorized (Pushdown) 879 883 4 17.9 55.9 7.2X
-Native ORC Vectorized 4881 4894 16 3.2 310.3 1.3X
-Native ORC Vectorized (Pushdown) 504 508 5 31.2 32.0 12.6X
+Parquet Vectorized 6330 6336 7 2.5 402.4 1.0X
+Parquet Vectorized (Pushdown) 874 879 4 18.0 55.6 7.2X
+Native ORC Vectorized 4910 4938 19 3.2 312.2 1.3X
+Native ORC Vectorized (Pushdown) 506 507 1 31.1 32.1 12.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
InSet -> InFilters (values count: 100, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6342 6360 16 2.5 403.2 1.0X
-Parquet Vectorized (Pushdown) 3308 3329 22 4.8 210.3 1.9X
-Native ORC Vectorized 4865 4886 16 3.2 309.3 1.3X
-Native ORC Vectorized (Pushdown) 550 552 2 28.6 34.9 11.5X
+Parquet Vectorized 6330 6338 7 2.5 402.4 1.0X
+Parquet Vectorized (Pushdown) 3244 3255 11 4.8 206.3 2.0X
+Native ORC Vectorized 4919 4939 14 3.2 312.8 1.3X
+Native ORC Vectorized (Pushdown) 579 588 11 27.2 36.8 10.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
InSet -> InFilters (values count: 100, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6352 6364 12 2.5 403.8 1.0X
-Parquet Vectorized (Pushdown) 5622 5644 25 2.8 357.4 1.1X
-Native ORC Vectorized 4888 4898 11 3.2 310.8 1.3X
-Native ORC Vectorized (Pushdown) 576 581 4 27.3 36.6 11.0X
+Parquet Vectorized 6332 6358 19 2.5 402.6 1.0X
+Parquet Vectorized (Pushdown) 5562 5579 21 2.8 353.6 1.1X
+Native ORC Vectorized 4927 4946 16 3.2 313.2 1.3X
+Native ORC Vectorized (Pushdown) 591 593 1 26.6 37.6 10.7X
================================================================================================
Pushdown benchmark for tinyint
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 tinyint row (value = CAST(63 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 2796 2818 23 5.6 177.8 1.0X
-Parquet Vectorized (Pushdown) 111 127 17 141.6 7.1 25.2X
-Native ORC Vectorized 2199 2223 17 7.2 139.8 1.3X
-Native ORC Vectorized (Pushdown) 118 135 16 133.8 7.5 23.8X
+Parquet Vectorized 2632 2651 13 6.0 167.3 1.0X
+Parquet Vectorized (Pushdown) 105 121 17 150.3 6.7 25.2X
+Native ORC Vectorized 2183 2201 13 7.2 138.8 1.2X
+Native ORC Vectorized (Pushdown) 120 138 20 130.8 7.6 21.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 10% tinyint rows (value < CAST(12 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 3387 3439 50 4.6 215.3 1.0X
-Parquet Vectorized (Pushdown) 995 1014 14 15.8 63.3 3.4X
-Native ORC Vectorized 2750 2769 15 5.7 174.8 1.2X
-Native ORC Vectorized (Pushdown) 888 909 26 17.7 56.5 3.8X
+Parquet Vectorized 3288 3347 55 4.8 209.1 1.0X
+Parquet Vectorized (Pushdown) 953 967 13 16.5 60.6 3.5X
+Native ORC Vectorized 2820 2827 7 5.6 179.3 1.2X
+Native ORC Vectorized (Pushdown) 937 942 3 16.8 59.6 3.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 50% tinyint rows (value < CAST(63 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6036 6059 24 2.6 383.8 1.0X
-Parquet Vectorized (Pushdown) 4750 4767 14 3.3 302.0 1.3X
-Native ORC Vectorized 5234 5255 18 3.0 332.8 1.2X
-Native ORC Vectorized (Pushdown) 4180 4191 19 3.8 265.8 1.4X
+Parquet Vectorized 5802 5841 53 2.7 368.9 1.0X
+Parquet Vectorized (Pushdown) 4542 4559 16 3.5 288.8 1.3X
+Native ORC Vectorized 5429 5445 11 2.9 345.2 1.1X
+Native ORC Vectorized (Pushdown) 4359 4373 12 3.6 277.1 1.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 90% tinyint rows (value < CAST(114 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 8795 8807 11 1.8 559.2 1.0X
-Parquet Vectorized (Pushdown) 8512 8539 23 1.8 541.2 1.0X
-Native ORC Vectorized 7798 7816 20 2.0 495.8 1.1X
-Native ORC Vectorized (Pushdown) 7622 7649 44 2.1 484.6 1.2X
+Parquet Vectorized 8235 8274 39 1.9 523.6 1.0X
+Parquet Vectorized (Pushdown) 8031 8051 16 2.0 510.6 1.0X
+Native ORC Vectorized 8040 8056 13 2.0 511.2 1.0X
+Native ORC Vectorized (Pushdown) 7876 7882 9 2.0 500.7 1.0X
================================================================================================
Pushdown benchmark for Timestamp
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 timestamp stored as INT96 row (value = timestamp_seconds(7864320)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 3139 3152 11 5.0 199.6 1.0X
-Parquet Vectorized (Pushdown) 3143 3149 8 5.0 199.8 1.0X
-Native ORC Vectorized 2025 2035 10 7.8 128.8 1.6X
-Native ORC Vectorized (Pushdown) 40 44 5 390.9 2.6 78.0X
+Parquet Vectorized 3011 3017 4 5.2 191.4 1.0X
+Parquet Vectorized (Pushdown) 3012 3030 27 5.2 191.5 1.0X
+Native ORC Vectorized 2018 2032 11 7.8 128.3 1.5X
+Native ORC Vectorized (Pushdown) 43 47 5 369.2 2.7 70.7X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 10% timestamp stored as INT96 rows (value < timestamp_seconds(1572864)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 3808 3822 16 4.1 242.1 1.0X
-Parquet Vectorized (Pushdown) 3800 3809 10 4.1 241.6 1.0X
-Native ORC Vectorized 2667 2681 19 5.9 169.6 1.4X
-Native ORC Vectorized (Pushdown) 879 885 5 17.9 55.9 4.3X
+Parquet Vectorized 3674 3710 49 4.3 233.6 1.0X
+Parquet Vectorized (Pushdown) 3683 3694 10 4.3 234.2 1.0X
+Native ORC Vectorized 2722 2740 19 5.8 173.1 1.3X
+Native ORC Vectorized (Pushdown) 942 946 4 16.7 59.9 3.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 50% timestamp stored as INT96 rows (value < timestamp_seconds(7864320)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6618 6632 11 2.4 420.8 1.0X
-Parquet Vectorized (Pushdown) 6637 6645 11 2.4 422.0 1.0X
-Native ORC Vectorized 5201 5215 18 3.0 330.7 1.3X
-Native ORC Vectorized (Pushdown) 4180 4189 6 3.8 265.8 1.6X
+Parquet Vectorized 6337 6368 20 2.5 402.9 1.0X
+Parquet Vectorized (Pushdown) 6358 6376 21 2.5 404.2 1.0X
+Native ORC Vectorized 5698 5721 15 2.8 362.3 1.1X
+Native ORC Vectorized (Pushdown) 4688 4703 13 3.4 298.0 1.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 90% timestamp stored as INT96 rows (value < timestamp_seconds(14155776)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 9362 9370 12 1.7 595.2 1.0X
-Parquet Vectorized (Pushdown) 9129 9137 14 1.7 580.4 1.0X
-Native ORC Vectorized 7670 7713 43 2.1 487.6 1.2X
-Native ORC Vectorized (Pushdown) 7481 7505 33 2.1 475.6 1.3X
+Parquet Vectorized 8881 8914 29 1.8 564.6 1.0X
+Parquet Vectorized (Pushdown) 8916 8933 13 1.8 566.8 1.0X
+Native ORC Vectorized 8438 8454 13 1.9 536.5 1.1X
+Native ORC Vectorized (Pushdown) 8251 8265 9 1.9 524.6 1.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 timestamp stored as TIMESTAMP_MICROS row (value = timestamp_seconds(7864320)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 2609 2626 22 6.0 165.9 1.0X
-Parquet Vectorized (Pushdown) 71 74 4 222.2 4.5 36.8X
-Native ORC Vectorized 2015 2029 14 7.8 128.1 1.3X
-Native ORC Vectorized (Pushdown) 40 43 4 397.2 2.5 65.9X
+Parquet Vectorized 2497 2508 10 6.3 158.8 1.0X
+Parquet Vectorized (Pushdown) 69 73 4 227.6 4.4 36.1X
+Native ORC Vectorized 2011 2015 5 7.8 127.9 1.2X
+Native ORC Vectorized (Pushdown) 42 44 3 377.1 2.7 59.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 10% timestamp stored as TIMESTAMP_MICROS rows (value < timestamp_seconds(1572864)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 3242 3252 9 4.9 206.1 1.0X
-Parquet Vectorized (Pushdown) 974 978 3 16.1 61.9 3.3X
-Native ORC Vectorized 2641 2650 6 6.0 167.9 1.2X
-Native ORC Vectorized (Pushdown) 867 868 1 18.2 55.1 3.7X
+Parquet Vectorized 3141 3146 6 5.0 199.7 1.0X
+Parquet Vectorized (Pushdown) 976 978 1 16.1 62.1 3.2X
+Native ORC Vectorized 2741 2747 5 5.7 174.3 1.1X
+Native ORC Vectorized (Pushdown) 940 944 2 16.7 59.8 3.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 50% timestamp stored as TIMESTAMP_MICROS rows (value < timestamp_seconds(7864320)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 5990 6015 19 2.6 380.9 1.0X
-Parquet Vectorized (Pushdown) 4747 4810 61 3.3 301.8 1.3X
-Native ORC Vectorized 5172 5193 15 3.0 328.8 1.2X
-Native ORC Vectorized (Pushdown) 4160 4166 5 3.8 264.5 1.4X
+Parquet Vectorized 5793 5833 45 2.7 368.3 1.0X
+Parquet Vectorized (Pushdown) 4601 4617 15 3.4 292.6 1.3X
+Native ORC Vectorized 5587 5601 18 2.8 355.2 1.0X
+Native ORC Vectorized (Pushdown) 4537 4551 11 3.5 288.4 1.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 90% timestamp stored as TIMESTAMP_MICROS rows (value < timestamp_seconds(14155776)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 8667 8710 31 1.8 551.1 1.0X
-Parquet Vectorized (Pushdown) 8479 8484 5 1.9 539.1 1.0X
-Native ORC Vectorized 7685 7720 37 2.0 488.6 1.1X
-Native ORC Vectorized (Pushdown) 7487 7530 50 2.1 476.0 1.2X
+Parquet Vectorized 8361 8369 8 1.9 531.6 1.0X
+Parquet Vectorized (Pushdown) 8155 8177 14 1.9 518.5 1.0X
+Native ORC Vectorized 8452 8461 12 1.9 537.3 1.0X
+Native ORC Vectorized (Pushdown) 8246 8267 14 1.9 524.3 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 timestamp stored as TIMESTAMP_MILLIS row (value = timestamp_seconds(7864320)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 2770 2777 5 5.7 176.1 1.0X
-Parquet Vectorized (Pushdown) 71 73 3 222.8 4.5 39.2X
-Native ORC Vectorized 2025 2036 11 7.8 128.7 1.4X
-Native ORC Vectorized (Pushdown) 40 44 4 394.9 2.5 69.5X
+Parquet Vectorized 2660 2661 2 5.9 169.1 1.0X
+Parquet Vectorized (Pushdown) 68 71 4 231.8 4.3 39.2X
+Native ORC Vectorized 2019 2023 6 7.8 128.3 1.3X
+Native ORC Vectorized (Pushdown) 41 44 4 380.7 2.6 64.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 10% timestamp stored as TIMESTAMP_MILLIS rows (value < timestamp_seconds(1572864)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 3428 3447 28 4.6 218.0 1.0X
-Parquet Vectorized (Pushdown) 1006 1011 4 15.6 64.0 3.4X
-Native ORC Vectorized 2667 2677 8 5.9 169.6 1.3X
-Native ORC Vectorized (Pushdown) 876 881 3 17.9 55.7 3.9X
+Parquet Vectorized 3312 3326 15 4.7 210.6 1.0X
+Parquet Vectorized (Pushdown) 991 994 4 15.9 63.0 3.3X
+Native ORC Vectorized 2738 2749 8 5.7 174.1 1.2X
+Native ORC Vectorized (Pushdown) 941 944 3 16.7 59.8 3.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 50% timestamp stored as TIMESTAMP_MILLIS rows (value < timestamp_seconds(7864320)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6140 6151 9 2.6 390.4 1.0X
-Parquet Vectorized (Pushdown) 4807 4813 5 3.3 305.6 1.3X
-Native ORC Vectorized 5154 5162 8 3.1 327.7 1.2X
-Native ORC Vectorized (Pushdown) 4122 4131 10 3.8 262.0 1.5X
+Parquet Vectorized 5947 5965 18 2.6 378.1 1.0X
+Parquet Vectorized (Pushdown) 4690 4707 14 3.4 298.2 1.3X
+Native ORC Vectorized 5579 5591 11 2.8 354.7 1.1X
+Native ORC Vectorized (Pushdown) 4552 4567 13 3.5 289.4 1.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 90% timestamp stored as TIMESTAMP_MILLIS rows (value < timestamp_seconds(14155776)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 8880 8900 13 1.8 564.6 1.0X
-Parquet Vectorized (Pushdown) 8616 8629 14 1.8 547.8 1.0X
-Native ORC Vectorized 7696 7735 46 2.0 489.3 1.2X
-Native ORC Vectorized (Pushdown) 7497 7510 9 2.1 476.7 1.2X
+Parquet Vectorized 8541 8556 11 1.8 543.0 1.0X
+Parquet Vectorized (Pushdown) 8321 8341 21 1.9 529.0 1.0X
+Native ORC Vectorized 8434 8456 20 1.9 536.2 1.0X
+Native ORC Vectorized (Pushdown) 8261 8268 12 1.9 525.2 1.0X
================================================================================================
Pushdown benchmark with many filters
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 row with 1 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 51 75 17 0.0 51336693.0 1.0X
-Parquet Vectorized (Pushdown) 52 54 3 0.0 51779493.0 1.0X
-Native ORC Vectorized 45 46 2 0.0 44817022.0 1.1X
-Native ORC Vectorized (Pushdown) 47 51 5 0.0 46799315.0 1.1X
+Parquet Vectorized 47 54 10 0.0 47073806.0 1.0X
+Parquet Vectorized (Pushdown) 48 51 3 0.0 48254597.0 1.0X
+Native ORC Vectorized 43 46 5 0.0 42518323.0 1.1X
+Native ORC Vectorized (Pushdown) 44 47 3 0.0 44456944.0 1.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 row with 250 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 170 181 9 0.0 170098519.0 1.0X
-Parquet Vectorized (Pushdown) 177 183 5 0.0 176637480.0 1.0X
-Native ORC Vectorized 166 169 3 0.0 165753639.0 1.0X
-Native ORC Vectorized (Pushdown) 167 173 5 0.0 167076087.0 1.0X
+Parquet Vectorized 170 178 7 0.0 170168877.0 1.0X
+Parquet Vectorized (Pushdown) 173 180 7 0.0 173290596.0 1.0X
+Native ORC Vectorized 165 173 6 0.0 164773308.0 1.0X
+Native ORC Vectorized (Pushdown) 165 171 5 0.0 165283905.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 row with 500 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 510 530 18 0.0 510147157.0 1.0X
-Parquet Vectorized (Pushdown) 518 539 19 0.0 517520226.0 1.0X
-Native ORC Vectorized 502 510 9 0.0 502346348.0 1.0X
-Native ORC Vectorized (Pushdown) 514 530 17 0.0 513941675.0 1.0X
+Parquet Vectorized 525 535 13 0.0 524597822.0 1.0X
+Parquet Vectorized (Pushdown) 535 552 16 0.0 534526800.0 1.0X
+Native ORC Vectorized 519 528 8 0.0 519431475.0 1.0X
+Native ORC Vectorized (Pushdown) 525 538 14 0.0 525191651.0 1.0X
diff --git a/sql/core/benchmarks/FilterPushdownBenchmark-results.txt b/sql/core/benchmarks/FilterPushdownBenchmark-results.txt
index 89eb049c3a0fa..a672109ce5844 100644
--- a/sql/core/benchmarks/FilterPushdownBenchmark-results.txt
+++ b/sql/core/benchmarks/FilterPushdownBenchmark-results.txt
@@ -2,733 +2,733 @@
Pushdown for many distinct value case
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 0 string row (value IS NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6856 6933 83 2.3 435.9 1.0X
-Parquet Vectorized (Pushdown) 331 338 7 47.5 21.0 20.7X
-Native ORC Vectorized 5021 5090 53 3.1 319.3 1.4X
-Native ORC Vectorized (Pushdown) 303 317 11 52.0 19.2 22.7X
+Parquet Vectorized 6883 6923 37 2.3 437.6 1.0X
+Parquet Vectorized (Pushdown) 335 349 11 46.9 21.3 20.5X
+Native ORC Vectorized 5462 5502 36 2.9 347.3 1.3X
+Native ORC Vectorized (Pushdown) 320 337 11 49.2 20.3 21.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 0 string row ('7864320' < value < '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6956 6978 20 2.3 442.2 1.0X
-Parquet Vectorized (Pushdown) 319 334 17 49.3 20.3 21.8X
-Native ORC Vectorized 5161 5171 11 3.0 328.1 1.3X
-Native ORC Vectorized (Pushdown) 310 326 18 50.8 19.7 22.5X
+Parquet Vectorized 6943 6965 22 2.3 441.4 1.0X
+Parquet Vectorized (Pushdown) 335 359 18 46.9 21.3 20.7X
+Native ORC Vectorized 5559 5582 17 2.8 353.4 1.2X
+Native ORC Vectorized (Pushdown) 327 335 6 48.1 20.8 21.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 string row (value = '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6911 6952 39 2.3 439.4 1.0X
-Parquet Vectorized (Pushdown) 302 313 10 52.2 19.2 22.9X
-Native ORC Vectorized 5121 5145 23 3.1 325.6 1.3X
-Native ORC Vectorized (Pushdown) 295 303 7 53.3 18.8 23.4X
+Parquet Vectorized 6928 6941 14 2.3 440.4 1.0X
+Parquet Vectorized (Pushdown) 317 327 7 49.6 20.1 21.9X
+Native ORC Vectorized 5571 5591 26 2.8 354.2 1.2X
+Native ORC Vectorized (Pushdown) 313 320 9 50.3 19.9 22.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 string row (value <=> '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6908 6916 8 2.3 439.2 1.0X
-Parquet Vectorized (Pushdown) 293 306 11 53.7 18.6 23.6X
-Native ORC Vectorized 5113 5133 16 3.1 325.1 1.4X
-Native ORC Vectorized (Pushdown) 289 295 3 54.4 18.4 23.9X
+Parquet Vectorized 6899 6914 12 2.3 438.6 1.0X
+Parquet Vectorized (Pushdown) 310 327 15 50.8 19.7 22.3X
+Native ORC Vectorized 5547 5557 6 2.8 352.7 1.2X
+Native ORC Vectorized (Pushdown) 303 309 4 51.9 19.3 22.8X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 string row ('7864320' <= value <= '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6953 6979 24 2.3 442.1 1.0X
-Parquet Vectorized (Pushdown) 300 305 6 52.4 19.1 23.1X
-Native ORC Vectorized 5147 5175 25 3.1 327.2 1.4X
-Native ORC Vectorized (Pushdown) 296 303 7 53.1 18.8 23.5X
+Parquet Vectorized 6944 6971 41 2.3 441.5 1.0X
+Parquet Vectorized (Pushdown) 309 320 11 50.8 19.7 22.4X
+Native ORC Vectorized 5534 5554 22 2.8 351.8 1.3X
+Native ORC Vectorized (Pushdown) 307 326 16 51.2 19.5 22.6X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select all string rows (value IS NOT NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 14342 14384 46 1.1 911.9 1.0X
-Parquet Vectorized (Pushdown) 14364 14404 29 1.1 913.2 1.0X
-Native ORC Vectorized 12523 12580 38 1.3 796.2 1.1X
-Native ORC Vectorized (Pushdown) 12619 12644 24 1.2 802.3 1.1X
+Parquet Vectorized 14452 14519 48 1.1 918.9 1.0X
+Parquet Vectorized (Pushdown) 14521 14546 19 1.1 923.2 1.0X
+Native ORC Vectorized 13001 13039 35 1.2 826.6 1.1X
+Native ORC Vectorized (Pushdown) 13112 13137 36 1.2 833.6 1.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 0 int row (value IS NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6539 6556 14 2.4 415.7 1.0X
-Parquet Vectorized (Pushdown) 274 280 6 57.5 17.4 23.9X
-Native ORC Vectorized 4657 4676 14 3.4 296.1 1.4X
-Native ORC Vectorized (Pushdown) 277 281 5 56.8 17.6 23.6X
+Parquet Vectorized 6546 6562 15 2.4 416.2 1.0X
+Parquet Vectorized (Pushdown) 292 301 7 53.8 18.6 22.4X
+Native ORC Vectorized 5044 5050 7 3.1 320.7 1.3X
+Native ORC Vectorized (Pushdown) 288 297 5 54.6 18.3 22.7X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 0 int row (7864320 < value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6488 6520 39 2.4 412.5 1.0X
-Parquet Vectorized (Pushdown) 279 287 5 56.3 17.8 23.2X
-Native ORC Vectorized 4664 4677 13 3.4 296.5 1.4X
-Native ORC Vectorized (Pushdown) 282 287 4 55.8 17.9 23.0X
+Parquet Vectorized 6530 6539 12 2.4 415.2 1.0X
+Parquet Vectorized (Pushdown) 301 306 4 52.2 19.1 21.7X
+Native ORC Vectorized 5040 5057 15 3.1 320.4 1.3X
+Native ORC Vectorized (Pushdown) 294 304 11 53.5 18.7 22.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 int row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6538 6546 7 2.4 415.7 1.0X
-Parquet Vectorized (Pushdown) 278 283 5 56.6 17.7 23.5X
-Native ORC Vectorized 4742 4764 20 3.3 301.5 1.4X
-Native ORC Vectorized (Pushdown) 282 288 7 55.8 17.9 23.2X
+Parquet Vectorized 6543 6555 17 2.4 416.0 1.0X
+Parquet Vectorized (Pushdown) 298 305 7 52.8 18.9 22.0X
+Native ORC Vectorized 5084 5093 6 3.1 323.2 1.3X
+Native ORC Vectorized (Pushdown) 290 298 10 54.2 18.5 22.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 int row (value <=> 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6544 6573 21 2.4 416.0 1.0X
-Parquet Vectorized (Pushdown) 276 286 7 56.9 17.6 23.7X
-Native ORC Vectorized 4709 4749 56 3.3 299.4 1.4X
-Native ORC Vectorized (Pushdown) 276 283 4 57.1 17.5 23.7X
+Parquet Vectorized 6550 6583 46 2.4 416.5 1.0X
+Parquet Vectorized (Pushdown) 293 298 3 53.6 18.6 22.3X
+Native ORC Vectorized 5081 5093 12 3.1 323.0 1.3X
+Native ORC Vectorized (Pushdown) 285 298 12 55.2 18.1 23.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 int row (7864320 <= value <= 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6532 6545 8 2.4 415.3 1.0X
-Parquet Vectorized (Pushdown) 277 279 2 56.8 17.6 23.6X
-Native ORC Vectorized 4704 4731 28 3.3 299.0 1.4X
-Native ORC Vectorized (Pushdown) 282 287 4 55.8 17.9 23.2X
+Parquet Vectorized 6546 6574 17 2.4 416.2 1.0X
+Parquet Vectorized (Pushdown) 297 302 3 52.9 18.9 22.0X
+Native ORC Vectorized 5072 5079 4 3.1 322.5 1.3X
+Native ORC Vectorized (Pushdown) 289 296 6 54.4 18.4 22.6X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 int row (7864319 < value < 7864321): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6538 6558 23 2.4 415.7 1.0X
-Parquet Vectorized (Pushdown) 275 287 10 57.2 17.5 23.8X
-Native ORC Vectorized 4723 4749 25 3.3 300.3 1.4X
-Native ORC Vectorized (Pushdown) 278 282 3 56.6 17.7 23.5X
+Parquet Vectorized 6535 6575 43 2.4 415.5 1.0X
+Parquet Vectorized (Pushdown) 295 299 3 53.4 18.7 22.2X
+Native ORC Vectorized 5070 5074 2 3.1 322.3 1.3X
+Native ORC Vectorized (Pushdown) 289 294 3 54.4 18.4 22.6X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 10% int rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 7258 7290 37 2.2 461.5 1.0X
-Parquet Vectorized (Pushdown) 1623 1633 8 9.7 103.2 4.5X
-Native ORC Vectorized 5374 5380 5 2.9 341.6 1.4X
-Native ORC Vectorized (Pushdown) 1414 1420 6 11.1 89.9 5.1X
+Parquet Vectorized 7234 7253 18 2.2 459.9 1.0X
+Parquet Vectorized (Pushdown) 1629 1633 5 9.7 103.6 4.4X
+Native ORC Vectorized 5766 5771 5 2.7 366.6 1.3X
+Native ORC Vectorized (Pushdown) 1467 1471 4 10.7 93.3 4.9X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 50% int rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 10004 10038 29 1.6 636.0 1.0X
-Parquet Vectorized (Pushdown) 6892 6906 11 2.3 438.2 1.5X
-Native ORC Vectorized 8061 8078 15 2.0 512.5 1.2X
-Native ORC Vectorized (Pushdown) 5897 5910 12 2.7 374.9 1.7X
+Parquet Vectorized 9845 9864 22 1.6 626.0 1.0X
+Parquet Vectorized (Pushdown) 6755 6762 11 2.3 429.5 1.5X
+Native ORC Vectorized 8373 8380 8 1.9 532.4 1.2X
+Native ORC Vectorized (Pushdown) 6027 6041 15 2.6 383.2 1.6X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 90% int rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 12533 12571 35 1.3 796.9 1.0X
-Parquet Vectorized (Pushdown) 11864 11902 33 1.3 754.3 1.1X
-Native ORC Vectorized 10727 10812 85 1.5 682.0 1.2X
-Native ORC Vectorized (Pushdown) 10366 10408 35 1.5 659.1 1.2X
+Parquet Vectorized 12382 12396 13 1.3 787.2 1.0X
+Parquet Vectorized (Pushdown) 11785 11802 11 1.3 749.3 1.1X
+Native ORC Vectorized 10944 10975 25 1.4 695.8 1.1X
+Native ORC Vectorized (Pushdown) 10555 10561 9 1.5 671.1 1.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select all int rows (value IS NOT NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 13484 13490 7 1.2 857.3 1.0X
-Parquet Vectorized (Pushdown) 13500 13533 22 1.2 858.3 1.0X
-Native ORC Vectorized 11483 11532 39 1.4 730.0 1.2X
-Native ORC Vectorized (Pushdown) 11477 11592 107 1.4 729.7 1.2X
+Parquet Vectorized 13150 13174 22 1.2 836.1 1.0X
+Parquet Vectorized (Pushdown) 13194 13220 20 1.2 838.9 1.0X
+Native ORC Vectorized 11537 11552 14 1.4 733.5 1.1X
+Native ORC Vectorized (Pushdown) 11622 11639 15 1.4 738.9 1.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select all int rows (value > -1): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 13032 13061 20 1.2 828.6 1.0X
-Parquet Vectorized (Pushdown) 13076 13100 19 1.2 831.4 1.0X
-Native ORC Vectorized 11456 11486 47 1.4 728.3 1.1X
-Native ORC Vectorized (Pushdown) 11538 11569 25 1.4 733.5 1.1X
+Parquet Vectorized 13086 13099 11 1.2 832.0 1.0X
+Parquet Vectorized (Pushdown) 13137 13156 15 1.2 835.2 1.0X
+Native ORC Vectorized 11614 11630 20 1.4 738.4 1.1X
+Native ORC Vectorized (Pushdown) 11685 11702 18 1.3 742.9 1.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select all int rows (value != -1): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 13201 13227 23 1.2 839.3 1.0X
-Parquet Vectorized (Pushdown) 13322 13336 24 1.2 847.0 1.0X
-Native ORC Vectorized 11276 11304 19 1.4 716.9 1.2X
-Native ORC Vectorized (Pushdown) 11397 11430 41 1.4 724.6 1.2X
+Parquet Vectorized 13062 13077 11 1.2 830.5 1.0X
+Parquet Vectorized (Pushdown) 13127 13151 27 1.2 834.6 1.0X
+Native ORC Vectorized 11692 11786 192 1.3 743.4 1.1X
+Native ORC Vectorized (Pushdown) 11803 11846 41 1.3 750.4 1.1X
================================================================================================
Pushdown for few distinct value case (use dictionary encoding)
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 0 distinct string row (value IS NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 5934 5955 21 2.7 377.3 1.0X
-Parquet Vectorized (Pushdown) 240 249 10 65.6 15.3 24.7X
-Native ORC Vectorized 6197 6233 41 2.5 394.0 1.0X
-Native ORC Vectorized (Pushdown) 924 937 9 17.0 58.8 6.4X
+Parquet Vectorized 5890 5919 34 2.7 374.5 1.0X
+Parquet Vectorized (Pushdown) 240 244 3 65.5 15.3 24.5X
+Native ORC Vectorized 6629 6641 12 2.4 421.5 0.9X
+Native ORC Vectorized (Pushdown) 952 962 8 16.5 60.5 6.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 0 distinct string row ('100' < value < '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6043 6055 9 2.6 384.2 1.0X
-Parquet Vectorized (Pushdown) 243 248 6 64.8 15.4 24.9X
-Native ORC Vectorized 6428 6455 19 2.4 408.7 0.9X
-Native ORC Vectorized (Pushdown) 930 938 7 16.9 59.1 6.5X
+Parquet Vectorized 6043 6053 11 2.6 384.2 1.0X
+Parquet Vectorized (Pushdown) 242 245 2 64.9 15.4 24.9X
+Native ORC Vectorized 6856 6867 8 2.3 435.9 0.9X
+Native ORC Vectorized (Pushdown) 960 967 5 16.4 61.1 6.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 distinct string row (value = '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 5954 5969 13 2.6 378.5 1.0X
-Parquet Vectorized (Pushdown) 290 294 3 54.3 18.4 20.6X
-Native ORC Vectorized 6362 6379 14 2.5 404.5 0.9X
-Native ORC Vectorized (Pushdown) 957 975 18 16.4 60.9 6.2X
+Parquet Vectorized 5951 5984 19 2.6 378.4 1.0X
+Parquet Vectorized (Pushdown) 292 296 4 53.8 18.6 20.4X
+Native ORC Vectorized 6823 7080 199 2.3 433.8 0.9X
+Native ORC Vectorized (Pushdown) 1008 1042 26 15.6 64.1 5.9X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 distinct string row (value <=> '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 5962 5984 24 2.6 379.0 1.0X
-Parquet Vectorized (Pushdown) 289 295 7 54.4 18.4 20.6X
-Native ORC Vectorized 6353 6382 17 2.5 403.9 0.9X
-Native ORC Vectorized (Pushdown) 952 963 13 16.5 60.5 6.3X
+Parquet Vectorized 5941 5958 11 2.6 377.7 1.0X
+Parquet Vectorized (Pushdown) 292 296 4 53.9 18.5 20.4X
+Native ORC Vectorized 6766 7201 429 2.3 430.2 0.9X
+Native ORC Vectorized (Pushdown) 1333 1380 41 11.8 84.7 4.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 distinct string row ('100' <= value <= '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6008 6017 8 2.6 382.0 1.0X
-Parquet Vectorized (Pushdown) 290 297 5 54.2 18.5 20.7X
-Native ORC Vectorized 6437 6457 17 2.4 409.3 0.9X
-Native ORC Vectorized (Pushdown) 972 975 2 16.2 61.8 6.2X
+Parquet Vectorized 6076 6112 22 2.6 386.3 1.0X
+Parquet Vectorized (Pushdown) 301 312 8 52.3 19.1 20.2X
+Native ORC Vectorized 8039 8099 56 2.0 511.1 0.8X
+Native ORC Vectorized (Pushdown) 1339 1380 50 11.7 85.2 4.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select all distinct string rows (value IS NOT NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 14051 14073 29 1.1 893.3 1.0X
-Parquet Vectorized (Pushdown) 14119 14153 26 1.1 897.7 1.0X
-Native ORC Vectorized 14169 14225 54 1.1 900.8 1.0X
-Native ORC Vectorized (Pushdown) 14404 14427 28 1.1 915.8 1.0X
+Parquet Vectorized 14721 14761 24 1.1 935.9 1.0X
+Parquet Vectorized (Pushdown) 14796 14846 70 1.1 940.7 1.0X
+Native ORC Vectorized 16834 17184 208 0.9 1070.3 0.9X
+Native ORC Vectorized (Pushdown) 17994 18093 111 0.9 1144.0 0.8X
================================================================================================
Pushdown benchmark for StringStartsWith
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
StringStartsWith filter: (value like '10%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 7175 7182 6 2.2 456.2 1.0X
-Parquet Vectorized (Pushdown) 944 950 7 16.7 60.0 7.6X
-Native ORC Vectorized 5261 5285 15 3.0 334.5 1.4X
-Native ORC Vectorized (Pushdown) 5356 5363 11 2.9 340.5 1.3X
+Parquet Vectorized 7173 7238 68 2.2 456.1 1.0X
+Parquet Vectorized (Pushdown) 942 946 3 16.7 59.9 7.6X
+Native ORC Vectorized 5658 5672 9 2.8 359.7 1.3X
+Native ORC Vectorized (Pushdown) 5758 5774 20 2.7 366.1 1.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
StringStartsWith filter: (value like '1000%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6984 6993 10 2.3 444.1 1.0X
-Parquet Vectorized (Pushdown) 277 284 5 56.8 17.6 25.2X
-Native ORC Vectorized 5115 5154 32 3.1 325.2 1.4X
-Native ORC Vectorized (Pushdown) 5195 5217 13 3.0 330.3 1.3X
+Parquet Vectorized 7001 7010 11 2.2 445.1 1.0X
+Parquet Vectorized (Pushdown) 283 286 3 55.6 18.0 24.7X
+Native ORC Vectorized 5551 5686 146 2.8 352.9 1.3X
+Native ORC Vectorized (Pushdown) 5627 5640 16 2.8 357.7 1.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
StringStartsWith filter: (value like '786432%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6979 6997 12 2.3 443.7 1.0X
-Parquet Vectorized (Pushdown) 274 280 9 57.3 17.4 25.4X
-Native ORC Vectorized 5119 5136 15 3.1 325.5 1.4X
-Native ORC Vectorized (Pushdown) 5198 5208 12 3.0 330.5 1.3X
+Parquet Vectorized 6988 7047 59 2.3 444.3 1.0X
+Parquet Vectorized (Pushdown) 277 284 6 56.7 17.6 25.2X
+Native ORC Vectorized 5540 5572 25 2.8 352.2 1.3X
+Native ORC Vectorized (Pushdown) 5590 5668 138 2.8 355.4 1.2X
================================================================================================
Pushdown benchmark for StringEndsWith
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
StringEndsWith filter: (value like '%10'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 5973 5987 14 2.6 379.7 1.0X
-Parquet Vectorized (Pushdown) 370 382 16 42.5 23.5 16.1X
-Native ORC Vectorized 6351 6383 32 2.5 403.8 0.9X
-Native ORC Vectorized (Pushdown) 6544 6582 25 2.4 416.0 0.9X
+Parquet Vectorized 5945 5962 14 2.6 378.0 1.0X
+Parquet Vectorized (Pushdown) 366 372 5 42.9 23.3 16.2X
+Native ORC Vectorized 6752 6760 8 2.3 429.3 0.9X
+Native ORC Vectorized (Pushdown) 6960 6970 10 2.3 442.5 0.9X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
StringEndsWith filter: (value like '%1000'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 5923 5933 8 2.7 376.6 1.0X
-Parquet Vectorized (Pushdown) 275 283 9 57.2 17.5 21.5X
-Native ORC Vectorized 6303 6319 15 2.5 400.8 0.9X
-Native ORC Vectorized (Pushdown) 6516 6534 23 2.4 414.3 0.9X
+Parquet Vectorized 5870 5880 8 2.7 373.2 1.0X
+Parquet Vectorized (Pushdown) 270 274 3 58.2 17.2 21.7X
+Native ORC Vectorized 6702 6714 7 2.3 426.1 0.9X
+Native ORC Vectorized (Pushdown) 6889 6899 7 2.3 438.0 0.9X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
StringEndsWith filter: (value like '%786432'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 5917 5931 12 2.7 376.2 1.0X
-Parquet Vectorized (Pushdown) 273 277 5 57.6 17.4 21.7X
-Native ORC Vectorized 6287 6313 29 2.5 399.7 0.9X
-Native ORC Vectorized (Pushdown) 6497 6528 39 2.4 413.1 0.9X
+Parquet Vectorized 5875 5884 9 2.7 373.5 1.0X
+Parquet Vectorized (Pushdown) 273 275 2 57.7 17.3 21.6X
+Native ORC Vectorized 6701 6719 15 2.3 426.0 0.9X
+Native ORC Vectorized (Pushdown) 6898 6907 14 2.3 438.5 0.9X
================================================================================================
Pushdown benchmark for StringContains
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
StringContains filter: (value like '%10%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6150 6166 17 2.6 391.0 1.0X
-Parquet Vectorized (Pushdown) 807 812 6 19.5 51.3 7.6X
-Native ORC Vectorized 6520 6534 11 2.4 414.5 0.9X
-Native ORC Vectorized (Pushdown) 6732 6769 43 2.3 428.0 0.9X
+Parquet Vectorized 6133 6149 15 2.6 390.0 1.0X
+Parquet Vectorized (Pushdown) 800 804 2 19.7 50.9 7.7X
+Native ORC Vectorized 6956 6979 21 2.3 442.3 0.9X
+Native ORC Vectorized (Pushdown) 7151 7176 30 2.2 454.6 0.9X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
StringContains filter: (value like '%1000%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 5929 5947 22 2.7 376.9 1.0X
-Parquet Vectorized (Pushdown) 271 274 2 58.0 17.2 21.9X
-Native ORC Vectorized 6293 6322 18 2.5 400.1 0.9X
-Native ORC Vectorized (Pushdown) 6524 6551 23 2.4 414.8 0.9X
+Parquet Vectorized 5890 5907 30 2.7 374.5 1.0X
+Parquet Vectorized (Pushdown) 273 275 2 57.6 17.4 21.6X
+Native ORC Vectorized 6734 6755 18 2.3 428.2 0.9X
+Native ORC Vectorized (Pushdown) 6939 6969 40 2.3 441.2 0.8X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
StringContains filter: (value like '%786432%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 5928 5940 12 2.7 376.9 1.0X
-Parquet Vectorized (Pushdown) 270 276 4 58.2 17.2 21.9X
-Native ORC Vectorized 6315 6331 14 2.5 401.5 0.9X
-Native ORC Vectorized (Pushdown) 6541 6560 24 2.4 415.9 0.9X
+Parquet Vectorized 5903 5912 7 2.7 375.3 1.0X
+Parquet Vectorized (Pushdown) 272 277 7 57.8 17.3 21.7X
+Native ORC Vectorized 6750 6754 5 2.3 429.2 0.9X
+Native ORC Vectorized (Pushdown) 6959 6964 3 2.3 442.4 0.8X
================================================================================================
Pushdown benchmark for decimal
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 decimal(9, 2) row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 2844 2858 11 5.5 180.8 1.0X
-Parquet Vectorized (Pushdown) 71 75 4 220.7 4.5 39.9X
-Native ORC Vectorized 3269 3284 13 4.8 207.8 0.9X
-Native ORC Vectorized (Pushdown) 57 62 5 277.1 3.6 50.1X
+Parquet Vectorized 2841 2858 10 5.5 180.6 1.0X
+Parquet Vectorized (Pushdown) 72 75 3 218.2 4.6 39.4X
+Native ORC Vectorized 3436 3446 11 4.6 218.5 0.8X
+Native ORC Vectorized (Pushdown) 60 63 5 261.1 3.8 47.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 10% decimal(9, 2) rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 4167 4190 17 3.8 264.9 1.0X
-Parquet Vectorized (Pushdown) 1956 1963 7 8.0 124.3 2.1X
-Native ORC Vectorized 4702 4733 34 3.3 298.9 0.9X
-Native ORC Vectorized (Pushdown) 2072 2081 8 7.6 131.8 2.0X
+Parquet Vectorized 4213 4222 9 3.7 267.9 1.0X
+Parquet Vectorized (Pushdown) 1983 1995 12 7.9 126.1 2.1X
+Native ORC Vectorized 4828 4877 49 3.3 307.0 0.9X
+Native ORC Vectorized (Pushdown) 2110 2113 2 7.5 134.2 2.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 50% decimal(9, 2) rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 9042 9066 19 1.7 574.9 1.0X
-Parquet Vectorized (Pushdown) 8702 8717 13 1.8 553.2 1.0X
-Native ORC Vectorized 9335 9389 60 1.7 593.5 1.0X
-Native ORC Vectorized (Pushdown) 8897 8916 17 1.8 565.7 1.0X
+Parquet Vectorized 8728 8781 48 1.8 554.9 1.0X
+Parquet Vectorized (Pushdown) 8352 8367 10 1.9 531.0 1.0X
+Native ORC Vectorized 9324 9363 72 1.7 592.8 0.9X
+Native ORC Vectorized (Pushdown) 8880 8907 21 1.8 564.6 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 90% decimal(9, 2) rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 10057 10077 30 1.6 639.4 1.0X
-Parquet Vectorized (Pushdown) 10085 10114 25 1.6 641.2 1.0X
-Native ORC Vectorized 10713 10729 23 1.5 681.1 0.9X
-Native ORC Vectorized (Pushdown) 10724 10732 8 1.5 681.8 0.9X
+Parquet Vectorized 9920 9933 13 1.6 630.7 1.0X
+Parquet Vectorized (Pushdown) 9939 9967 37 1.6 631.9 1.0X
+Native ORC Vectorized 10375 10402 31 1.5 659.7 1.0X
+Native ORC Vectorized (Pushdown) 10399 10412 12 1.5 661.2 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 decimal(18, 2) row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 3024 3051 16 5.2 192.3 1.0X
-Parquet Vectorized (Pushdown) 70 73 3 224.1 4.5 43.1X
-Native ORC Vectorized 3265 3286 21 4.8 207.6 0.9X
-Native ORC Vectorized (Pushdown) 55 59 3 288.1 3.5 55.4X
+Parquet Vectorized 3006 3036 18 5.2 191.1 1.0X
+Parquet Vectorized (Pushdown) 70 71 3 226.0 4.4 43.2X
+Native ORC Vectorized 3412 3426 18 4.6 216.9 0.9X
+Native ORC Vectorized (Pushdown) 57 61 4 273.9 3.7 52.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 10% decimal(18, 2) rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 3784 3803 19 4.2 240.6 1.0X
-Parquet Vectorized (Pushdown) 1083 1086 2 14.5 68.9 3.5X
-Native ORC Vectorized 4014 4036 21 3.9 255.2 0.9X
-Native ORC Vectorized (Pushdown) 1060 1065 5 14.8 67.4 3.6X
+Parquet Vectorized 3776 3787 9 4.2 240.1 1.0X
+Parquet Vectorized (Pushdown) 1081 1089 5 14.5 68.7 3.5X
+Native ORC Vectorized 4137 4149 20 3.8 263.0 0.9X
+Native ORC Vectorized (Pushdown) 1103 1106 4 14.3 70.1 3.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 50% decimal(18, 2) rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6657 6670 11 2.4 423.2 1.0X
-Parquet Vectorized (Pushdown) 5177 5188 7 3.0 329.1 1.3X
-Native ORC Vectorized 6809 6821 11 2.3 432.9 1.0X
-Native ORC Vectorized (Pushdown) 5138 5159 21 3.1 326.6 1.3X
+Parquet Vectorized 6676 6690 11 2.4 424.5 1.0X
+Parquet Vectorized (Pushdown) 5195 5201 5 3.0 330.3 1.3X
+Native ORC Vectorized 6988 6997 8 2.3 444.3 1.0X
+Native ORC Vectorized (Pushdown) 5310 5313 3 3.0 337.6 1.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 90% decimal(18, 2) rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 9548 9572 19 1.6 607.0 1.0X
-Parquet Vectorized (Pushdown) 9291 9322 27 1.7 590.7 1.0X
-Native ORC Vectorized 9610 9623 13 1.6 611.0 1.0X
-Native ORC Vectorized (Pushdown) 9260 9297 30 1.7 588.7 1.0X
+Parquet Vectorized 9514 9520 9 1.7 604.9 1.0X
+Parquet Vectorized (Pushdown) 9238 9281 48 1.7 587.3 1.0X
+Native ORC Vectorized 9832 9868 23 1.6 625.1 1.0X
+Native ORC Vectorized (Pushdown) 9492 9499 5 1.7 603.5 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 decimal(38, 2) row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 4320 4337 10 3.6 274.7 1.0X
-Parquet Vectorized (Pushdown) 78 82 5 201.0 5.0 55.2X
-Native ORC Vectorized 3340 3363 21 4.7 212.4 1.3X
-Native ORC Vectorized (Pushdown) 55 57 2 285.9 3.5 78.5X
+Parquet Vectorized 4281 4291 8 3.7 272.2 1.0X
+Parquet Vectorized (Pushdown) 76 79 3 206.4 4.8 56.2X
+Native ORC Vectorized 3458 3474 17 4.5 219.8 1.2X
+Native ORC Vectorized (Pushdown) 58 60 3 273.4 3.7 74.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 10% decimal(38, 2) rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 5208 5229 32 3.0 331.1 1.0X
-Parquet Vectorized (Pushdown) 1386 1393 9 11.3 88.1 3.8X
-Native ORC Vectorized 4160 4198 56 3.8 264.5 1.3X
-Native ORC Vectorized (Pushdown) 1187 1194 9 13.3 75.5 4.4X
+Parquet Vectorized 5238 5246 8 3.0 333.0 1.0X
+Parquet Vectorized (Pushdown) 1422 1424 1 11.1 90.4 3.7X
+Native ORC Vectorized 4308 4321 15 3.7 273.9 1.2X
+Native ORC Vectorized (Pushdown) 1227 1230 3 12.8 78.0 4.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 50% decimal(38, 2) rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 8789 8818 34 1.8 558.8 1.0X
-Parquet Vectorized (Pushdown) 6686 6697 13 2.4 425.1 1.3X
-Native ORC Vectorized 7476 7500 26 2.1 475.3 1.2X
-Native ORC Vectorized (Pushdown) 5812 5837 37 2.7 369.5 1.5X
+Parquet Vectorized 8951 8969 18 1.8 569.1 1.0X
+Parquet Vectorized (Pushdown) 6863 6900 29 2.3 436.4 1.3X
+Native ORC Vectorized 7606 7620 10 2.1 483.6 1.2X
+Native ORC Vectorized (Pushdown) 5900 5906 7 2.7 375.1 1.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 90% decimal(38, 2) rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 12304 12358 67 1.3 782.2 1.0X
-Parquet Vectorized (Pushdown) 11966 12032 48 1.3 760.8 1.0X
-Native ORC Vectorized 10772 10813 30 1.5 684.9 1.1X
-Native ORC Vectorized (Pushdown) 10441 10482 35 1.5 663.8 1.2X
+Parquet Vectorized 12623 12678 67 1.2 802.5 1.0X
+Parquet Vectorized (Pushdown) 12207 12224 16 1.3 776.1 1.0X
+Native ORC Vectorized 10898 10906 5 1.4 692.9 1.2X
+Native ORC Vectorized (Pushdown) 10575 10588 11 1.5 672.3 1.2X
================================================================================================
Pushdown benchmark for InSet -> InFilters
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
InSet -> InFilters (values count: 5, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6578 6604 16 2.4 418.2 1.0X
-Parquet Vectorized (Pushdown) 294 298 4 53.5 18.7 22.4X
-Native ORC Vectorized 4871 4897 29 3.2 309.7 1.4X
-Native ORC Vectorized (Pushdown) 286 292 4 54.9 18.2 23.0X
+Parquet Vectorized 6516 6546 29 2.4 414.3 1.0X
+Parquet Vectorized (Pushdown) 289 292 4 54.5 18.4 22.6X
+Native ORC Vectorized 4902 4907 4 3.2 311.7 1.3X
+Native ORC Vectorized (Pushdown) 296 304 10 53.2 18.8 22.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
InSet -> InFilters (values count: 5, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6590 6598 7 2.4 419.0 1.0X
-Parquet Vectorized (Pushdown) 294 303 9 53.5 18.7 22.4X
-Native ORC Vectorized 4778 4800 25 3.3 303.8 1.4X
-Native ORC Vectorized (Pushdown) 283 286 2 55.6 18.0 23.3X
+Parquet Vectorized 6532 6549 13 2.4 415.3 1.0X
+Parquet Vectorized (Pushdown) 285 287 2 55.2 18.1 22.9X
+Native ORC Vectorized 4884 4887 3 3.2 310.5 1.3X
+Native ORC Vectorized (Pushdown) 292 300 8 53.8 18.6 22.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
InSet -> InFilters (values count: 5, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6532 6560 29 2.4 415.3 1.0X
-Parquet Vectorized (Pushdown) 289 294 5 54.5 18.4 22.6X
-Native ORC Vectorized 4769 4784 11 3.3 303.2 1.4X
-Native ORC Vectorized (Pushdown) 283 285 2 55.6 18.0 23.1X
+Parquet Vectorized 6524 6531 7 2.4 414.8 1.0X
+Parquet Vectorized (Pushdown) 287 291 5 54.8 18.2 22.7X
+Native ORC Vectorized 4897 4912 16 3.2 311.3 1.3X
+Native ORC Vectorized (Pushdown) 294 300 7 53.5 18.7 22.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
InSet -> InFilters (values count: 10, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6565 6610 62 2.4 417.4 1.0X
-Parquet Vectorized (Pushdown) 302 313 11 52.1 19.2 21.8X
-Native ORC Vectorized 4787 4816 57 3.3 304.3 1.4X
-Native ORC Vectorized (Pushdown) 296 298 3 53.2 18.8 22.2X
+Parquet Vectorized 6534 6553 12 2.4 415.4 1.0X
+Parquet Vectorized (Pushdown) 302 306 3 52.1 19.2 21.6X
+Native ORC Vectorized 4889 4900 13 3.2 310.9 1.3X
+Native ORC Vectorized (Pushdown) 310 314 6 50.8 19.7 21.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
InSet -> InFilters (values count: 10, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6569 6591 15 2.4 417.6 1.0X
-Parquet Vectorized (Pushdown) 303 307 3 51.8 19.3 21.6X
-Native ORC Vectorized 4772 4801 25 3.3 303.4 1.4X
-Native ORC Vectorized (Pushdown) 297 304 11 53.0 18.9 22.1X
+Parquet Vectorized 6521 6537 15 2.4 414.6 1.0X
+Parquet Vectorized (Pushdown) 303 306 2 51.9 19.3 21.5X
+Native ORC Vectorized 4892 4897 4 3.2 311.0 1.3X
+Native ORC Vectorized (Pushdown) 310 319 13 50.8 19.7 21.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
InSet -> InFilters (values count: 10, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6552 6564 17 2.4 416.5 1.0X
-Parquet Vectorized (Pushdown) 307 314 11 51.3 19.5 21.4X
-Native ORC Vectorized 4782 4799 11 3.3 304.0 1.4X
-Native ORC Vectorized (Pushdown) 300 305 5 52.4 19.1 21.8X
+Parquet Vectorized 6517 6526 10 2.4 414.4 1.0X
+Parquet Vectorized (Pushdown) 304 309 4 51.8 19.3 21.5X
+Native ORC Vectorized 4891 4898 9 3.2 310.9 1.3X
+Native ORC Vectorized (Pushdown) 310 315 5 50.8 19.7 21.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
InSet -> InFilters (values count: 50, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6741 6765 20 2.3 428.6 1.0X
-Parquet Vectorized (Pushdown) 895 899 5 17.6 56.9 7.5X
-Native ORC Vectorized 4998 5024 19 3.1 317.8 1.3X
-Native ORC Vectorized (Pushdown) 397 401 3 39.6 25.3 17.0X
+Parquet Vectorized 6739 6750 7 2.3 428.5 1.0X
+Parquet Vectorized (Pushdown) 886 889 2 17.8 56.3 7.6X
+Native ORC Vectorized 5109 5115 3 3.1 324.8 1.3X
+Native ORC Vectorized (Pushdown) 417 425 9 37.7 26.5 16.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
InSet -> InFilters (values count: 50, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6749 6756 9 2.3 429.1 1.0X
-Parquet Vectorized (Pushdown) 3368 3387 24 4.7 214.1 2.0X
-Native ORC Vectorized 4978 5002 16 3.2 316.5 1.4X
-Native ORC Vectorized (Pushdown) 426 430 5 36.9 27.1 15.8X
+Parquet Vectorized 6742 6750 7 2.3 428.6 1.0X
+Parquet Vectorized (Pushdown) 3390 3412 37 4.6 215.5 2.0X
+Native ORC Vectorized 5124 5127 5 3.1 325.8 1.3X
+Native ORC Vectorized (Pushdown) 443 448 3 35.5 28.2 15.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
InSet -> InFilters (values count: 50, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6724 6744 13 2.3 427.5 1.0X
-Parquet Vectorized (Pushdown) 6089 6117 29 2.6 387.1 1.1X
-Native ORC Vectorized 4978 4995 14 3.2 316.5 1.4X
-Native ORC Vectorized (Pushdown) 423 424 1 37.2 26.9 15.9X
+Parquet Vectorized 6737 6741 3 2.3 428.3 1.0X
+Parquet Vectorized (Pushdown) 6038 6051 12 2.6 383.9 1.1X
+Native ORC Vectorized 5095 5110 15 3.1 323.9 1.3X
+Native ORC Vectorized (Pushdown) 444 448 5 35.5 28.2 15.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
InSet -> InFilters (values count: 100, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6727 6765 45 2.3 427.7 1.0X
-Parquet Vectorized (Pushdown) 923 928 7 17.0 58.7 7.3X
-Native ORC Vectorized 4938 4959 18 3.2 313.9 1.4X
-Native ORC Vectorized (Pushdown) 500 509 7 31.5 31.8 13.5X
+Parquet Vectorized 6679 6694 13 2.4 424.7 1.0X
+Parquet Vectorized (Pushdown) 894 898 3 17.6 56.9 7.5X
+Native ORC Vectorized 5069 5087 30 3.1 322.3 1.3X
+Native ORC Vectorized (Pushdown) 514 521 6 30.6 32.7 13.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
InSet -> InFilters (values count: 100, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6705 6743 54 2.3 426.3 1.0X
-Parquet Vectorized (Pushdown) 3479 3492 19 4.5 221.2 1.9X
-Native ORC Vectorized 4952 4959 8 3.2 314.9 1.4X
-Native ORC Vectorized (Pushdown) 581 587 6 27.1 36.9 11.5X
+Parquet Vectorized 6685 6698 9 2.4 425.1 1.0X
+Parquet Vectorized (Pushdown) 3472 3482 8 4.5 220.7 1.9X
+Native ORC Vectorized 5078 5084 4 3.1 322.8 1.3X
+Native ORC Vectorized (Pushdown) 589 593 4 26.7 37.4 11.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
InSet -> InFilters (values count: 100, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6696 6718 21 2.3 425.7 1.0X
-Parquet Vectorized (Pushdown) 6017 6033 17 2.6 382.6 1.1X
-Native ORC Vectorized 4926 4943 11 3.2 313.2 1.4X
-Native ORC Vectorized (Pushdown) 583 590 5 27.0 37.1 11.5X
+Parquet Vectorized 6685 6710 28 2.4 425.1 1.0X
+Parquet Vectorized (Pushdown) 6014 6035 30 2.6 382.4 1.1X
+Native ORC Vectorized 5078 5086 8 3.1 322.8 1.3X
+Native ORC Vectorized (Pushdown) 607 612 7 25.9 38.6 11.0X
================================================================================================
Pushdown benchmark for tinyint
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 tinyint row (value = CAST(63 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 3127 3134 6 5.0 198.8 1.0X
-Parquet Vectorized (Pushdown) 109 114 6 144.1 6.9 28.6X
-Native ORC Vectorized 2198 2204 4 7.2 139.8 1.4X
-Native ORC Vectorized (Pushdown) 114 117 3 137.4 7.3 27.3X
+Parquet Vectorized 3186 3197 20 4.9 202.6 1.0X
+Parquet Vectorized (Pushdown) 110 113 3 142.6 7.0 28.9X
+Native ORC Vectorized 2375 2386 9 6.6 151.0 1.3X
+Native ORC Vectorized (Pushdown) 119 122 3 131.8 7.6 26.7X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 10% tinyint rows (value < CAST(12 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 3775 3787 16 4.2 240.0 1.0X
-Parquet Vectorized (Pushdown) 1009 1011 2 15.6 64.2 3.7X
-Native ORC Vectorized 2801 2805 3 5.6 178.1 1.3X
-Native ORC Vectorized (Pushdown) 900 908 12 17.5 57.3 4.2X
+Parquet Vectorized 3763 3770 8 4.2 239.3 1.0X
+Parquet Vectorized (Pushdown) 1006 1011 5 15.6 64.0 3.7X
+Native ORC Vectorized 2942 2982 46 5.3 187.1 1.3X
+Native ORC Vectorized (Pushdown) 874 877 3 18.0 55.6 4.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 50% tinyint rows (value < CAST(63 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6493 6501 6 2.4 412.8 1.0X
-Parquet Vectorized (Pushdown) 4968 4989 30 3.2 315.8 1.3X
-Native ORC Vectorized 5371 5406 36 2.9 341.5 1.2X
-Native ORC Vectorized (Pushdown) 4292 4310 16 3.7 272.9 1.5X
+Parquet Vectorized 6306 6312 7 2.5 400.9 1.0X
+Parquet Vectorized (Pushdown) 4795 4802 6 3.3 304.8 1.3X
+Native ORC Vectorized 5414 5423 6 2.9 344.2 1.2X
+Native ORC Vectorized (Pushdown) 4282 4284 3 3.7 272.2 1.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 90% tinyint rows (value < CAST(114 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 9195 9235 69 1.7 584.6 1.0X
-Parquet Vectorized (Pushdown) 8940 8970 48 1.8 568.4 1.0X
-Native ORC Vectorized 7767 7796 46 2.0 493.8 1.2X
-Native ORC Vectorized (Pushdown) 7592 7621 27 2.1 482.7 1.2X
+Parquet Vectorized 9069 9075 5 1.7 576.6 1.0X
+Parquet Vectorized (Pushdown) 8806 8818 9 1.8 559.9 1.0X
+Native ORC Vectorized 7795 7800 5 2.0 495.6 1.2X
+Native ORC Vectorized (Pushdown) 7586 7605 16 2.1 482.3 1.2X
================================================================================================
Pushdown benchmark for Timestamp
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 timestamp stored as INT96 row (value = timestamp_seconds(7864320)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 3252 3261 7 4.8 206.8 1.0X
-Parquet Vectorized (Pushdown) 3242 3262 24 4.9 206.1 1.0X
-Native ORC Vectorized 2097 2110 11 7.5 133.3 1.6X
-Native ORC Vectorized (Pushdown) 39 43 4 401.0 2.5 82.9X
+Parquet Vectorized 3221 3224 3 4.9 204.8 1.0X
+Parquet Vectorized (Pushdown) 3228 3235 8 4.9 205.3 1.0X
+Native ORC Vectorized 2229 2241 15 7.1 141.7 1.4X
+Native ORC Vectorized (Pushdown) 43 45 3 369.5 2.7 75.7X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 10% timestamp stored as INT96 rows (value < timestamp_seconds(1572864)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 3941 3946 5 4.0 250.6 1.0X
-Parquet Vectorized (Pushdown) 3930 3942 16 4.0 249.8 1.0X
-Native ORC Vectorized 2713 2722 16 5.8 172.5 1.5X
-Native ORC Vectorized (Pushdown) 852 858 9 18.5 54.2 4.6X
+Parquet Vectorized 3932 3953 18 4.0 250.0 1.0X
+Parquet Vectorized (Pushdown) 3926 3932 4 4.0 249.6 1.0X
+Native ORC Vectorized 2893 2902 13 5.4 183.9 1.4X
+Native ORC Vectorized (Pushdown) 914 922 8 17.2 58.1 4.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 50% timestamp stored as INT96 rows (value < timestamp_seconds(7864320)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6700 6735 46 2.3 426.0 1.0X
-Parquet Vectorized (Pushdown) 6701 6727 19 2.3 426.0 1.0X
-Native ORC Vectorized 5113 5127 18 3.1 325.1 1.3X
-Native ORC Vectorized (Pushdown) 4045 4067 31 3.9 257.2 1.7X
+Parquet Vectorized 6853 6857 3 2.3 435.7 1.0X
+Parquet Vectorized (Pushdown) 6848 6857 14 2.3 435.4 1.0X
+Native ORC Vectorized 5512 5540 37 2.9 350.4 1.2X
+Native ORC Vectorized (Pushdown) 4361 4386 27 3.6 277.3 1.6X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 90% timestamp stored as INT96 rows (value < timestamp_seconds(14155776)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 9366 9381 19 1.7 595.5 1.0X
-Parquet Vectorized (Pushdown) 9369 9383 22 1.7 595.7 1.0X
-Native ORC Vectorized 7608 7687 83 2.1 483.7 1.2X
-Native ORC Vectorized (Pushdown) 7411 7431 19 2.1 471.2 1.3X
+Parquet Vectorized 9465 9534 92 1.7 601.7 1.0X
+Parquet Vectorized (Pushdown) 9532 9575 45 1.7 606.1 1.0X
+Native ORC Vectorized 8097 8146 39 1.9 514.8 1.2X
+Native ORC Vectorized (Pushdown) 8029 8064 27 2.0 510.5 1.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 timestamp stored as TIMESTAMP_MICROS row (value = timestamp_seconds(7864320)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 3004 3041 38 5.2 191.0 1.0X
-Parquet Vectorized (Pushdown) 69 72 4 229.2 4.4 43.8X
-Native ORC Vectorized 2095 2101 5 7.5 133.2 1.4X
-Native ORC Vectorized (Pushdown) 38 41 3 409.1 2.4 78.1X
+Parquet Vectorized 2994 3008 15 5.3 190.3 1.0X
+Parquet Vectorized (Pushdown) 70 75 3 223.8 4.5 42.6X
+Native ORC Vectorized 2247 2257 12 7.0 142.8 1.3X
+Native ORC Vectorized (Pushdown) 44 51 5 354.7 2.8 67.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 10% timestamp stored as TIMESTAMP_MICROS rows (value < timestamp_seconds(1572864)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 3705 3715 10 4.2 235.6 1.0X
-Parquet Vectorized (Pushdown) 1050 1052 2 15.0 66.7 3.5X
-Native ORC Vectorized 2714 2726 9 5.8 172.6 1.4X
-Native ORC Vectorized (Pushdown) 851 854 6 18.5 54.1 4.4X
+Parquet Vectorized 3719 3796 54 4.2 236.4 1.0X
+Parquet Vectorized (Pushdown) 1074 1091 12 14.6 68.3 3.5X
+Native ORC Vectorized 2922 2951 17 5.4 185.8 1.3X
+Native ORC Vectorized (Pushdown) 926 944 13 17.0 58.9 4.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 50% timestamp stored as TIMESTAMP_MICROS rows (value < timestamp_seconds(7864320)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6438 6454 13 2.4 409.3 1.0X
-Parquet Vectorized (Pushdown) 4976 4983 5 3.2 316.4 1.3X
-Native ORC Vectorized 5124 5140 16 3.1 325.8 1.3X
-Native ORC Vectorized (Pushdown) 4055 4089 30 3.9 257.8 1.6X
+Parquet Vectorized 6719 6753 25 2.3 427.2 1.0X
+Parquet Vectorized (Pushdown) 5237 5278 27 3.0 333.0 1.3X
+Native ORC Vectorized 5596 5604 10 2.8 355.8 1.2X
+Native ORC Vectorized (Pushdown) 4331 4388 47 3.6 275.3 1.6X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 90% timestamp stored as TIMESTAMP_MICROS rows (value < timestamp_seconds(14155776)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 9096 9115 18 1.7 578.3 1.0X
-Parquet Vectorized (Pushdown) 8823 8839 11 1.8 561.0 1.0X
-Native ORC Vectorized 7554 7578 23 2.1 480.2 1.2X
-Native ORC Vectorized (Pushdown) 7372 7392 16 2.1 468.7 1.2X
+Parquet Vectorized 9216 9232 14 1.7 585.9 1.0X
+Parquet Vectorized (Pushdown) 8949 8960 11 1.8 569.0 1.0X
+Native ORC Vectorized 8266 8296 51 1.9 525.5 1.1X
+Native ORC Vectorized (Pushdown) 8030 8045 16 2.0 510.6 1.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 timestamp stored as TIMESTAMP_MILLIS row (value = timestamp_seconds(7864320)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 3027 3036 7 5.2 192.5 1.0X
-Parquet Vectorized (Pushdown) 69 71 3 228.9 4.4 44.1X
-Native ORC Vectorized 2093 2100 11 7.5 133.1 1.4X
-Native ORC Vectorized (Pushdown) 38 41 3 411.8 2.4 79.2X
+Parquet Vectorized 3022 3027 5 5.2 192.1 1.0X
+Parquet Vectorized (Pushdown) 69 70 2 228.7 4.4 43.9X
+Native ORC Vectorized 2225 2230 3 7.1 141.5 1.4X
+Native ORC Vectorized (Pushdown) 42 44 3 377.3 2.7 72.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 10% timestamp stored as TIMESTAMP_MILLIS rows (value < timestamp_seconds(1572864)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 3718 3735 21 4.2 236.4 1.0X
-Parquet Vectorized (Pushdown) 1043 1049 7 15.1 66.3 3.6X
-Native ORC Vectorized 2707 2719 17 5.8 172.1 1.4X
-Native ORC Vectorized (Pushdown) 848 852 3 18.6 53.9 4.4X
+Parquet Vectorized 3711 3727 14 4.2 236.0 1.0X
+Parquet Vectorized (Pushdown) 1053 1056 3 14.9 66.9 3.5X
+Native ORC Vectorized 2889 2917 54 5.4 183.7 1.3X
+Native ORC Vectorized (Pushdown) 912 915 2 17.3 58.0 4.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 50% timestamp stored as TIMESTAMP_MILLIS rows (value < timestamp_seconds(7864320)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 6450 6462 13 2.4 410.1 1.0X
-Parquet Vectorized (Pushdown) 4988 4998 8 3.2 317.1 1.3X
-Native ORC Vectorized 5126 5140 15 3.1 325.9 1.3X
-Native ORC Vectorized (Pushdown) 4049 4060 8 3.9 257.4 1.6X
+Parquet Vectorized 6607 6613 10 2.4 420.1 1.0X
+Parquet Vectorized (Pushdown) 5149 5167 33 3.1 327.3 1.3X
+Native ORC Vectorized 5498 5511 9 2.9 349.6 1.2X
+Native ORC Vectorized (Pushdown) 4356 4368 10 3.6 276.9 1.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 90% timestamp stored as TIMESTAMP_MILLIS rows (value < timestamp_seconds(14155776)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 9130 9146 18 1.7 580.4 1.0X
-Parquet Vectorized (Pushdown) 8854 8885 20 1.8 562.9 1.0X
-Native ORC Vectorized 7538 7557 20 2.1 479.3 1.2X
-Native ORC Vectorized (Pushdown) 7310 7361 38 2.2 464.8 1.2X
+Parquet Vectorized 9222 9225 2 1.7 586.3 1.0X
+Parquet Vectorized (Pushdown) 8952 8972 19 1.8 569.2 1.0X
+Native ORC Vectorized 8251 8266 16 1.9 524.6 1.1X
+Native ORC Vectorized (Pushdown) 8030 8044 16 2.0 510.5 1.1X
================================================================================================
Pushdown benchmark with many filters
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 row with 1 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 48 49 2 0.0 47572233.0 1.0X
-Parquet Vectorized (Pushdown) 48 51 3 0.0 47933317.0 1.0X
-Native ORC Vectorized 41 43 2 0.0 41232680.0 1.2X
-Native ORC Vectorized (Pushdown) 44 50 8 0.0 43791797.0 1.1X
+Parquet Vectorized 47 49 5 0.0 46814687.0 1.0X
+Parquet Vectorized (Pushdown) 48 50 4 0.0 48049852.0 1.0X
+Native ORC Vectorized 42 44 4 0.0 41770894.0 1.1X
+Native ORC Vectorized (Pushdown) 45 47 4 0.0 44627062.0 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 row with 250 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 172 179 5 0.0 171668021.0 1.0X
-Parquet Vectorized (Pushdown) 175 183 8 0.0 174931280.0 1.0X
-Native ORC Vectorized 165 173 8 0.0 164882447.0 1.0X
-Native ORC Vectorized (Pushdown) 169 174 6 0.0 168734657.0 1.0X
+Parquet Vectorized 174 179 5 0.0 173730940.0 1.0X
+Parquet Vectorized (Pushdown) 177 183 6 0.0 177380451.0 1.0X
+Native ORC Vectorized 168 174 4 0.0 168397398.0 1.0X
+Native ORC Vectorized (Pushdown) 174 178 6 0.0 173527951.0 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select 1 row with 500 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Parquet Vectorized 523 547 25 0.0 523403894.0 1.0X
-Parquet Vectorized (Pushdown) 546 559 13 0.0 545572756.0 1.0X
-Native ORC Vectorized 525 544 15 0.0 525242709.0 1.0X
-Native ORC Vectorized (Pushdown) 532 554 28 0.0 531883827.0 1.0X
+Parquet Vectorized 545 556 11 0.0 544697398.0 1.0X
+Parquet Vectorized (Pushdown) 564 568 3 0.0 564247787.0 1.0X
+Native ORC Vectorized 536 549 11 0.0 536299871.0 1.0X
+Native ORC Vectorized (Pushdown) 545 548 3 0.0 544510520.0 1.0X
diff --git a/sql/core/benchmarks/GenerateExecBenchmark-jdk21-results.txt b/sql/core/benchmarks/GenerateExecBenchmark-jdk21-results.txt
index dc19c4fac6858..68ee1e97ee016 100644
--- a/sql/core/benchmarks/GenerateExecBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/GenerateExecBenchmark-jdk21-results.txt
@@ -2,11 +2,11 @@
GenerateExec benchmark
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
GenerateExec Benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-GenerateExec Benchmark wholestage off 70672 71244 809 1.4 706.7 1.0X
-GenerateExec Benchmark wholestage on 21022 21399 226 4.8 210.2 3.4X
+GenerateExec Benchmark wholestage off 72498 73647 1625 1.4 725.0 1.0X
+GenerateExec Benchmark wholestage on 23805 23956 189 4.2 238.1 3.0X
diff --git a/sql/core/benchmarks/GenerateExecBenchmark-results.txt b/sql/core/benchmarks/GenerateExecBenchmark-results.txt
index ab7a58d4f8a6d..bbdacf15ef8f0 100644
--- a/sql/core/benchmarks/GenerateExecBenchmark-results.txt
+++ b/sql/core/benchmarks/GenerateExecBenchmark-results.txt
@@ -2,11 +2,11 @@
GenerateExec benchmark
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
GenerateExec Benchmark: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-GenerateExec Benchmark wholestage off 69333 69516 259 1.4 693.3 1.0X
-GenerateExec Benchmark wholestage on 23359 23683 302 4.3 233.6 3.0X
+GenerateExec Benchmark wholestage off 70300 70379 111 1.4 703.0 1.0X
+GenerateExec Benchmark wholestage on 25435 25593 321 3.9 254.3 2.8X
diff --git a/sql/core/benchmarks/HashedRelationMetricsBenchmark-jdk21-results.txt b/sql/core/benchmarks/HashedRelationMetricsBenchmark-jdk21-results.txt
index edb74e92c5ee0..7449722cdcbef 100644
--- a/sql/core/benchmarks/HashedRelationMetricsBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/HashedRelationMetricsBenchmark-jdk21-results.txt
@@ -2,10 +2,10 @@
LongToUnsafeRowMap metrics
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
LongToUnsafeRowMap metrics: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-LongToUnsafeRowMap 265 268 3 1.9 529.3 1.0X
+LongToUnsafeRowMap 267 271 4 1.9 534.8 1.0X
diff --git a/sql/core/benchmarks/HashedRelationMetricsBenchmark-results.txt b/sql/core/benchmarks/HashedRelationMetricsBenchmark-results.txt
index e8fcbffee58b8..55452a515f1f2 100644
--- a/sql/core/benchmarks/HashedRelationMetricsBenchmark-results.txt
+++ b/sql/core/benchmarks/HashedRelationMetricsBenchmark-results.txt
@@ -2,10 +2,10 @@
LongToUnsafeRowMap metrics
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
LongToUnsafeRowMap metrics: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-LongToUnsafeRowMap 249 252 2 2.0 498.8 1.0X
+LongToUnsafeRowMap 258 263 6 1.9 515.9 1.0X
diff --git a/sql/core/benchmarks/InExpressionBenchmark-jdk21-results.txt b/sql/core/benchmarks/InExpressionBenchmark-jdk21-results.txt
index 3a7e026c04398..e6f609f1c52f0 100644
--- a/sql/core/benchmarks/InExpressionBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/InExpressionBenchmark-jdk21-results.txt
@@ -2,739 +2,739 @@
In Expression Benchmark
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 37 50 9 273.5 3.7 1.0X
-InSet expression 65 75 8 153.6 6.5 0.6X
+In expression 40 53 8 252.1 4.0 1.0X
+InSet expression 67 81 10 148.2 6.7 0.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 34 40 6 290.9 3.4 1.0X
-InSet expression 59 64 4 168.1 5.9 0.6X
+In expression 37 42 7 270.7 3.7 1.0X
+InSet expression 63 65 2 159.3 6.3 0.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 63 67 6 158.6 6.3 1.0X
-InSet expression 76 78 2 131.4 7.6 0.8X
+In expression 63 68 7 158.2 6.3 1.0X
+InSet expression 77 81 4 129.2 7.7 0.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 104 106 2 96.2 10.4 1.0X
-InSet expression 83 86 3 120.8 8.3 1.3X
+In expression 107 110 3 93.4 10.7 1.0X
+InSet expression 84 88 4 119.3 8.4 1.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 183 185 3 54.7 18.3 1.0X
-InSet expression 96 100 2 104.1 9.6 1.9X
+In expression 185 187 2 54.0 18.5 1.0X
+InSet expression 97 101 3 102.7 9.7 1.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 396 402 7 25.2 39.6 1.0X
-InSet expression 192 194 2 52.1 19.2 2.1X
+In expression 396 402 8 25.2 39.6 1.0X
+InSet expression 191 194 3 52.3 19.1 2.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 29 34 5 340.7 2.9 1.0X
-InSet expression 80 82 2 125.3 8.0 0.4X
+In expression 30 34 5 329.7 3.0 1.0X
+InSet expression 76 80 2 131.0 7.6 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 38 44 9 260.8 3.8 1.0X
-InSet expression 95 97 2 105.8 9.5 0.4X
+In expression 40 44 4 248.7 4.0 1.0X
+InSet expression 93 96 8 107.9 9.3 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 69 71 2 143.9 6.9 1.0X
-InSet expression 94 98 2 105.9 9.4 0.7X
+In expression 71 72 2 140.3 7.1 1.0X
+InSet expression 91 93 1 109.7 9.1 0.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 117 118 1 85.6 11.7 1.0X
-InSet expression 131 135 3 76.5 13.1 0.9X
+In expression 119 121 3 84.1 11.9 1.0X
+InSet expression 130 132 3 77.2 13.0 0.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 195 197 2 51.3 19.5 1.0X
-InSet expression 119 122 2 84.0 11.9 1.6X
+In expression 197 202 10 50.7 19.7 1.0X
+InSet expression 115 118 2 86.8 11.5 1.7X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 370 371 1 27.1 37.0 1.0X
-InSet expression 129 131 1 77.5 12.9 2.9X
+In expression 369 371 4 27.1 36.9 1.0X
+InSet expression 123 126 2 81.6 12.3 3.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
300 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 553 558 4 18.1 55.3 1.0X
-InSet expression 142 144 2 70.4 14.2 3.9X
+In expression 550 551 1 18.2 55.0 1.0X
+InSet expression 136 139 2 73.7 13.6 4.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
400 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 733 737 3 13.6 73.3 1.0X
-InSet expression 154 157 3 64.9 15.4 4.8X
+In expression 735 736 1 13.6 73.5 1.0X
+InSet expression 150 152 1 66.6 15.0 4.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
500 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 938 945 5 10.7 93.8 1.0X
-InSet expression 168 171 2 59.4 16.8 5.6X
+In expression 988 992 3 10.1 98.8 1.0X
+InSet expression 162 165 1 61.6 16.2 6.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 26 28 3 389.4 2.6 1.0X
-InSet expression 77 80 3 129.7 7.7 0.3X
+In expression 27 30 4 367.2 2.7 1.0X
+InSet expression 73 76 2 136.3 7.3 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 34 37 4 295.5 3.4 1.0X
-InSet expression 94 96 1 106.4 9.4 0.4X
+In expression 35 39 5 283.8 3.5 1.0X
+InSet expression 92 95 2 108.3 9.2 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 59 61 3 169.7 5.9 1.0X
-InSet expression 122 126 7 82.1 12.2 0.5X
+In expression 61 63 3 165.0 6.1 1.0X
+InSet expression 121 124 2 82.4 12.1 0.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 110 112 4 91.0 11.0 1.0X
-InSet expression 138 141 2 72.5 13.8 0.8X
+In expression 112 113 2 89.4 11.2 1.0X
+InSet expression 135 139 4 73.9 13.5 0.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 192 196 4 52.0 19.2 1.0X
-InSet expression 122 125 2 81.9 12.2 1.6X
+In expression 194 195 2 51.4 19.4 1.0X
+InSet expression 117 121 2 85.2 11.7 1.7X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 342 344 3 29.3 34.2 1.0X
-InSet expression 129 132 3 77.5 12.9 2.6X
+In expression 343 344 1 29.1 34.3 1.0X
+InSet expression 123 126 2 81.2 12.3 2.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
300 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 513 521 7 19.5 51.3 1.0X
-InSet expression 139 142 2 72.2 13.9 3.7X
+In expression 514 515 1 19.5 51.4 1.0X
+InSet expression 134 136 1 74.7 13.4 3.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
400 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 660 661 1 15.1 66.0 1.0X
-InSet expression 153 156 1 65.4 15.3 4.3X
+In expression 662 665 4 15.1 66.2 1.0X
+InSet expression 147 151 3 68.1 14.7 4.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
500 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 858 865 6 11.7 85.8 1.0X
-InSet expression 164 173 21 61.1 16.4 5.2X
+In expression 847 853 4 11.8 84.7 1.0X
+InSet expression 160 162 2 62.7 16.0 5.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 25 27 4 398.8 2.5 1.0X
-InSet expression 86 87 1 116.6 8.6 0.3X
+In expression 26 28 3 385.2 2.6 1.0X
+InSet expression 81 83 1 123.4 8.1 0.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 36 40 5 277.9 3.6 1.0X
-InSet expression 106 109 3 94.4 10.6 0.3X
+In expression 38 40 3 266.2 3.8 1.0X
+InSet expression 98 100 1 102.4 9.8 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 58 60 3 172.3 5.8 1.0X
-InSet expression 107 108 1 93.9 10.7 0.5X
+In expression 60 62 3 167.1 6.0 1.0X
+InSet expression 98 100 2 102.2 9.8 0.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 115 116 2 87.1 11.5 1.0X
-InSet expression 151 154 1 66.0 15.1 0.8X
+In expression 117 118 1 85.5 11.7 1.0X
+InSet expression 135 138 2 73.9 13.5 0.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 192 198 5 52.2 19.2 1.0X
-InSet expression 129 131 3 77.7 12.9 1.5X
+In expression 184 193 4 54.4 18.4 1.0X
+InSet expression 118 121 2 84.4 11.8 1.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 353 357 5 28.3 35.3 1.0X
-InSet expression 134 136 1 74.5 13.4 2.6X
+In expression 354 355 1 28.3 35.4 1.0X
+InSet expression 124 125 1 80.7 12.4 2.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
300 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 508 527 21 19.7 50.8 1.0X
-InSet expression 146 148 2 68.3 14.6 3.5X
+In expression 513 513 1 19.5 51.3 1.0X
+InSet expression 136 138 1 73.6 13.6 3.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
400 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 668 685 15 15.0 66.8 1.0X
-InSet expression 159 162 2 62.8 15.9 4.2X
+In expression 678 680 4 14.8 67.8 1.0X
+InSet expression 148 152 3 67.5 14.8 4.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
500 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 833 860 47 12.0 83.3 1.0X
-InSet expression 173 176 2 57.8 17.3 4.8X
+In expression 829 834 6 12.1 82.9 1.0X
+InSet expression 160 163 1 62.3 16.0 5.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 20 23 5 509.7 2.0 1.0X
-InSet expression 82 84 2 122.5 8.2 0.2X
+In expression 20 24 4 495.0 2.0 1.0X
+InSet expression 78 80 1 127.9 7.8 0.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 27 29 3 364.7 2.7 1.0X
-InSet expression 102 104 1 97.7 10.2 0.3X
+In expression 29 31 3 349.1 2.9 1.0X
+InSet expression 95 97 1 105.3 9.5 0.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 56 58 3 178.9 5.6 1.0X
-InSet expression 118 120 1 84.4 11.8 0.5X
+In expression 58 60 2 173.7 5.8 1.0X
+InSet expression 108 110 1 92.3 10.8 0.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 99 99 1 101.5 9.9 1.0X
-InSet expression 147 149 1 68.0 14.7 0.7X
+In expression 100 101 2 99.7 10.0 1.0X
+InSet expression 134 135 1 74.8 13.4 0.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 178 180 2 56.1 17.8 1.0X
-InSet expression 133 136 3 75.0 13.3 1.3X
+In expression 180 181 1 55.6 18.0 1.0X
+InSet expression 121 124 2 82.4 12.1 1.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 338 344 6 29.5 33.8 1.0X
-InSet expression 138 139 1 72.7 13.8 2.5X
+In expression 338 347 6 29.6 33.8 1.0X
+InSet expression 128 130 1 77.9 12.8 2.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
300 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 506 509 2 19.7 50.6 1.0X
-InSet expression 146 148 3 68.5 14.6 3.5X
+In expression 507 509 4 19.7 50.7 1.0X
+InSet expression 136 139 2 73.5 13.6 3.7X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
400 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 670 677 8 14.9 67.0 1.0X
-InSet expression 159 162 2 62.8 15.9 4.2X
+In expression 674 676 3 14.8 67.4 1.0X
+InSet expression 148 151 2 67.6 14.8 4.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
500 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 833 864 66 12.0 83.3 1.0X
-InSet expression 172 173 1 58.3 17.2 4.9X
+In expression 833 867 62 12.0 83.3 1.0X
+InSet expression 160 162 1 62.6 16.0 5.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 19 20 3 537.7 1.9 1.0X
-InSet expression 75 76 1 134.2 7.5 0.2X
+In expression 20 22 2 498.7 2.0 1.0X
+InSet expression 87 89 2 115.6 8.7 0.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 28 30 2 356.7 2.8 1.0X
-InSet expression 91 94 3 109.5 9.1 0.3X
+In expression 30 32 3 337.9 3.0 1.0X
+InSet expression 102 105 4 98.4 10.2 0.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 50 52 3 199.6 5.0 1.0X
-InSet expression 91 93 1 109.5 9.1 0.5X
+In expression 54 57 3 185.2 5.4 1.0X
+InSet expression 102 104 2 98.0 10.2 0.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 91 92 1 110.4 9.1 1.0X
-InSet expression 129 130 1 77.7 12.9 0.7X
+In expression 93 94 1 107.8 9.3 1.0X
+InSet expression 137 139 2 73.2 13.7 0.7X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 170 171 1 58.9 17.0 1.0X
-InSet expression 110 111 1 91.1 11.0 1.5X
+In expression 173 180 3 57.7 17.3 1.0X
+InSet expression 122 125 4 81.8 12.2 1.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 328 345 14 30.5 32.8 1.0X
-InSet expression 117 119 2 85.5 11.7 2.8X
+In expression 330 344 14 30.3 33.0 1.0X
+InSet expression 130 132 2 77.0 13.0 2.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 36 39 3 279.2 3.6 1.0X
-InSet expression 77 79 2 129.3 7.7 0.5X
+In expression 35 41 4 287.7 3.5 1.0X
+InSet expression 78 79 1 128.8 7.8 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 58 63 3 173.0 5.8 1.0X
-InSet expression 99 101 2 101.4 9.9 0.6X
+In expression 63 64 2 158.9 6.3 1.0X
+InSet expression 101 103 1 98.9 10.1 0.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 147 149 2 67.8 14.7 1.0X
-InSet expression 98 100 2 102.3 9.8 1.5X
+In expression 149 150 1 67.1 14.9 1.0X
+InSet expression 100 106 18 100.4 10.0 1.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 228 231 3 43.8 22.8 1.0X
-InSet expression 144 146 2 69.3 14.4 1.6X
+In expression 232 237 4 43.2 23.2 1.0X
+InSet expression 147 151 4 67.8 14.7 1.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 382 388 6 26.2 38.2 1.0X
-InSet expression 115 117 2 86.8 11.5 3.3X
+In expression 384 386 1 26.0 38.4 1.0X
+InSet expression 119 121 2 84.4 11.9 3.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 1921 1958 80 5.2 192.1 1.0X
-InSet expression 117 119 1 85.2 11.7 16.4X
+In expression 1901 1936 67 5.3 190.1 1.0X
+InSet expression 122 124 2 82.0 12.2 15.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 36 38 2 274.4 3.6 1.0X
-InSet expression 77 79 1 129.8 7.7 0.5X
+In expression 38 40 3 263.8 3.8 1.0X
+InSet expression 78 80 2 127.9 7.8 0.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 60 61 3 167.8 6.0 1.0X
-InSet expression 98 100 1 101.8 9.8 0.6X
+In expression 62 63 2 161.8 6.2 1.0X
+InSet expression 100 101 1 99.6 10.0 0.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 147 149 2 67.9 14.7 1.0X
-InSet expression 97 98 1 103.5 9.7 1.5X
+In expression 150 151 1 66.6 15.0 1.0X
+InSet expression 101 102 1 99.5 10.1 1.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 228 229 1 43.9 22.8 1.0X
-InSet expression 144 147 2 69.4 14.4 1.6X
+In expression 231 232 1 43.3 23.1 1.0X
+InSet expression 144 147 1 69.6 14.4 1.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 394 396 2 25.4 39.4 1.0X
-InSet expression 117 119 1 85.4 11.7 3.4X
+In expression 390 395 4 25.6 39.0 1.0X
+InSet expression 117 118 1 85.8 11.7 3.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 2228 2268 82 4.5 222.8 1.0X
-InSet expression 120 122 2 83.4 12.0 18.6X
+In expression 2231 2275 79 4.5 223.1 1.0X
+InSet expression 120 122 1 83.6 12.0 18.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 19 21 2 51.8 19.3 1.0X
-InSet expression 49 51 1 20.4 48.9 0.4X
+In expression 21 23 2 47.6 21.0 1.0X
+InSet expression 51 53 2 19.5 51.3 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 25 27 2 39.4 25.4 1.0X
-InSet expression 51 53 3 19.6 50.9 0.5X
+In expression 27 28 2 37.1 26.9 1.0X
+InSet expression 54 56 3 18.5 54.1 0.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 40 43 4 25.2 39.7 1.0X
-InSet expression 52 56 8 19.4 51.6 0.8X
+In expression 41 45 3 24.5 40.8 1.0X
+InSet expression 54 55 2 18.6 53.8 0.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 78 83 4 12.8 78.4 1.0X
-InSet expression 57 58 2 17.7 56.6 1.4X
+In expression 80 84 4 12.5 80.0 1.0X
+InSet expression 59 61 2 17.0 58.9 1.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 241 242 1 4.1 241.4 1.0X
-InSet expression 54 59 4 18.4 54.4 4.4X
+In expression 241 242 2 4.2 240.9 1.0X
+InSet expression 57 58 1 17.6 56.9 4.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 582 586 3 1.7 581.8 1.0X
-InSet expression 56 58 1 17.8 56.2 10.3X
+In expression 580 582 2 1.7 580.2 1.0X
+InSet expression 59 61 2 17.1 58.5 9.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 5 6 2 211.6 4.7 1.0X
-InSet expression 5 5 2 219.6 4.6 1.0X
+In expression 5 7 2 201.5 5.0 1.0X
+InSet expression 5 7 2 207.6 4.8 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 5 6 2 212.4 4.7 1.0X
-InSet expression 5 5 2 218.3 4.6 1.0X
+In expression 5 6 2 207.3 4.8 1.0X
+InSet expression 5 6 2 213.9 4.7 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 5 6 2 196.9 5.1 1.0X
-InSet expression 5 6 2 202.0 4.9 1.0X
+In expression 5 7 2 197.8 5.1 1.0X
+InSet expression 5 6 2 198.9 5.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 6 6 2 174.3 5.7 1.0X
-InSet expression 6 7 2 172.5 5.8 1.0X
+In expression 6 8 2 170.6 5.9 1.0X
+InSet expression 6 7 2 168.6 5.9 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 7 8 2 135.3 7.4 1.0X
-InSet expression 7 8 2 138.4 7.2 1.0X
+In expression 8 9 2 129.1 7.7 1.0X
+InSet expression 7 9 2 136.3 7.3 1.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 11 11 2 94.3 10.6 1.0X
-InSet expression 11 12 3 94.2 10.6 1.0X
+In expression 10 12 2 95.3 10.5 1.0X
+InSet expression 11 13 3 92.8 10.8 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 28 29 2 35.5 28.2 1.0X
-InSet expression 42 43 1 24.1 41.5 0.7X
+In expression 28 30 1 35.2 28.4 1.0X
+InSet expression 42 44 1 23.6 42.3 0.7X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 33 34 2 29.9 33.4 1.0X
-InSet expression 44 45 2 22.7 44.1 0.8X
+In expression 34 35 1 29.5 33.9 1.0X
+InSet expression 45 47 2 22.1 45.2 0.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 42 43 1 23.8 42.0 1.0X
-InSet expression 48 49 1 20.6 48.5 0.9X
+In expression 43 45 1 23.2 43.1 1.0X
+InSet expression 50 51 1 20.1 49.9 0.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 57 61 3 17.6 57.0 1.0X
-InSet expression 52 55 3 19.1 52.3 1.1X
+In expression 59 63 3 17.0 58.8 1.0X
+InSet expression 54 55 1 18.6 53.9 1.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 88 89 1 11.3 88.3 1.0X
-InSet expression 49 50 1 20.4 48.9 1.8X
+In expression 90 94 3 11.1 89.7 1.0X
+InSet expression 51 52 2 19.7 50.7 1.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 509 511 2 2.0 508.9 1.0X
-InSet expression 49 50 1 20.2 49.4 10.3X
+In expression 427 430 4 2.3 427.2 1.0X
+InSet expression 52 53 3 19.4 51.5 8.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 16 18 2 607.3 1.6 1.0X
-InSet expression 81 82 1 124.0 8.1 0.2X
+In expression 17 19 2 591.4 1.7 1.0X
+InSet expression 81 82 1 123.9 8.1 0.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 22 23 2 451.5 2.2 1.0X
-InSet expression 94 96 1 105.9 9.4 0.2X
+In expression 22 24 2 451.1 2.2 1.0X
+InSet expression 94 95 1 106.1 9.4 0.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 47 49 3 213.9 4.7 1.0X
-InSet expression 128 130 1 77.9 12.8 0.4X
+In expression 48 50 2 210.3 4.8 1.0X
+InSet expression 127 129 4 79.0 12.7 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 80 81 1 125.0 8.0 1.0X
-InSet expression 161 163 1 62.1 16.1 0.5X
+In expression 81 82 1 123.0 8.1 1.0X
+InSet expression 159 161 2 63.0 15.9 0.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 151 152 1 66.0 15.1 1.0X
-InSet expression 137 139 2 72.8 13.7 1.1X
+In expression 153 154 1 65.4 15.3 1.0X
+InSet expression 137 139 1 73.1 13.7 1.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 296 308 11 33.7 29.6 1.0X
-InSet expression 132 134 2 75.7 13.2 2.2X
+In expression 295 305 11 33.9 29.5 1.0X
+InSet expression 131 133 1 76.4 13.1 2.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 287 289 2 34.8 28.7 1.0X
-InSet expression 282 290 6 35.5 28.2 1.0X
+In expression 313 314 1 32.0 31.3 1.0X
+InSet expression 306 307 2 32.7 30.6 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 304 305 1 32.9 30.4 1.0X
-InSet expression 286 289 2 35.0 28.6 1.1X
+In expression 325 329 4 30.8 32.5 1.0X
+InSet expression 306 308 2 32.7 30.6 1.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 327 335 7 30.6 32.7 1.0X
-InSet expression 291 296 6 34.4 29.1 1.1X
+In expression 364 365 2 27.5 36.4 1.0X
+InSet expression 306 308 1 32.7 30.6 1.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 397 399 1 25.2 39.7 1.0X
-InSet expression 282 286 3 35.5 28.2 1.4X
+In expression 411 412 1 24.3 41.1 1.0X
+InSet expression 305 308 2 32.8 30.5 1.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 476 478 1 21.0 47.6 1.0X
-InSet expression 283 288 4 35.3 28.3 1.7X
+In expression 493 496 4 20.3 49.3 1.0X
+InSet expression 310 312 1 32.2 31.0 1.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 626 628 2 16.0 62.6 1.0X
-InSet expression 287 291 4 34.9 28.7 2.2X
+In expression 657 660 3 15.2 65.7 1.0X
+InSet expression 311 312 1 32.2 31.1 2.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
300 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 788 791 2 12.7 78.8 1.0X
-InSet expression 286 292 5 35.0 28.6 2.8X
+In expression 812 862 28 12.3 81.2 1.0X
+InSet expression 316 317 2 31.7 31.6 2.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
400 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 956 960 5 10.5 95.6 1.0X
-InSet expression 293 297 2 34.1 29.3 3.3X
+In expression 978 981 3 10.2 97.8 1.0X
+InSet expression 311 317 5 32.1 31.1 3.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
500 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 1114 1120 4 9.0 111.4 1.0X
-InSet expression 380 382 1 26.3 38.0 2.9X
+In expression 1144 1152 8 8.7 114.4 1.0X
+InSet expression 391 392 1 25.6 39.1 2.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 25 26 1 40.3 24.8 1.0X
-InSet expression 57 58 1 17.6 56.7 0.4X
+In expression 27 28 1 37.1 27.0 1.0X
+InSet expression 60 61 1 16.8 59.6 0.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 39 43 3 25.4 39.4 1.0X
-InSet expression 85 87 1 11.7 85.5 0.5X
+In expression 42 46 3 24.0 41.7 1.0X
+InSet expression 89 90 1 11.3 88.7 0.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 133 134 1 7.5 133.0 1.0X
-InSet expression 100 103 3 10.0 100.2 1.3X
+In expression 169 175 4 5.9 169.4 1.0X
+InSet expression 104 105 2 9.7 103.6 1.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 410 411 1 2.4 409.7 1.0X
-InSet expression 129 133 4 7.7 129.1 3.2X
+In expression 393 396 3 2.5 393.4 1.0X
+InSet expression 133 136 3 7.5 133.1 3.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 792 795 2 1.3 792.0 1.0X
-InSet expression 147 148 2 6.8 147.0 5.4X
+In expression 817 821 4 1.2 817.2 1.0X
+InSet expression 151 151 0 6.6 150.5 5.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 1758 1924 334 0.6 1758.0 1.0X
-InSet expression 164 166 1 6.1 164.3 10.7X
+In expression 1702 1856 335 0.6 1702.4 1.0X
+InSet expression 167 168 2 6.0 166.6 10.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 16 17 2 61.4 16.3 1.0X
-InSet expression 81 83 1 12.3 81.5 0.2X
+In expression 17 18 1 58.4 17.1 1.0X
+InSet expression 87 88 1 11.5 87.0 0.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 23 24 1 43.1 23.2 1.0X
-InSet expression 126 128 1 7.9 126.2 0.2X
+In expression 24 25 2 42.1 23.7 1.0X
+InSet expression 132 134 2 7.6 131.8 0.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 70 74 3 14.3 70.1 1.0X
-InSet expression 149 153 9 6.7 148.5 0.5X
+In expression 72 77 4 14.0 71.6 1.0X
+InSet expression 155 157 2 6.5 154.7 0.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 207 211 3 4.8 207.1 1.0X
-InSet expression 192 196 2 5.2 192.4 1.1X
+In expression 198 203 4 5.0 198.5 1.0X
+InSet expression 201 203 3 5.0 200.6 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 575 580 4 1.7 575.3 1.0X
-InSet expression 218 221 2 4.6 218.5 2.6X
+In expression 562 564 3 1.8 561.8 1.0X
+InSet expression 224 227 1 4.5 224.1 2.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 1351 1527 240 0.7 1350.5 1.0X
-InSet expression 249 252 2 4.0 249.3 5.4X
+In expression 1333 1511 260 0.8 1333.3 1.0X
+InSet expression 269 274 7 3.7 269.2 5.0X
diff --git a/sql/core/benchmarks/InExpressionBenchmark-results.txt b/sql/core/benchmarks/InExpressionBenchmark-results.txt
index 16e3c78bdb722..a72d901035c49 100644
--- a/sql/core/benchmarks/InExpressionBenchmark-results.txt
+++ b/sql/core/benchmarks/InExpressionBenchmark-results.txt
@@ -2,739 +2,739 @@
In Expression Benchmark
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 40 56 8 249.7 4.0 1.0X
-InSet expression 80 87 4 124.5 8.0 0.5X
+In expression 38 52 9 264.5 3.8 1.0X
+InSet expression 85 91 4 117.7 8.5 0.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 38 42 4 264.1 3.8 1.0X
-InSet expression 76 80 4 130.9 7.6 0.5X
+In expression 37 42 5 273.4 3.7 1.0X
+InSet expression 82 86 4 122.1 8.2 0.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 66 69 3 151.9 6.6 1.0X
-InSet expression 85 89 4 117.8 8.5 0.8X
+In expression 65 67 2 154.1 6.5 1.0X
+InSet expression 83 88 5 120.8 8.3 0.8X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 106 108 3 94.2 10.6 1.0X
-InSet expression 90 93 3 111.3 9.0 1.2X
+In expression 106 108 2 94.6 10.6 1.0X
+InSet expression 89 91 4 112.9 8.9 1.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 194 197 4 51.5 19.4 1.0X
-InSet expression 101 103 3 99.3 10.1 1.9X
+In expression 194 196 3 51.6 19.4 1.0X
+InSet expression 98 101 2 101.6 9.8 2.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 397 400 3 25.2 39.7 1.0X
-InSet expression 197 205 6 50.9 19.7 2.0X
+In expression 397 400 2 25.2 39.7 1.0X
+InSet expression 195 199 5 51.4 19.5 2.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 31 34 4 323.9 3.1 1.0X
-InSet expression 100 103 4 99.8 10.0 0.3X
+In expression 34 37 3 297.7 3.4 1.0X
+InSet expression 98 101 2 101.7 9.8 0.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 37 39 2 272.3 3.7 1.0X
-InSet expression 115 118 2 86.6 11.5 0.3X
+In expression 36 39 2 274.2 3.6 1.0X
+InSet expression 113 116 3 88.7 11.3 0.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 72 74 4 138.7 7.2 1.0X
-InSet expression 115 117 1 87.0 11.5 0.6X
+In expression 72 74 2 138.5 7.2 1.0X
+InSet expression 112 115 2 89.0 11.2 0.6X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 118 121 5 84.8 11.8 1.0X
-InSet expression 142 146 4 70.4 14.2 0.8X
+In expression 118 120 3 84.9 11.8 1.0X
+InSet expression 142 144 1 70.6 14.2 0.8X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 186 189 5 53.7 18.6 1.0X
-InSet expression 126 131 5 79.2 12.6 1.5X
+In expression 198 200 2 50.4 19.8 1.0X
+InSet expression 126 129 3 79.2 12.6 1.6X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 366 373 14 27.4 36.6 1.0X
-InSet expression 134 138 4 74.7 13.4 2.7X
+In expression 366 368 2 27.3 36.6 1.0X
+InSet expression 132 135 2 75.6 13.2 2.8X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
300 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 550 552 4 18.2 55.0 1.0X
-InSet expression 146 151 4 68.4 14.6 3.8X
+In expression 551 552 2 18.2 55.1 1.0X
+InSet expression 146 149 2 68.3 14.6 3.8X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
400 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 668 673 5 15.0 66.8 1.0X
-InSet expression 161 164 5 62.3 16.1 4.2X
+In expression 668 669 1 15.0 66.8 1.0X
+InSet expression 158 162 3 63.2 15.8 4.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
500 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 941 946 6 10.6 94.1 1.0X
-InSet expression 174 183 18 57.4 17.4 5.4X
+In expression 939 941 3 10.7 93.9 1.0X
+InSet expression 172 174 1 58.1 17.2 5.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 30 31 3 336.9 3.0 1.0X
-InSet expression 97 99 2 103.3 9.7 0.3X
+In expression 30 31 1 337.0 3.0 1.0X
+InSet expression 94 96 1 105.9 9.4 0.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 37 39 2 270.3 3.7 1.0X
-InSet expression 114 116 2 87.9 11.4 0.3X
+In expression 36 39 3 274.8 3.6 1.0X
+InSet expression 112 115 2 89.0 11.2 0.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 70 72 2 143.1 7.0 1.0X
-InSet expression 133 136 2 75.1 13.3 0.5X
+In expression 70 71 2 142.3 7.0 1.0X
+InSet expression 133 135 1 75.1 13.3 0.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 105 107 1 94.8 10.5 1.0X
-InSet expression 148 151 1 67.4 14.8 0.7X
+In expression 106 107 1 94.4 10.6 1.0X
+InSet expression 149 151 2 67.3 14.9 0.7X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 196 200 5 51.0 19.6 1.0X
-InSet expression 129 133 3 77.4 12.9 1.5X
+In expression 196 197 2 51.0 19.6 1.0X
+InSet expression 128 131 2 78.0 12.8 1.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 354 356 3 28.3 35.4 1.0X
-InSet expression 134 137 5 74.9 13.4 2.6X
+In expression 354 355 1 28.3 35.4 1.0X
+InSet expression 133 135 1 74.9 13.3 2.7X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
300 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 659 662 4 15.2 65.9 1.0X
-InSet expression 147 150 5 68.2 14.7 4.5X
+In expression 502 502 0 19.9 50.2 1.0X
+InSet expression 143 145 1 70.0 14.3 3.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
400 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 668 669 2 15.0 66.8 1.0X
-InSet expression 157 160 2 63.6 15.7 4.3X
+In expression 669 671 3 14.9 66.9 1.0X
+InSet expression 155 158 2 64.6 15.5 4.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
500 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 854 857 4 11.7 85.4 1.0X
-InSet expression 169 173 3 59.2 16.9 5.1X
+In expression 860 861 2 11.6 86.0 1.0X
+InSet expression 167 172 3 59.9 16.7 5.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 28 30 2 354.7 2.8 1.0X
-InSet expression 99 100 1 101.3 9.9 0.3X
+In expression 28 30 3 362.5 2.8 1.0X
+InSet expression 96 98 2 103.7 9.6 0.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 35 36 2 285.9 3.5 1.0X
-InSet expression 114 116 1 87.8 11.4 0.3X
+In expression 36 37 1 278.5 3.6 1.0X
+InSet expression 112 114 1 89.3 11.2 0.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 62 64 3 161.4 6.2 1.0X
-InSet expression 117 120 3 85.3 11.7 0.5X
+In expression 62 65 7 161.9 6.2 1.0X
+InSet expression 115 117 1 86.8 11.5 0.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 114 115 1 87.6 11.4 1.0X
-InSet expression 145 147 1 69.2 14.5 0.8X
+In expression 105 106 1 95.3 10.5 1.0X
+InSet expression 143 144 2 70.2 14.3 0.7X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 195 197 2 51.4 19.5 1.0X
-InSet expression 126 127 1 79.6 12.6 1.5X
+In expression 184 184 1 54.4 18.4 1.0X
+InSet expression 122 123 1 82.3 12.2 1.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 351 357 3 28.5 35.1 1.0X
-InSet expression 131 135 5 76.2 13.1 2.7X
+In expression 352 355 2 28.4 35.2 1.0X
+InSet expression 128 131 3 78.3 12.8 2.8X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
300 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 511 520 5 19.6 51.1 1.0X
-InSet expression 143 147 8 70.0 14.3 3.6X
+In expression 514 533 18 19.4 51.4 1.0X
+InSet expression 140 143 5 71.5 14.0 3.7X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
400 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 673 689 10 14.9 67.3 1.0X
-InSet expression 154 157 3 64.9 15.4 4.4X
+In expression 668 678 8 15.0 66.8 1.0X
+InSet expression 151 154 1 66.1 15.1 4.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
500 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 828 860 42 12.1 82.8 1.0X
-InSet expression 167 170 6 60.0 16.7 5.0X
+In expression 836 850 13 12.0 83.6 1.0X
+InSet expression 164 166 1 60.9 16.4 5.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 22 25 4 461.8 2.2 1.0X
-InSet expression 95 98 3 105.3 9.5 0.2X
+In expression 22 24 4 451.0 2.2 1.0X
+InSet expression 93 95 2 107.4 9.3 0.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 28 30 2 354.7 2.8 1.0X
-InSet expression 110 114 3 90.7 11.0 0.3X
+In expression 28 30 2 351.9 2.8 1.0X
+InSet expression 109 110 1 92.1 10.9 0.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 55 57 1 180.8 5.5 1.0X
-InSet expression 125 129 7 79.8 12.5 0.4X
+In expression 55 57 2 180.3 5.5 1.0X
+InSet expression 124 125 1 80.8 12.4 0.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 100 101 1 100.3 10.0 1.0X
-InSet expression 142 145 3 70.3 14.2 0.7X
+In expression 101 102 1 99.3 10.1 1.0X
+InSet expression 143 145 1 69.9 14.3 0.7X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 179 180 1 55.9 17.9 1.0X
-InSet expression 128 132 8 78.1 12.8 1.4X
+In expression 180 184 8 55.6 18.0 1.0X
+InSet expression 124 128 3 80.3 12.4 1.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 345 348 3 29.0 34.5 1.0X
-InSet expression 132 135 2 75.6 13.2 2.6X
+In expression 346 347 2 28.9 34.6 1.0X
+InSet expression 129 133 5 77.3 12.9 2.7X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
300 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 507 509 2 19.7 50.7 1.0X
-InSet expression 142 145 5 70.6 14.2 3.6X
+In expression 506 508 3 19.8 50.6 1.0X
+InSet expression 139 142 3 72.0 13.9 3.6X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
400 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 662 667 4 15.1 66.2 1.0X
-InSet expression 154 156 1 65.0 15.4 4.3X
+In expression 658 664 4 15.2 65.8 1.0X
+InSet expression 151 153 2 66.3 15.1 4.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
500 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 820 856 78 12.2 82.0 1.0X
-InSet expression 165 170 4 60.4 16.5 5.0X
+In expression 821 864 95 12.2 82.1 1.0X
+InSet expression 163 165 1 61.4 16.3 5.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 21 23 3 481.5 2.1 1.0X
-InSet expression 87 89 2 114.5 8.7 0.2X
+In expression 21 23 3 467.1 2.1 1.0X
+InSet expression 88 89 2 114.2 8.8 0.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 29 31 3 350.3 2.9 1.0X
-InSet expression 103 107 3 97.0 10.3 0.3X
+In expression 29 31 1 343.5 2.9 1.0X
+InSet expression 103 106 4 97.3 10.3 0.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 54 55 2 186.1 5.4 1.0X
-InSet expression 105 111 13 95.4 10.5 0.5X
+In expression 55 56 1 183.2 5.5 1.0X
+InSet expression 105 108 3 94.9 10.5 0.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 92 97 3 108.4 9.2 1.0X
+In expression 95 97 2 105.7 9.5 1.0X
InSet expression 134 137 2 74.5 13.4 0.7X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 174 183 6 57.6 17.4 1.0X
-InSet expression 113 119 14 88.2 11.3 1.5X
+In expression 177 182 4 56.6 17.7 1.0X
+InSet expression 115 117 1 86.8 11.5 1.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 330 352 13 30.3 33.0 1.0X
-InSet expression 118 121 2 84.8 11.8 2.8X
+In expression 330 342 12 30.3 33.0 1.0X
+InSet expression 121 123 1 82.3 12.1 2.7X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 35 37 2 284.5 3.5 1.0X
-InSet expression 115 118 3 87.3 11.5 0.3X
+In expression 40 42 2 252.8 4.0 1.0X
+InSet expression 95 97 1 104.8 9.5 0.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 63 65 1 157.8 6.3 1.0X
-InSet expression 145 146 1 69.0 14.5 0.4X
+In expression 64 65 1 156.9 6.4 1.0X
+InSet expression 115 117 1 86.8 11.5 0.6X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 127 129 1 78.5 12.7 1.0X
-InSet expression 144 147 4 69.5 14.4 0.9X
+In expression 127 129 3 78.7 12.7 1.0X
+InSet expression 117 118 1 85.4 11.7 1.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
In expression 209 210 1 47.8 20.9 1.0X
-InSet expression 189 191 2 52.8 18.9 1.1X
+InSet expression 158 160 3 63.4 15.8 1.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 366 368 3 27.3 36.6 1.0X
-InSet expression 148 151 5 67.7 14.8 2.5X
+In expression 367 367 1 27.3 36.7 1.0X
+InSet expression 128 130 2 78.4 12.8 2.9X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 1693 1819 98 5.9 169.3 1.0X
-InSet expression 151 154 3 66.1 15.1 11.2X
+In expression 1672 1739 94 6.0 167.2 1.0X
+InSet expression 133 135 2 75.2 13.3 12.6X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 41 43 3 243.1 4.1 1.0X
-InSet expression 115 117 1 86.6 11.5 0.4X
+In expression 40 42 4 252.2 4.0 1.0X
+InSet expression 116 119 6 86.0 11.6 0.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 63 65 1 157.9 6.3 1.0X
-InSet expression 145 147 1 69.2 14.5 0.4X
+In expression 62 64 2 160.7 6.2 1.0X
+InSet expression 146 148 1 68.3 14.6 0.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 129 131 2 77.4 12.9 1.0X
-InSet expression 144 146 2 69.4 14.4 0.9X
+In expression 129 130 1 77.4 12.9 1.0X
+InSet expression 143 144 1 70.0 14.3 0.9X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 210 211 2 47.7 21.0 1.0X
-InSet expression 191 195 4 52.4 19.1 1.1X
+In expression 209 210 1 47.8 20.9 1.0X
+InSet expression 192 195 3 52.0 19.2 1.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 367 368 1 27.3 36.7 1.0X
-InSet expression 150 163 36 66.6 15.0 2.4X
+In expression 366 367 1 27.3 36.6 1.0X
+InSet expression 150 152 2 66.5 15.0 2.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 1929 2031 96 5.2 192.9 1.0X
-InSet expression 149 152 1 67.1 14.9 12.9X
+In expression 1953 2014 103 5.1 195.3 1.0X
+InSet expression 154 156 1 64.8 15.4 12.7X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 20 22 3 49.9 20.0 1.0X
-InSet expression 62 65 3 16.0 62.5 0.3X
+In expression 20 22 3 49.4 20.2 1.0X
+InSet expression 58 60 1 17.2 58.1 0.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 25 27 2 39.3 25.5 1.0X
-InSet expression 64 66 2 15.6 64.2 0.4X
+In expression 25 27 1 39.9 25.0 1.0X
+InSet expression 60 62 2 16.6 60.4 0.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 41 43 2 24.4 41.0 1.0X
-InSet expression 65 67 2 15.3 65.3 0.6X
+In expression 41 43 1 24.3 41.1 1.0X
+InSet expression 60 63 2 16.5 60.5 0.7X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 79 81 3 12.6 79.1 1.0X
-InSet expression 70 72 1 14.4 69.7 1.1X
+In expression 79 80 1 12.6 79.3 1.0X
+InSet expression 65 68 2 15.4 65.1 1.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 236 237 2 4.2 235.9 1.0X
-InSet expression 69 71 2 14.4 69.3 3.4X
+In expression 237 238 2 4.2 236.6 1.0X
+InSet expression 63 64 1 15.9 62.9 3.8X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 558 623 139 1.8 557.5 1.0X
-InSet expression 70 73 2 14.2 70.4 7.9X
+In expression 558 616 124 1.8 558.3 1.0X
+InSet expression 65 67 1 15.5 64.7 8.6X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 5 6 1 201.1 5.0 1.0X
-InSet expression 5 6 1 205.5 4.9 1.0X
+In expression 5 7 1 182.6 5.5 1.0X
+InSet expression 5 7 1 190.9 5.2 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 5 6 1 202.4 4.9 1.0X
-InSet expression 5 6 1 206.3 4.8 1.0X
+In expression 5 6 1 192.6 5.2 1.0X
+InSet expression 5 6 1 201.3 5.0 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 5 6 1 191.2 5.2 1.0X
-InSet expression 5 6 1 190.9 5.2 1.0X
+In expression 6 7 1 180.3 5.5 1.0X
+InSet expression 5 6 1 186.4 5.4 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 6 6 1 168.7 5.9 1.0X
-InSet expression 6 6 1 168.2 5.9 1.0X
+In expression 6 7 1 163.0 6.1 1.0X
+InSet expression 6 7 1 164.1 6.1 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 8 9 1 130.5 7.7 1.0X
-InSet expression 8 8 1 132.0 7.6 1.0X
+In expression 8 10 1 123.9 8.1 1.0X
+InSet expression 8 9 1 124.0 8.1 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 11 12 2 90.9 11.0 1.0X
-InSet expression 11 12 1 90.6 11.0 1.0X
+In expression 11 12 1 88.2 11.3 1.0X
+InSet expression 11 13 2 89.3 11.2 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 30 31 1 33.5 29.9 1.0X
-InSet expression 47 48 1 21.2 47.1 0.6X
+In expression 29 31 1 34.3 29.2 1.0X
+InSet expression 47 49 2 21.1 47.4 0.6X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 35 38 3 28.4 35.3 1.0X
-InSet expression 49 51 2 20.2 49.4 0.7X
+In expression 35 36 1 28.8 34.7 1.0X
+InSet expression 49 50 1 20.6 48.5 0.7X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 44 45 1 22.8 43.9 1.0X
-InSet expression 53 55 2 18.7 53.4 0.8X
+In expression 43 44 1 23.4 42.8 1.0X
+InSet expression 53 54 1 19.0 52.6 0.8X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 59 60 2 17.0 58.7 1.0X
-InSet expression 57 59 1 17.5 57.3 1.0X
+In expression 57 59 1 17.5 57.3 1.0X
+InSet expression 55 58 4 18.0 55.4 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 88 90 2 11.3 88.4 1.0X
-InSet expression 53 55 1 18.8 53.3 1.7X
+In expression 86 87 1 11.6 85.9 1.0X
+InSet expression 52 54 2 19.2 52.0 1.7X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 394 398 9 2.5 393.9 1.0X
-InSet expression 54 56 2 18.6 53.7 7.3X
+In expression 378 379 1 2.6 377.6 1.0X
+InSet expression 52 55 2 19.1 52.5 7.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 16 19 2 607.4 1.6 1.0X
-InSet expression 89 91 1 111.7 8.9 0.2X
+In expression 16 18 3 623.0 1.6 1.0X
+InSet expression 90 92 1 110.8 9.0 0.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 24 26 1 412.7 2.4 1.0X
-InSet expression 100 101 1 100.2 10.0 0.2X
+In expression 23 24 1 430.0 2.3 1.0X
+InSet expression 101 107 16 99.4 10.1 0.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 47 48 2 210.7 4.7 1.0X
-InSet expression 127 129 1 79.0 12.7 0.4X
+In expression 46 47 1 219.5 4.6 1.0X
+InSet expression 127 129 4 78.9 12.7 0.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 82 84 2 121.4 8.2 1.0X
-InSet expression 154 156 2 65.0 15.4 0.5X
+In expression 81 82 1 123.7 8.1 1.0X
+InSet expression 154 156 2 64.9 15.4 0.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 154 154 1 65.1 15.4 1.0X
-InSet expression 132 135 3 75.5 13.2 1.2X
+In expression 153 153 1 65.4 15.3 1.0X
+InSet expression 135 136 1 74.2 13.5 1.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 295 306 12 33.9 29.5 1.0X
-InSet expression 128 130 2 78.1 12.8 2.3X
+In expression 295 306 13 33.9 29.5 1.0X
+InSet expression 130 131 1 77.2 13.0 2.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 371 373 3 27.0 37.1 1.0X
-InSet expression 365 371 5 27.4 36.5 1.0X
+In expression 367 367 1 27.3 36.7 1.0X
+InSet expression 363 364 1 27.5 36.3 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 380 382 1 26.3 38.0 1.0X
-InSet expression 364 365 1 27.5 36.4 1.0X
+In expression 373 380 5 26.8 37.3 1.0X
+InSet expression 369 374 3 27.1 36.9 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 404 407 3 24.7 40.4 1.0X
-InSet expression 369 371 1 27.1 36.9 1.1X
+In expression 413 415 1 24.2 41.3 1.0X
+InSet expression 362 363 1 27.6 36.2 1.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 460 462 3 21.8 46.0 1.0X
-InSet expression 363 365 2 27.5 36.3 1.3X
+In expression 457 461 6 21.9 45.7 1.0X
+InSet expression 366 367 1 27.3 36.6 1.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 543 552 12 18.4 54.3 1.0X
-InSet expression 370 372 3 27.1 37.0 1.5X
+In expression 545 546 1 18.4 54.5 1.0X
+InSet expression 364 372 15 27.5 36.4 1.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 706 711 6 14.2 70.6 1.0X
-InSet expression 371 376 5 26.9 37.1 1.9X
+In expression 707 708 1 14.1 70.7 1.0X
+InSet expression 368 378 7 27.2 36.8 1.9X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
300 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 868 882 10 11.5 86.8 1.0X
-InSet expression 366 376 6 27.4 36.6 2.4X
+In expression 866 868 3 11.6 86.6 1.0X
+InSet expression 384 386 1 26.0 38.4 2.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
400 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 1026 1041 9 9.7 102.6 1.0X
-InSet expression 370 379 7 27.0 37.0 2.8X
+In expression 1031 1034 2 9.7 103.1 1.0X
+InSet expression 385 385 1 26.0 38.5 2.7X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
500 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 1209 1214 6 8.3 120.9 1.0X
-InSet expression 463 471 5 21.6 46.3 2.6X
+In expression 1220 1223 3 8.2 122.0 1.0X
+InSet expression 462 463 2 21.7 46.2 2.6X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 27 29 2 36.8 27.1 1.0X
-InSet expression 58 59 1 17.4 57.6 0.5X
+In expression 25 26 1 40.2 24.9 1.0X
+InSet expression 55 57 1 18.2 54.9 0.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 42 43 2 24.1 41.6 1.0X
-InSet expression 85 87 1 11.8 85.1 0.5X
+In expression 40 41 2 25.0 40.0 1.0X
+InSet expression 83 85 1 12.1 83.0 0.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 181 183 2 5.5 181.2 1.0X
-InSet expression 99 102 4 10.1 99.4 1.8X
+In expression 135 136 1 7.4 135.0 1.0X
+InSet expression 98 100 2 10.2 98.1 1.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 384 389 8 2.6 384.2 1.0X
-InSet expression 128 131 2 7.8 128.1 3.0X
+In expression 372 375 5 2.7 371.7 1.0X
+InSet expression 125 129 4 8.0 125.3 3.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 769 775 5 1.3 769.0 1.0X
-InSet expression 143 146 1 7.0 143.3 5.4X
+In expression 753 755 1 1.3 753.3 1.0X
+InSet expression 141 143 1 7.1 141.0 5.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 1684 1898 330 0.6 1684.1 1.0X
-InSet expression 162 165 4 6.2 162.1 10.4X
+In expression 1502 1786 267 0.7 1501.6 1.0X
+InSet expression 160 160 1 6.3 159.6 9.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
5 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 22 23 1 45.4 22.0 1.0X
-InSet expression 80 83 6 12.4 80.4 0.3X
+In expression 20 22 2 48.8 20.5 1.0X
+InSet expression 77 80 3 12.9 77.2 0.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
10 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 32 33 2 31.6 31.7 1.0X
-InSet expression 121 123 2 8.2 121.3 0.3X
+In expression 30 32 1 33.2 30.1 1.0X
+InSet expression 117 118 1 8.6 116.6 0.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
25 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 85 86 1 11.7 85.2 1.0X
-InSet expression 142 145 3 7.0 142.3 0.6X
+In expression 70 71 1 14.3 70.0 1.0X
+InSet expression 137 142 14 7.3 136.6 0.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
50 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 203 205 4 4.9 202.7 1.0X
-InSet expression 182 186 2 5.5 182.4 1.1X
+In expression 217 220 2 4.6 216.6 1.0X
+InSet expression 177 179 3 5.7 176.6 1.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
100 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 489 511 13 2.0 489.0 1.0X
-InSet expression 205 207 1 4.9 205.0 2.4X
+In expression 510 513 2 2.0 510.3 1.0X
+InSet expression 198 202 5 5.0 198.0 2.6X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
200 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-In expression 1256 1403 264 0.8 1256.3 1.0X
-InSet expression 237 245 11 4.2 237.1 5.3X
+In expression 1222 1430 265 0.8 1222.2 1.0X
+InSet expression 233 235 2 4.3 232.6 5.3X
diff --git a/sql/core/benchmarks/InMemoryColumnarBenchmark-jdk21-results.txt b/sql/core/benchmarks/InMemoryColumnarBenchmark-jdk21-results.txt
index cc5acfe9c1670..e200db961f9cd 100644
--- a/sql/core/benchmarks/InMemoryColumnarBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/InMemoryColumnarBenchmark-jdk21-results.txt
@@ -2,11 +2,11 @@
Int In-memory with 1000000 rows
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Int In-Memory scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------
-columnar deserialization + columnar-to-row 193 253 52 5.2 193.2 1.0X
-row-based deserialization 146 155 9 6.9 145.7 1.3X
+columnar deserialization + columnar-to-row 177 222 65 5.6 177.5 1.0X
+row-based deserialization 129 164 55 7.7 129.5 1.4X
diff --git a/sql/core/benchmarks/InMemoryColumnarBenchmark-results.txt b/sql/core/benchmarks/InMemoryColumnarBenchmark-results.txt
index 45d9390a5602e..51ac715059fae 100644
--- a/sql/core/benchmarks/InMemoryColumnarBenchmark-results.txt
+++ b/sql/core/benchmarks/InMemoryColumnarBenchmark-results.txt
@@ -2,11 +2,11 @@
Int In-memory with 1000000 rows
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Int In-Memory scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------
-columnar deserialization + columnar-to-row 179 220 36 5.6 178.7 1.0X
-row-based deserialization 171 219 70 5.9 170.5 1.0X
+columnar deserialization + columnar-to-row 222 257 41 4.5 222.3 1.0X
+row-based deserialization 140 178 63 7.2 139.8 1.6X
diff --git a/sql/core/benchmarks/InsertTableWithDynamicPartitionsBenchmark-jdk21-results.txt b/sql/core/benchmarks/InsertTableWithDynamicPartitionsBenchmark-jdk21-results.txt
index 9ab15d2804ac9..f86a35823fc47 100644
--- a/sql/core/benchmarks/InsertTableWithDynamicPartitionsBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/InsertTableWithDynamicPartitionsBenchmark-jdk21-results.txt
@@ -1,8 +1,8 @@
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
dynamic insert table benchmark, totalRows = 200000: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------------
-one partition column, 100 partitions 7569 7811 343 0.0 37844.8 1.0X
-two partition columns, 500 partitions 21079 21237 223 0.0 105396.1 0.4X
-three partition columns, 2000 partitions 58077 58838 1077 0.0 290383.9 0.1X
+one partition column, 100 partitions 7661 7709 67 0.0 38305.2 1.0X
+two partition columns, 500 partitions 19979 20122 202 0.0 99894.4 0.4X
+three partition columns, 2000 partitions 56004 56084 113 0.0 280021.2 0.1X
diff --git a/sql/core/benchmarks/InsertTableWithDynamicPartitionsBenchmark-results.txt b/sql/core/benchmarks/InsertTableWithDynamicPartitionsBenchmark-results.txt
index 9168ce1502a16..e5cbfddc48482 100644
--- a/sql/core/benchmarks/InsertTableWithDynamicPartitionsBenchmark-results.txt
+++ b/sql/core/benchmarks/InsertTableWithDynamicPartitionsBenchmark-results.txt
@@ -1,8 +1,8 @@
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
dynamic insert table benchmark, totalRows = 200000: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------------
-one partition column, 100 partitions 7507 7533 37 0.0 37536.8 1.0X
-two partition columns, 500 partitions 20204 20330 178 0.0 101021.9 0.4X
-three partition columns, 2000 partitions 54668 55077 578 0.0 273341.1 0.1X
+one partition column, 100 partitions 7780 7799 26 0.0 38901.1 1.0X
+two partition columns, 500 partitions 20050 20197 207 0.0 100252.3 0.4X
+three partition columns, 2000 partitions 55234 55365 186 0.0 276167.7 0.1X
diff --git a/sql/core/benchmarks/IntervalBenchmark-jdk21-results.txt b/sql/core/benchmarks/IntervalBenchmark-jdk21-results.txt
index 895b83b4c0e5e..5096583abec42 100644
--- a/sql/core/benchmarks/IntervalBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/IntervalBenchmark-jdk21-results.txt
@@ -1,40 +1,40 @@
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
cast strings to intervals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-prepare string w/ interval 423 439 15 2.4 423.2 1.0X
-prepare string w/o interval 384 393 9 2.6 383.6 1.1X
-1 units w/ interval 319 325 7 3.1 319.0 1.3X
-1 units w/o interval 348 351 3 2.9 348.4 1.2X
-2 units w/ interval 547 558 12 1.8 546.7 0.8X
-2 units w/o interval 530 549 18 1.9 529.8 0.8X
-3 units w/ interval 1074 1082 11 0.9 1074.1 0.4X
-3 units w/o interval 1061 1077 18 0.9 1061.5 0.4X
-4 units w/ interval 1208 1222 13 0.8 1208.0 0.4X
-4 units w/o interval 1216 1225 15 0.8 1216.4 0.3X
-5 units w/ interval 1340 1345 4 0.7 1339.9 0.3X
-5 units w/o interval 1333 1340 7 0.8 1332.9 0.3X
-6 units w/ interval 1481 1502 29 0.7 1481.4 0.3X
-6 units w/o interval 1513 1517 3 0.7 1513.4 0.3X
-7 units w/ interval 1835 1842 9 0.5 1834.7 0.2X
-7 units w/o interval 1830 1840 9 0.5 1829.8 0.2X
-8 units w/ interval 1931 1939 7 0.5 1931.2 0.2X
-8 units w/o interval 1920 1924 3 0.5 1920.4 0.2X
-9 units w/ interval 2051 2055 4 0.5 2050.7 0.2X
-9 units w/o interval 2061 2064 4 0.5 2060.6 0.2X
-10 units w/ interval 2346 2363 25 0.4 2346.0 0.2X
-10 units w/o interval 2337 2345 13 0.4 2336.7 0.2X
-11 units w/ interval 2979 2995 28 0.3 2979.0 0.1X
-11 units w/o interval 2966 2979 11 0.3 2966.0 0.1X
+prepare string w/ interval 409 451 37 2.4 409.1 1.0X
+prepare string w/o interval 387 407 18 2.6 387.1 1.1X
+1 units w/ interval 338 341 5 3.0 337.9 1.2X
+1 units w/o interval 358 369 10 2.8 358.0 1.1X
+2 units w/ interval 559 565 6 1.8 559.4 0.7X
+2 units w/o interval 542 555 15 1.8 542.2 0.8X
+3 units w/ interval 1057 1066 9 0.9 1057.4 0.4X
+3 units w/o interval 1047 1052 4 1.0 1047.2 0.4X
+4 units w/ interval 1209 1218 13 0.8 1208.8 0.3X
+4 units w/o interval 1208 1213 8 0.8 1208.1 0.3X
+5 units w/ interval 1315 1317 2 0.8 1315.3 0.3X
+5 units w/o interval 1321 1324 4 0.8 1320.5 0.3X
+6 units w/ interval 1457 1457 0 0.7 1457.0 0.3X
+6 units w/o interval 1452 1457 4 0.7 1451.7 0.3X
+7 units w/ interval 1816 1821 5 0.6 1815.6 0.2X
+7 units w/o interval 1836 1836 0 0.5 1835.8 0.2X
+8 units w/ interval 1902 1907 4 0.5 1902.3 0.2X
+8 units w/o interval 1900 1901 1 0.5 1899.6 0.2X
+9 units w/ interval 2046 2055 13 0.5 2045.8 0.2X
+9 units w/o interval 2061 2062 2 0.5 2060.7 0.2X
+10 units w/ interval 2331 2338 8 0.4 2330.9 0.2X
+10 units w/o interval 2325 2326 1 0.4 2325.0 0.2X
+11 units w/ interval 2507 2516 8 0.4 2507.4 0.2X
+11 units w/o interval 2508 2511 5 0.4 2507.8 0.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
make_interval(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------
-prepare make_interval() 328 337 7 3.0 328.4 1.0X
-make_interval(0, 1, 2, 3, 4, 5, 50.123456) 44 45 1 22.7 44.0 7.5X
-make_interval(*, *, 2, 3, 4, 5, 50.123456) 52 62 12 19.2 52.1 6.3X
-make_interval(0, 1, *, *, 4, 5, 50.123456) 56 58 3 17.9 56.0 5.9X
-make_interval(0, 1, 2, 3, *, *, *) 319 321 2 3.1 318.8 1.0X
-make_interval(*, *, *, *, *, *, *) 341 343 4 2.9 340.7 1.0X
+prepare make_interval() 328 329 1 3.0 328.1 1.0X
+make_interval(0, 1, 2, 3, 4, 5, 50.123456) 48 51 3 20.8 48.1 6.8X
+make_interval(*, *, 2, 3, 4, 5, 50.123456) 53 58 7 18.9 53.0 6.2X
+make_interval(0, 1, *, *, 4, 5, 50.123456) 57 57 1 17.6 56.8 5.8X
+make_interval(0, 1, 2, 3, *, *, *) 322 323 1 3.1 321.8 1.0X
+make_interval(*, *, *, *, *, *, *) 332 334 2 3.0 332.2 1.0X
diff --git a/sql/core/benchmarks/IntervalBenchmark-results.txt b/sql/core/benchmarks/IntervalBenchmark-results.txt
index c0a7962c2dc5f..fed39f6a291d2 100644
--- a/sql/core/benchmarks/IntervalBenchmark-results.txt
+++ b/sql/core/benchmarks/IntervalBenchmark-results.txt
@@ -1,40 +1,40 @@
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
cast strings to intervals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-prepare string w/ interval 395 404 10 2.5 395.1 1.0X
-prepare string w/o interval 368 378 16 2.7 367.6 1.1X
-1 units w/ interval 360 363 3 2.8 359.6 1.1X
-1 units w/o interval 346 352 9 2.9 345.7 1.1X
-2 units w/ interval 508 510 2 2.0 508.1 0.8X
-2 units w/o interval 490 494 5 2.0 490.3 0.8X
-3 units w/ interval 1127 1129 2 0.9 1126.9 0.4X
-3 units w/o interval 1113 1124 20 0.9 1112.9 0.4X
-4 units w/ interval 1265 1283 17 0.8 1265.0 0.3X
-4 units w/o interval 1248 1253 9 0.8 1247.7 0.3X
-5 units w/ interval 1388 1390 1 0.7 1388.5 0.3X
-5 units w/o interval 1379 1382 4 0.7 1379.2 0.3X
-6 units w/ interval 1530 1533 2 0.7 1530.2 0.3X
-6 units w/o interval 1510 1513 6 0.7 1509.6 0.3X
-7 units w/ interval 1866 1877 14 0.5 1866.2 0.2X
-7 units w/o interval 1841 1845 6 0.5 1840.7 0.2X
-8 units w/ interval 2068 2080 14 0.5 2068.1 0.2X
-8 units w/o interval 2062 2066 3 0.5 2061.9 0.2X
-9 units w/ interval 2642 2650 13 0.4 2641.6 0.1X
-9 units w/o interval 2626 2634 8 0.4 2625.9 0.2X
-10 units w/ interval 2817 2832 17 0.4 2817.3 0.1X
-10 units w/o interval 2775 2779 5 0.4 2774.9 0.1X
-11 units w/ interval 3096 3100 6 0.3 3095.8 0.1X
-11 units w/o interval 3080 3084 4 0.3 3080.1 0.1X
+prepare string w/ interval 403 419 13 2.5 403.2 1.0X
+prepare string w/o interval 389 390 1 2.6 388.8 1.0X
+1 units w/ interval 362 366 5 2.8 362.2 1.1X
+1 units w/o interval 363 370 6 2.8 362.8 1.1X
+2 units w/ interval 513 520 6 1.9 513.2 0.8X
+2 units w/o interval 499 503 3 2.0 499.3 0.8X
+3 units w/ interval 1135 1136 1 0.9 1135.1 0.4X
+3 units w/o interval 1125 1126 2 0.9 1124.9 0.4X
+4 units w/ interval 1226 1238 16 0.8 1226.2 0.3X
+4 units w/o interval 1233 1238 8 0.8 1233.2 0.3X
+5 units w/ interval 1384 1389 7 0.7 1383.7 0.3X
+5 units w/o interval 1366 1368 1 0.7 1366.2 0.3X
+6 units w/ interval 1547 1550 2 0.6 1547.4 0.3X
+6 units w/o interval 1526 1528 2 0.7 1526.4 0.3X
+7 units w/ interval 1858 1863 5 0.5 1858.5 0.2X
+7 units w/o interval 1853 1856 3 0.5 1853.3 0.2X
+8 units w/ interval 2063 2083 23 0.5 2062.7 0.2X
+8 units w/o interval 2049 2055 7 0.5 2048.5 0.2X
+9 units w/ interval 2599 2609 13 0.4 2599.2 0.2X
+9 units w/o interval 2586 2588 2 0.4 2585.9 0.2X
+10 units w/ interval 2811 2817 6 0.4 2811.4 0.1X
+10 units w/o interval 2788 2791 5 0.4 2788.4 0.1X
+11 units w/ interval 3070 3074 4 0.3 3069.9 0.1X
+11 units w/o interval 3056 3060 4 0.3 3056.3 0.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
make_interval(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------
-prepare make_interval() 344 348 4 2.9 343.8 1.0X
-make_interval(0, 1, 2, 3, 4, 5, 50.123456) 40 42 2 24.7 40.5 8.5X
-make_interval(*, *, 2, 3, 4, 5, 50.123456) 50 54 4 19.8 50.4 6.8X
-make_interval(0, 1, *, *, 4, 5, 50.123456) 55 59 3 18.1 55.1 6.2X
-make_interval(0, 1, 2, 3, *, *, *) 329 336 6 3.0 329.4 1.0X
-make_interval(*, *, *, *, *, *, *) 337 339 2 3.0 336.9 1.0X
+prepare make_interval() 350 352 2 2.9 349.8 1.0X
+make_interval(0, 1, 2, 3, 4, 5, 50.123456) 47 51 6 21.5 46.6 7.5X
+make_interval(*, *, 2, 3, 4, 5, 50.123456) 55 58 4 18.0 55.5 6.3X
+make_interval(0, 1, *, *, 4, 5, 50.123456) 56 58 2 17.9 55.9 6.3X
+make_interval(0, 1, 2, 3, *, *, *) 336 338 3 3.0 335.8 1.0X
+make_interval(*, *, *, *, *, *, *) 342 343 1 2.9 342.1 1.0X
diff --git a/sql/core/benchmarks/JoinBenchmark-jdk21-results.txt b/sql/core/benchmarks/JoinBenchmark-jdk21-results.txt
index cff5e95c7b36b..cc31554ecdfbf 100644
--- a/sql/core/benchmarks/JoinBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/JoinBenchmark-jdk21-results.txt
@@ -2,81 +2,81 @@
Join Benchmark
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Join w long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Join w long wholestage off 2249 2276 38 9.3 107.2 1.0X
-Join w long wholestage on 1050 1088 41 20.0 50.1 2.1X
+Join w long wholestage off 2144 2152 11 9.8 102.2 1.0X
+Join w long wholestage on 913 938 19 23.0 43.5 2.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Join w long duplicated: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Join w long duplicated wholestage off 2043 2046 4 10.3 97.4 1.0X
-Join w long duplicated wholestage on 902 921 18 23.3 43.0 2.3X
+Join w long duplicated wholestage off 2025 2029 6 10.4 96.6 1.0X
+Join w long duplicated wholestage on 947 997 28 22.1 45.2 2.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Join w 2 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Join w 2 ints wholestage off 107986 107997 15 0.2 5149.2 1.0X
-Join w 2 ints wholestage on 106359 106444 62 0.2 5071.6 1.0X
+Join w 2 ints wholestage off 108340 108384 62 0.2 5166.1 1.0X
+Join w 2 ints wholestage on 106411 106556 130 0.2 5074.1 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Join w 2 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Join w 2 longs wholestage off 3719 3858 196 5.6 177.4 1.0X
-Join w 2 longs wholestage on 2415 2498 54 8.7 115.2 1.5X
+Join w 2 longs wholestage off 3530 3550 28 5.9 168.3 1.0X
+Join w 2 longs wholestage on 1968 1989 14 10.7 93.8 1.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Join w 2 longs duplicated: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Join w 2 longs duplicated wholestage off 8503 8531 39 2.5 405.5 1.0X
-Join w 2 longs duplicated wholestage on 5581 5695 74 3.8 266.1 1.5X
+Join w 2 longs duplicated wholestage off 8952 9029 109 2.3 426.8 1.0X
+Join w 2 longs duplicated wholestage on 5694 5767 67 3.7 271.5 1.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
outer join w long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-outer join w long wholestage off 1770 1812 59 11.8 84.4 1.0X
-outer join w long wholestage on 1057 1096 32 19.8 50.4 1.7X
+outer join w long wholestage off 1574 1576 3 13.3 75.1 1.0X
+outer join w long wholestage on 934 993 102 22.4 44.6 1.7X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
semi join w long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-semi join w long wholestage off 1064 1068 6 19.7 50.7 1.0X
-semi join w long wholestage on 553 566 12 37.9 26.4 1.9X
+semi join w long wholestage off 1060 1063 4 19.8 50.6 1.0X
+semi join w long wholestage on 555 578 15 37.8 26.5 1.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
sort merge join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-sort merge join wholestage off 535 541 9 3.9 254.9 1.0X
-sort merge join wholestage on 470 482 14 4.5 224.0 1.1X
+sort merge join wholestage off 507 514 9 4.1 241.8 1.0X
+sort merge join wholestage on 461 468 6 4.5 219.8 1.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
sort merge join with duplicates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------
-sort merge join with duplicates wholestage off 1056 1057 1 2.0 503.7 1.0X
-sort merge join with duplicates wholestage on 955 966 7 2.2 455.6 1.1X
+sort merge join with duplicates wholestage off 1013 1015 2 2.1 483.0 1.0X
+sort merge join with duplicates wholestage on 921 931 10 2.3 439.0 1.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
shuffle hash join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-shuffle hash join wholestage off 536 554 25 7.8 127.9 1.0X
-shuffle hash join wholestage on 416 433 14 10.1 99.1 1.3X
+shuffle hash join wholestage off 516 528 17 8.1 123.1 1.0X
+shuffle hash join wholestage on 370 391 17 11.3 88.3 1.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
broadcast nested loop join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
-broadcast nested loop join wholestage off 27126 27236 155 0.8 1293.5 1.0X
-broadcast nested loop join wholestage on 21319 21567 223 1.0 1016.6 1.3X
+broadcast nested loop join wholestage off 25481 25537 80 0.8 1215.0 1.0X
+broadcast nested loop join wholestage on 17573 17808 226 1.2 838.0 1.4X
diff --git a/sql/core/benchmarks/JoinBenchmark-results.txt b/sql/core/benchmarks/JoinBenchmark-results.txt
index 9d546eff9eb1d..71fad9100fe82 100644
--- a/sql/core/benchmarks/JoinBenchmark-results.txt
+++ b/sql/core/benchmarks/JoinBenchmark-results.txt
@@ -2,81 +2,81 @@
Join Benchmark
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Join w long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Join w long wholestage off 2042 2057 21 10.3 97.4 1.0X
-Join w long wholestage on 955 1006 70 22.0 45.5 2.1X
+Join w long wholestage off 2023 2060 52 10.4 96.5 1.0X
+Join w long wholestage on 1018 1066 29 20.6 48.5 2.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Join w long duplicated: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Join w long duplicated wholestage off 1989 2101 159 10.5 94.8 1.0X
-Join w long duplicated wholestage on 905 925 22 23.2 43.1 2.2X
+Join w long duplicated wholestage off 1960 1982 32 10.7 93.4 1.0X
+Join w long duplicated wholestage on 901 928 31 23.3 43.0 2.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Join w 2 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Join w 2 ints wholestage off 112857 112868 15 0.2 5381.5 1.0X
-Join w 2 ints wholestage on 111346 111409 65 0.2 5309.4 1.0X
+Join w 2 ints wholestage off 112829 112886 81 0.2 5380.1 1.0X
+Join w 2 ints wholestage on 111531 111580 43 0.2 5318.2 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Join w 2 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Join w 2 longs wholestage off 3248 3257 13 6.5 154.9 1.0X
-Join w 2 longs wholestage on 2023 2051 36 10.4 96.5 1.6X
+Join w 2 longs wholestage off 3306 3312 8 6.3 157.7 1.0X
+Join w 2 longs wholestage on 2188 2225 37 9.6 104.3 1.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Join w 2 longs duplicated: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Join w 2 longs duplicated wholestage off 8464 8469 6 2.5 403.6 1.0X
-Join w 2 longs duplicated wholestage on 5481 5504 17 3.8 261.4 1.5X
+Join w 2 longs duplicated wholestage off 8196 8221 35 2.6 390.8 1.0X
+Join w 2 longs duplicated wholestage on 5451 5509 35 3.8 259.9 1.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
outer join w long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-outer join w long wholestage off 1590 1598 11 13.2 75.8 1.0X
-outer join w long wholestage on 942 947 4 22.3 44.9 1.7X
+outer join w long wholestage off 1710 1712 3 12.3 81.5 1.0X
+outer join w long wholestage on 1036 1045 10 20.2 49.4 1.7X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
semi join w long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-semi join w long wholestage off 1074 1076 3 19.5 51.2 1.0X
-semi join w long wholestage on 529 559 32 39.6 25.2 2.0X
+semi join w long wholestage off 1077 1116 56 19.5 51.4 1.0X
+semi join w long wholestage on 526 548 36 39.9 25.1 2.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
sort merge join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-sort merge join wholestage off 542 543 2 3.9 258.5 1.0X
-sort merge join wholestage on 467 480 16 4.5 222.8 1.2X
+sort merge join wholestage off 533 538 7 3.9 254.1 1.0X
+sort merge join wholestage on 460 470 6 4.6 219.2 1.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
sort merge join with duplicates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------
-sort merge join with duplicates wholestage off 1074 1076 3 2.0 512.1 1.0X
-sort merge join with duplicates wholestage on 957 966 6 2.2 456.4 1.1X
+sort merge join with duplicates wholestage off 1052 1054 3 2.0 501.5 1.0X
+sort merge join with duplicates wholestage on 944 950 6 2.2 450.1 1.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
shuffle hash join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-shuffle hash join wholestage off 484 486 3 8.7 115.4 1.0X
-shuffle hash join wholestage on 366 373 8 11.5 87.1 1.3X
+shuffle hash join wholestage off 483 487 6 8.7 115.1 1.0X
+shuffle hash join wholestage on 367 378 9 11.4 87.5 1.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
broadcast nested loop join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
-broadcast nested loop join wholestage off 26051 26090 55 0.8 1242.2 1.0X
-broadcast nested loop join wholestage on 17018 17068 32 1.2 811.5 1.5X
+broadcast nested loop join wholestage off 27392 27733 482 0.8 1306.2 1.0X
+broadcast nested loop join wholestage on 19262 19470 214 1.1 918.5 1.4X
diff --git a/sql/core/benchmarks/JsonBenchmark-jdk21-results.txt b/sql/core/benchmarks/JsonBenchmark-jdk21-results.txt
index 3a126b6f0da15..8222b122c0d5b 100644
--- a/sql/core/benchmarks/JsonBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/JsonBenchmark-jdk21-results.txt
@@ -3,128 +3,128 @@ Benchmark for performance of JSON parsing
================================================================================================
Preparing data for benchmarking ...
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
JSON schema inferring: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-No encoding 2253 2358 117 2.2 450.6 1.0X
-UTF-8 is set 4846 4861 13 1.0 969.2 0.5X
+No encoding 2449 2522 100 2.0 489.8 1.0X
+UTF-8 is set 4926 4949 35 1.0 985.2 0.5X
Preparing data for benchmarking ...
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
count a short column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-No encoding 2105 2123 18 2.4 421.0 1.0X
-UTF-8 is set 4657 4664 10 1.1 931.3 0.5X
+No encoding 1976 2030 55 2.5 395.3 1.0X
+UTF-8 is set 4585 4593 10 1.1 917.0 0.4X
Preparing data for benchmarking ...
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
count a wide column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-No encoding 4551 4598 77 0.2 4550.7 1.0X
-UTF-8 is set 4372 4379 8 0.2 4372.2 1.0X
+No encoding 4544 4562 18 0.2 4543.6 1.0X
+UTF-8 is set 4417 4433 14 0.2 4417.2 1.0X
Preparing data for benchmarking ...
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
select wide row: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-No encoding 9258 9360 152 0.0 185154.2 1.0X
-UTF-8 is set 10320 10346 23 0.0 206392.2 0.9X
+No encoding 10431 10576 227 0.0 208628.1 1.0X
+UTF-8 is set 11244 11277 30 0.0 224872.2 0.9X
Preparing data for benchmarking ...
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select a subset of 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Select 10 columns 1712 1713 1 0.6 1712.4 1.0X
-Select 1 column 1291 1298 10 0.8 1290.9 1.3X
+Select 10 columns 1735 1736 2 0.6 1734.5 1.0X
+Select 1 column 1146 1148 2 0.9 1146.2 1.5X
Preparing data for benchmarking ...
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
creation of JSON parser per line: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Short column without encoding 647 649 4 1.5 647.0 1.0X
-Short column with UTF-8 1169 1180 10 0.9 1169.2 0.6X
-Wide column without encoding 5268 5293 27 0.2 5267.8 0.1X
-Wide column with UTF-8 6536 6557 20 0.2 6535.9 0.1X
+Short column without encoding 601 603 3 1.7 600.7 1.0X
+Short column with UTF-8 1130 1132 2 0.9 1129.9 0.5X
+Wide column without encoding 5232 5270 41 0.2 5231.8 0.1X
+Wide column with UTF-8 8895 8907 10 0.1 8895.4 0.1X
Preparing data for benchmarking ...
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
JSON functions: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Text read 57 65 7 17.5 57.2 1.0X
-from_json 1080 1085 7 0.9 1079.9 0.1X
-json_tuple 1010 1012 2 1.0 1010.4 0.1X
-get_json_object wholestage off 1033 1038 4 1.0 1033.4 0.1X
-get_json_object wholestage on 951 952 2 1.1 950.9 0.1X
+Text read 59 65 5 16.9 59.1 1.0X
+from_json 1090 1103 14 0.9 1090.2 0.1X
+json_tuple 1044 1053 8 1.0 1043.6 0.1X
+get_json_object wholestage off 1052 1059 6 1.0 1052.4 0.1X
+get_json_object wholestage on 983 998 13 1.0 983.1 0.1X
Preparing data for benchmarking ...
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Dataset of json strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Text read 259 262 3 19.3 51.8 1.0X
-schema inferring 1918 1934 15 2.6 383.5 0.1X
-parsing 2804 2815 11 1.8 560.8 0.1X
+Text read 234 236 2 21.3 46.8 1.0X
+schema inferring 1793 1797 5 2.8 358.6 0.1X
+parsing 2582 2583 1 1.9 516.4 0.1X
Preparing data for benchmarking ...
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Json files in the per-line mode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Text read 583 592 8 8.6 116.7 1.0X
-Schema inferring 2412 2414 2 2.1 482.4 0.2X
-Parsing without charset 2962 2970 9 1.7 592.3 0.2X
-Parsing with UTF-8 5458 5462 5 0.9 1091.7 0.1X
+Text read 593 596 3 8.4 118.5 1.0X
+Schema inferring 2369 2370 1 2.1 473.7 0.3X
+Parsing without charset 2771 2776 6 1.8 554.3 0.2X
+Parsing with UTF-8 5392 5400 9 0.9 1078.4 0.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Create a dataset of timestamps 105 109 5 9.5 105.2 1.0X
-to_json(timestamp) 550 551 2 1.8 549.9 0.2X
-write timestamps to files 582 586 3 1.7 582.4 0.2X
-Create a dataset of dates 113 114 2 8.9 112.8 0.9X
-to_json(date) 417 419 3 2.4 416.8 0.3X
-write dates to files 390 392 2 2.6 390.2 0.3X
+Create a dataset of timestamps 105 107 2 9.5 105.2 1.0X
+to_json(timestamp) 541 546 8 1.8 540.6 0.2X
+write timestamps to files 572 574 3 1.7 571.9 0.2X
+Create a dataset of dates 110 113 3 9.1 110.2 1.0X
+to_json(date) 385 386 1 2.6 384.8 0.3X
+write dates to files 414 417 3 2.4 413.6 0.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------------------------------
-read timestamp text from files 150 154 3 6.6 150.4 1.0X
-read timestamps from files 1131 1149 19 0.9 1130.9 0.1X
-infer timestamps from files 2155 2156 0 0.5 2155.3 0.1X
-read date text from files 138 143 4 7.2 138.3 1.1X
-read date from files 718 721 5 1.4 718.1 0.2X
-timestamp strings 142 143 1 7.0 142.4 1.1X
-parse timestamps from Dataset[String] 1270 1272 3 0.8 1269.8 0.1X
-infer timestamps from Dataset[String] 2224 2227 3 0.4 2224.4 0.1X
-date strings 195 196 1 5.1 195.3 0.8X
-parse dates from Dataset[String] 972 973 1 1.0 972.4 0.2X
-from_json(timestamp) 1739 1742 3 0.6 1739.4 0.1X
-from_json(date) 1400 1401 1 0.7 1400.2 0.1X
-infer error timestamps from Dataset[String] with default format 1433 1437 4 0.7 1433.0 0.1X
-infer error timestamps from Dataset[String] with user-provided format 1439 1442 3 0.7 1439.0 0.1X
-infer error timestamps from Dataset[String] with legacy format 1475 1480 4 0.7 1475.3 0.1X
+read timestamp text from files 158 160 3 6.3 157.5 1.0X
+read timestamps from files 1078 1084 5 0.9 1078.0 0.1X
+infer timestamps from files 2034 2041 11 0.5 2034.1 0.1X
+read date text from files 146 153 6 6.8 146.3 1.1X
+read date from files 674 675 2 1.5 673.7 0.2X
+timestamp strings 136 138 2 7.4 135.6 1.2X
+parse timestamps from Dataset[String] 1255 1260 6 0.8 1254.7 0.1X
+infer timestamps from Dataset[String] 2203 2210 6 0.5 2203.3 0.1X
+date strings 202 202 0 5.0 201.8 0.8X
+parse dates from Dataset[String] 955 959 5 1.0 954.8 0.2X
+from_json(timestamp) 1731 1736 6 0.6 1731.2 0.1X
+from_json(date) 1438 1440 3 0.7 1437.9 0.1X
+infer error timestamps from Dataset[String] with default format 1352 1358 4 0.7 1352.5 0.1X
+infer error timestamps from Dataset[String] with user-provided format 1352 1352 0 0.7 1351.6 0.1X
+infer error timestamps from Dataset[String] with legacy format 1378 1381 4 0.7 1377.7 0.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-w/o filters 6068 6076 11 0.0 60681.7 1.0X
-pushdown disabled 5926 5934 10 0.0 59263.4 1.0X
-w/ filters 650 654 4 0.2 6496.2 9.3X
+w/o filters 6114 6115 1 0.0 61142.8 1.0X
+pushdown disabled 6017 6030 12 0.0 60165.6 1.0X
+w/ filters 712 721 7 0.1 7124.1 8.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Partial JSON results: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-parse invalid JSON 2249 2258 14 0.0 224874.5 1.0X
+parse invalid JSON 2534 2540 7 0.0 253448.0 1.0X
diff --git a/sql/core/benchmarks/JsonBenchmark-results.txt b/sql/core/benchmarks/JsonBenchmark-results.txt
index f3115219608e8..43f7ce63768a6 100644
--- a/sql/core/benchmarks/JsonBenchmark-results.txt
+++ b/sql/core/benchmarks/JsonBenchmark-results.txt
@@ -3,128 +3,128 @@ Benchmark for performance of JSON parsing
================================================================================================
Preparing data for benchmarking ...
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
JSON schema inferring: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-No encoding 2310 2321 12 2.2 461.9 1.0X
-UTF-8 is set 4784 4795 10 1.0 956.9 0.5X
+No encoding 2179 2239 75 2.3 435.9 1.0X
+UTF-8 is set 5291 5328 33 0.9 1058.2 0.4X
Preparing data for benchmarking ...
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
count a short column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-No encoding 2215 2246 30 2.3 443.0 1.0X
-UTF-8 is set 4718 4724 7 1.1 943.5 0.5X
+No encoding 2029 2079 72 2.5 405.8 1.0X
+UTF-8 is set 4499 4501 2 1.1 899.8 0.5X
Preparing data for benchmarking ...
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
count a wide column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-No encoding 4269 4376 151 0.2 4268.9 1.0X
-UTF-8 is set 4447 4459 15 0.2 4446.8 1.0X
+No encoding 4605 4651 40 0.2 4604.9 1.0X
+UTF-8 is set 4661 4685 24 0.2 4660.5 1.0X
Preparing data for benchmarking ...
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
select wide row: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-No encoding 9025 9101 70 0.0 180505.7 1.0X
-UTF-8 is set 9983 10004 24 0.0 199666.6 0.9X
+No encoding 8833 8999 215 0.0 176664.5 1.0X
+UTF-8 is set 9740 9745 5 0.0 194807.7 0.9X
Preparing data for benchmarking ...
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Select a subset of 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Select 10 columns 1585 1588 5 0.6 1584.6 1.0X
-Select 1 column 1253 1256 4 0.8 1253.2 1.3X
+Select 10 columns 1487 1497 12 0.7 1487.0 1.0X
+Select 1 column 1083 1088 4 0.9 1083.4 1.4X
Preparing data for benchmarking ...
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
creation of JSON parser per line: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Short column without encoding 644 646 2 1.6 644.2 1.0X
-Short column with UTF-8 1163 1174 18 0.9 1162.6 0.6X
-Wide column without encoding 5178 5191 12 0.2 5177.7 0.1X
-Wide column with UTF-8 5983 5987 4 0.2 5982.7 0.1X
+Short column without encoding 600 607 7 1.7 600.1 1.0X
+Short column with UTF-8 1073 1081 9 0.9 1072.8 0.6X
+Wide column without encoding 5205 5216 15 0.2 5205.5 0.1X
+Wide column with UTF-8 6720 6730 10 0.1 6719.9 0.1X
Preparing data for benchmarking ...
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
JSON functions: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Text read 56 59 3 17.9 55.8 1.0X
-from_json 1039 1057 28 1.0 1038.8 0.1X
-json_tuple 1036 1040 5 1.0 1035.5 0.1X
-get_json_object wholestage off 1056 1061 8 0.9 1055.9 0.1X
-get_json_object wholestage on 993 999 6 1.0 992.7 0.1X
+Text read 64 71 6 15.6 64.1 1.0X
+from_json 1058 1061 2 0.9 1058.5 0.1X
+json_tuple 1004 1006 3 1.0 1003.6 0.1X
+get_json_object wholestage off 1036 1044 7 1.0 1036.3 0.1X
+get_json_object wholestage on 953 956 3 1.0 953.0 0.1X
Preparing data for benchmarking ...
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Dataset of json strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Text read 260 263 4 19.2 52.0 1.0X
-schema inferring 1765 1768 2 2.8 353.0 0.1X
-parsing 2756 2775 18 1.8 551.2 0.1X
+Text read 249 253 6 20.0 49.9 1.0X
+schema inferring 1822 1831 15 2.7 364.4 0.1X
+parsing 2444 2456 10 2.0 488.8 0.1X
Preparing data for benchmarking ...
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Json files in the per-line mode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Text read 580 583 2 8.6 116.1 1.0X
-Schema inferring 2349 2355 8 2.1 469.8 0.2X
-Parsing without charset 2935 2940 8 1.7 587.0 0.2X
-Parsing with UTF-8 5463 5504 69 0.9 1092.6 0.1X
+Text read 602 603 1 8.3 120.4 1.0X
+Schema inferring 2349 2352 3 2.1 469.7 0.3X
+Parsing without charset 2701 2709 6 1.9 540.2 0.2X
+Parsing with UTF-8 5177 5184 7 1.0 1035.3 0.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Create a dataset of timestamps 102 106 6 9.8 101.9 1.0X
-to_json(timestamp) 634 637 3 1.6 634.4 0.2X
-write timestamps to files 676 678 2 1.5 676.2 0.2X
-Create a dataset of dates 114 117 2 8.7 114.4 0.9X
-to_json(date) 439 440 1 2.3 439.1 0.2X
-write dates to files 435 438 2 2.3 435.2 0.2X
+Create a dataset of timestamps 110 113 3 9.1 110.3 1.0X
+to_json(timestamp) 633 637 6 1.6 632.6 0.2X
+write timestamps to files 668 672 4 1.5 667.9 0.2X
+Create a dataset of dates 115 117 3 8.7 115.4 1.0X
+to_json(date) 437 438 0 2.3 437.2 0.3X
+write dates to files 424 428 3 2.4 424.4 0.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------------------------------
-read timestamp text from files 151 154 2 6.6 151.5 1.0X
-read timestamps from files 1049 1051 3 1.0 1048.9 0.1X
-infer timestamps from files 1959 1967 10 0.5 1958.7 0.1X
-read date text from files 136 141 5 7.4 135.9 1.1X
-read date from files 703 713 10 1.4 702.7 0.2X
-timestamp strings 141 143 2 7.1 141.1 1.1X
-parse timestamps from Dataset[String] 1191 1195 7 0.8 1190.9 0.1X
-infer timestamps from Dataset[String] 2065 2071 9 0.5 2065.3 0.1X
-date strings 203 205 2 4.9 202.6 0.7X
-parse dates from Dataset[String] 967 970 4 1.0 967.0 0.2X
-from_json(timestamp) 1629 1632 2 0.6 1628.9 0.1X
-from_json(date) 1371 1396 41 0.7 1371.0 0.1X
-infer error timestamps from Dataset[String] with default format 1400 1404 4 0.7 1400.1 0.1X
-infer error timestamps from Dataset[String] with user-provided format 1401 1405 5 0.7 1400.8 0.1X
-infer error timestamps from Dataset[String] with legacy format 1429 1431 2 0.7 1429.0 0.1X
+read timestamp text from files 152 158 6 6.6 152.2 1.0X
+read timestamps from files 983 984 1 1.0 982.9 0.2X
+infer timestamps from files 1857 1866 8 0.5 1857.2 0.1X
+read date text from files 147 153 9 6.8 146.5 1.0X
+read date from files 671 674 4 1.5 671.0 0.2X
+timestamp strings 135 138 2 7.4 135.0 1.1X
+parse timestamps from Dataset[String] 1138 1140 3 0.9 1137.8 0.1X
+infer timestamps from Dataset[String] 1985 1992 6 0.5 1985.4 0.1X
+date strings 229 230 1 4.4 228.8 0.7X
+parse dates from Dataset[String] 916 917 1 1.1 915.7 0.2X
+from_json(timestamp) 1644 1648 7 0.6 1643.7 0.1X
+from_json(date) 1390 1400 9 0.7 1390.4 0.1X
+infer error timestamps from Dataset[String] with default format 1317 1322 9 0.8 1317.1 0.1X
+infer error timestamps from Dataset[String] with user-provided format 1316 1330 12 0.8 1316.4 0.1X
+infer error timestamps from Dataset[String] with legacy format 1341 1344 3 0.7 1341.2 0.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-w/o filters 6452 6469 19 0.0 64519.3 1.0X
-pushdown disabled 6245 6253 8 0.0 62445.5 1.0X
-w/ filters 556 560 4 0.2 5563.4 11.6X
+w/o filters 5704 5706 2 0.0 57043.0 1.0X
+pushdown disabled 5604 5622 24 0.0 56042.1 1.0X
+w/ filters 643 654 11 0.2 6430.2 8.9X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Partial JSON results: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-parse invalid JSON 2255 2376 205 0.0 225510.6 1.0X
+parse invalid JSON 2332 2422 150 0.0 233209.5 1.0X
diff --git a/sql/core/benchmarks/LargeRowBenchmark-jdk21-results.txt b/sql/core/benchmarks/LargeRowBenchmark-jdk21-results.txt
index cbd78a086ea36..a34a68164c754 100644
--- a/sql/core/benchmarks/LargeRowBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/LargeRowBenchmark-jdk21-results.txt
@@ -2,25 +2,25 @@
Large Row Benchmark
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
#rows: 100, #cols: 10, cell: 1.3 MB: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-built-in UPPER 5912 6241 466 0.0 59115266.3 1.0X
-udf UPPER 4116 4134 25 0.0 41162058.1 1.4X
+built-in UPPER 5851 6220 521 0.0 58513209.6 1.0X
+udf UPPER 5127 5162 50 0.0 51266519.2 1.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
#rows: 1, #cols: 1, cell: 300.0 MB: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-built-in UPPER 1339 1346 10 0.0 1339197183.0 1.0X
-udf UPPER 932 969 50 0.0 931637076.0 1.4X
+built-in UPPER 1315 1322 9 0.0 1315306640.0 1.0X
+udf UPPER 1001 1004 3 0.0 1001420387.0 1.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
#rows: 1, #cols: 200, cell: 1.0 MB: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-built-in UPPER 1116 1137 30 0.0 1115727291.0 1.0X
-udf UPPER 1187 1217 43 0.0 1186838839.0 0.9X
+built-in UPPER 1130 1145 21 0.0 1130109481.0 1.0X
+udf UPPER 1199 1248 70 0.0 1198653800.0 0.9X
diff --git a/sql/core/benchmarks/LargeRowBenchmark-results.txt b/sql/core/benchmarks/LargeRowBenchmark-results.txt
index 8c8eea8b92226..a8dc06172135f 100644
--- a/sql/core/benchmarks/LargeRowBenchmark-results.txt
+++ b/sql/core/benchmarks/LargeRowBenchmark-results.txt
@@ -2,25 +2,25 @@
Large Row Benchmark
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
#rows: 100, #cols: 10, cell: 1.3 MB: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-built-in UPPER 6896 6962 93 0.0 68960045.7 1.0X
-udf UPPER 4315 4367 74 0.0 43151937.6 1.6X
+built-in UPPER 6774 6793 27 0.0 67735578.9 1.0X
+udf UPPER 4768 4770 2 0.0 47680577.5 1.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
#rows: 1, #cols: 1, cell: 300.0 MB: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-built-in UPPER 1532 1534 4 0.0 1531617745.0 1.0X
-udf UPPER 996 999 3 0.0 995955863.0 1.5X
+built-in UPPER 1520 1530 14 0.0 1520168353.0 1.0X
+udf UPPER 1048 1049 2 0.0 1047675797.0 1.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
#rows: 1, #cols: 200, cell: 1.0 MB: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-built-in UPPER 1263 1308 63 0.0 1263021540.0 1.0X
-udf UPPER 1366 1438 102 0.0 1365656763.0 0.9X
+built-in UPPER 1250 1267 25 0.0 1249716088.0 1.0X
+udf UPPER 1231 1309 110 0.0 1231441056.0 1.0X
diff --git a/sql/core/benchmarks/MakeDateTimeBenchmark-jdk21-results.txt b/sql/core/benchmarks/MakeDateTimeBenchmark-jdk21-results.txt
index e3a39091e0d88..df4d262091ac8 100644
--- a/sql/core/benchmarks/MakeDateTimeBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/MakeDateTimeBenchmark-jdk21-results.txt
@@ -1,22 +1,22 @@
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
make_date(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-prepare make_date() 2308 2354 49 43.3 23.1 1.0X
-make_date(2019, 9, 16) 2016 2102 80 49.6 20.2 1.1X
-make_date(*, *, *) 2866 2886 17 34.9 28.7 0.8X
+prepare make_date() 2331 2417 144 42.9 23.3 1.0X
+make_date(2019, 9, 16) 1966 1985 27 50.9 19.7 1.2X
+make_date(*, *, *) 2979 2989 9 33.6 29.8 0.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
make_timestamp(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-prepare make_timestamp() 333 334 2 3.0 332.6 1.0X
-make_timestamp(2019, 1, 2, 3, 4, 50.123456) 38 45 7 26.3 38.1 8.7X
-make_timestamp(2019, 1, 2, 3, 4, 60.000000) 35 44 9 29.0 34.5 9.6X
-make_timestamp(2019, 12, 31, 23, 59, 60.00) 33 39 4 30.0 33.3 10.0X
-make_timestamp(*, *, *, 3, 4, 50.123456) 144 148 5 7.0 143.8 2.3X
-make_timestamp(*, *, *, *, *, 0) 106 115 8 9.4 106.0 3.1X
-make_timestamp(*, *, *, *, *, 60.0) 141 141 0 7.1 140.6 2.4X
-make_timestamp(2019, 1, 2, *, *, *) 450 454 7 2.2 449.7 0.7X
-make_timestamp(*, *, *, *, *, *) 461 465 3 2.2 461.4 0.7X
+prepare make_timestamp() 331 337 5 3.0 330.6 1.0X
+make_timestamp(2019, 1, 2, 3, 4, 50.123456) 41 42 1 24.6 40.6 8.1X
+make_timestamp(2019, 1, 2, 3, 4, 60.000000) 38 40 3 26.2 38.1 8.7X
+make_timestamp(2019, 12, 31, 23, 59, 60.00) 35 39 5 28.6 35.0 9.5X
+make_timestamp(*, *, *, 3, 4, 50.123456) 171 171 0 5.8 171.1 1.9X
+make_timestamp(*, *, *, *, *, 0) 111 113 3 9.0 110.5 3.0X
+make_timestamp(*, *, *, *, *, 60.0) 139 142 3 7.2 138.7 2.4X
+make_timestamp(2019, 1, 2, *, *, *) 454 455 1 2.2 453.5 0.7X
+make_timestamp(*, *, *, *, *, *) 494 495 1 2.0 494.0 0.7X
diff --git a/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt b/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt
index 6fabd6588ce73..016c678bf9ed6 100644
--- a/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt
+++ b/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt
@@ -1,22 +1,22 @@
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
make_date(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-prepare make_date() 2197 2236 43 45.5 22.0 1.0X
-make_date(2019, 9, 16) 1756 1858 94 56.9 17.6 1.3X
-make_date(*, *, *) 2969 2971 2 33.7 29.7 0.7X
+prepare make_date() 2215 2250 31 45.1 22.2 1.0X
+make_date(2019, 9, 16) 1908 1968 65 52.4 19.1 1.2X
+make_date(*, *, *) 2978 3021 47 33.6 29.8 0.7X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
make_timestamp(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-prepare make_timestamp() 357 397 57 2.8 356.6 1.0X
-make_timestamp(2019, 1, 2, 3, 4, 50.123456) 45 46 1 22.3 44.9 7.9X
-make_timestamp(2019, 1, 2, 3, 4, 60.000000) 36 39 5 27.6 36.2 9.9X
-make_timestamp(2019, 12, 31, 23, 59, 60.00) 41 42 2 24.6 40.7 8.8X
-make_timestamp(*, *, *, 3, 4, 50.123456) 162 164 2 6.2 161.6 2.2X
-make_timestamp(*, *, *, *, *, 0) 108 115 6 9.3 107.6 3.3X
-make_timestamp(*, *, *, *, *, 60.0) 137 141 4 7.3 136.8 2.6X
-make_timestamp(2019, 1, 2, *, *, *) 482 487 5 2.1 481.9 0.7X
-make_timestamp(*, *, *, *, *, *) 480 485 5 2.1 480.2 0.7X
+prepare make_timestamp() 343 356 15 2.9 343.4 1.0X
+make_timestamp(2019, 1, 2, 3, 4, 50.123456) 40 41 1 25.1 39.8 8.6X
+make_timestamp(2019, 1, 2, 3, 4, 60.000000) 36 39 5 28.1 35.5 9.7X
+make_timestamp(2019, 12, 31, 23, 59, 60.00) 35 38 3 28.7 34.9 9.8X
+make_timestamp(*, *, *, 3, 4, 50.123456) 168 170 3 6.0 168.1 2.0X
+make_timestamp(*, *, *, *, *, 0) 108 109 2 9.3 108.0 3.2X
+make_timestamp(*, *, *, *, *, 60.0) 136 138 4 7.4 135.5 2.5X
+make_timestamp(2019, 1, 2, *, *, *) 438 441 4 2.3 438.4 0.8X
+make_timestamp(*, *, *, *, *, *) 466 468 2 2.1 466.0 0.7X
diff --git a/sql/core/benchmarks/MetadataStructBenchmark-jdk21-results.txt b/sql/core/benchmarks/MetadataStructBenchmark-jdk21-results.txt
index 35bf5113ee895..e07f622fdd53a 100644
--- a/sql/core/benchmarks/MetadataStructBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/MetadataStructBenchmark-jdk21-results.txt
@@ -2,45 +2,45 @@
Metadata Struct Benchmark
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Vectorized Parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-no metadata columns 629 650 18 7.9 125.8 1.0X
-_metadata.file_path 729 744 12 6.9 145.9 0.9X
-_metadata.file_name 735 745 16 6.8 146.9 0.9X
-_metadata.file_size 657 669 7 7.6 131.3 1.0X
-_metadata.file_block_start 658 668 9 7.6 131.6 1.0X
-_metadata.file_block_length 657 666 8 7.6 131.3 1.0X
-_metadata.file_modification_time 627 655 11 8.0 125.5 1.0X
-_metadata.row_index 710 719 11 7.0 142.0 0.9X
-_metadata 954 976 11 5.2 190.9 0.7X
+no metadata columns 639 659 16 7.8 127.8 1.0X
+_metadata.file_path 735 749 11 6.8 147.1 0.9X
+_metadata.file_name 735 745 9 6.8 147.0 0.9X
+_metadata.file_size 662 673 9 7.6 132.3 1.0X
+_metadata.file_block_start 631 666 12 7.9 126.3 1.0X
+_metadata.file_block_length 661 671 6 7.6 132.3 1.0X
+_metadata.file_modification_time 662 670 6 7.6 132.3 1.0X
+_metadata.row_index 695 715 12 7.2 139.0 0.9X
+_metadata 954 978 11 5.2 190.8 0.7X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Parquet-mr: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-no metadata columns 2703 2746 29 1.8 540.7 1.0X
-_metadata.file_path 3467 3505 21 1.4 693.4 0.8X
-_metadata.file_name 3515 3538 15 1.4 703.0 0.8X
-_metadata.file_size 3292 3322 17 1.5 658.5 0.8X
-_metadata.file_block_start 3312 3345 19 1.5 662.3 0.8X
-_metadata.file_block_length 3310 3339 16 1.5 662.0 0.8X
-_metadata.file_modification_time 3305 3355 31 1.5 661.1 0.8X
-_metadata.row_index 3625 3655 18 1.4 725.1 0.7X
-_metadata 4831 4871 27 1.0 966.2 0.6X
+no metadata columns 2758 2792 27 1.8 551.7 1.0X
+_metadata.file_path 3525 3567 22 1.4 705.0 0.8X
+_metadata.file_name 3511 3541 16 1.4 702.2 0.8X
+_metadata.file_size 3358 3391 30 1.5 671.6 0.8X
+_metadata.file_block_start 3363 3387 19 1.5 672.5 0.8X
+_metadata.file_block_length 3359 3386 20 1.5 671.8 0.8X
+_metadata.file_modification_time 3355 3387 20 1.5 671.0 0.8X
+_metadata.row_index 3650 3681 32 1.4 730.0 0.8X
+_metadata 4791 4824 24 1.0 958.2 0.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
JSON: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-no metadata columns 6623 6680 78 0.8 1324.5 1.0X
-_metadata.file_path 7465 7493 16 0.7 1493.1 0.9X
-_metadata.file_name 7485 7510 23 0.7 1497.0 0.9X
-_metadata.file_size 7311 7335 13 0.7 1462.1 0.9X
-_metadata.file_block_start 7316 7331 9 0.7 1463.3 0.9X
-_metadata.file_block_length 7327 7343 13 0.7 1465.4 0.9X
-_metadata.file_modification_time 7314 7331 11 0.7 1462.7 0.9X
-_metadata 8155 8175 15 0.6 1631.0 0.8X
+no metadata columns 6897 6946 65 0.7 1379.4 1.0X
+_metadata.file_path 7726 7756 14 0.6 1545.3 0.9X
+_metadata.file_name 7734 7793 38 0.6 1546.8 0.9X
+_metadata.file_size 7549 7581 20 0.7 1509.8 0.9X
+_metadata.file_block_start 7546 7602 33 0.7 1509.2 0.9X
+_metadata.file_block_length 7574 7610 24 0.7 1514.8 0.9X
+_metadata.file_modification_time 7561 7600 21 0.7 1512.1 0.9X
+_metadata 8303 8337 18 0.6 1660.5 0.8X
diff --git a/sql/core/benchmarks/MetadataStructBenchmark-results.txt b/sql/core/benchmarks/MetadataStructBenchmark-results.txt
index bc35900fc09a2..f3ea90a400b5c 100644
--- a/sql/core/benchmarks/MetadataStructBenchmark-results.txt
+++ b/sql/core/benchmarks/MetadataStructBenchmark-results.txt
@@ -2,45 +2,45 @@
Metadata Struct Benchmark
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Vectorized Parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-no metadata columns 634 661 21 7.9 126.9 1.0X
-_metadata.file_path 737 753 9 6.8 147.5 0.9X
-_metadata.file_name 723 741 9 6.9 144.6 0.9X
-_metadata.file_size 673 682 4 7.4 134.7 0.9X
-_metadata.file_block_start 676 680 4 7.4 135.1 0.9X
-_metadata.file_block_length 657 677 6 7.6 131.4 1.0X
-_metadata.file_modification_time 648 670 10 7.7 129.6 1.0X
-_metadata.row_index 718 727 7 7.0 143.7 0.9X
-_metadata 981 1015 15 5.1 196.3 0.6X
+no metadata columns 654 680 29 7.6 130.9 1.0X
+_metadata.file_path 750 765 16 6.7 150.0 0.9X
+_metadata.file_name 747 756 6 6.7 149.4 0.9X
+_metadata.file_size 679 689 6 7.4 135.8 1.0X
+_metadata.file_block_start 682 688 4 7.3 136.4 1.0X
+_metadata.file_block_length 682 691 8 7.3 136.5 1.0X
+_metadata.file_modification_time 685 690 4 7.3 136.9 1.0X
+_metadata.row_index 731 741 12 6.8 146.2 0.9X
+_metadata 1026 1035 8 4.9 205.1 0.6X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Parquet-mr: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-no metadata columns 2005 2050 21 2.5 401.0 1.0X
-_metadata.file_path 2828 2874 18 1.8 565.7 0.7X
-_metadata.file_name 2745 2812 25 1.8 549.0 0.7X
-_metadata.file_size 2527 2567 20 2.0 505.4 0.8X
-_metadata.file_block_start 2515 2546 16 2.0 503.0 0.8X
-_metadata.file_block_length 2531 2588 21 2.0 506.1 0.8X
-_metadata.file_modification_time 2595 2634 34 1.9 519.1 0.8X
-_metadata.row_index 3427 3478 24 1.5 685.4 0.6X
-_metadata 4669 4713 29 1.1 933.9 0.4X
+no metadata columns 2718 2741 18 1.8 543.7 1.0X
+_metadata.file_path 3470 3486 14 1.4 694.0 0.8X
+_metadata.file_name 3437 3458 22 1.5 687.3 0.8X
+_metadata.file_size 3320 3333 10 1.5 664.0 0.8X
+_metadata.file_block_start 3316 3332 14 1.5 663.1 0.8X
+_metadata.file_block_length 3313 3336 22 1.5 662.6 0.8X
+_metadata.file_modification_time 3314 3328 12 1.5 662.8 0.8X
+_metadata.row_index 3666 3687 25 1.4 733.1 0.7X
+_metadata 4883 4904 9 1.0 976.6 0.6X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
JSON: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-no metadata columns 6665 6893 88 0.8 1333.1 1.0X
-_metadata.file_path 7367 7424 39 0.7 1473.4 0.9X
-_metadata.file_name 7384 7445 39 0.7 1476.8 0.9X
-_metadata.file_size 7193 7227 20 0.7 1438.6 0.9X
-_metadata.file_block_start 7207 7250 27 0.7 1441.4 0.9X
-_metadata.file_block_length 7183 7345 154 0.7 1436.6 0.9X
-_metadata.file_modification_time 7236 7281 34 0.7 1447.2 0.9X
-_metadata 8172 8239 46 0.6 1634.5 0.8X
+no metadata columns 7035 7057 16 0.7 1407.0 1.0X
+_metadata.file_path 7916 7950 42 0.6 1583.2 0.9X
+_metadata.file_name 7926 7948 15 0.6 1585.3 0.9X
+_metadata.file_size 7671 7705 17 0.7 1534.2 0.9X
+_metadata.file_block_start 7697 7721 15 0.6 1539.4 0.9X
+_metadata.file_block_length 7706 7729 13 0.6 1541.2 0.9X
+_metadata.file_modification_time 7709 7741 22 0.6 1541.9 0.9X
+_metadata 8703 8727 14 0.6 1740.5 0.8X
diff --git a/sql/core/benchmarks/MetricsAggregationBenchmark-jdk21-results.txt b/sql/core/benchmarks/MetricsAggregationBenchmark-jdk21-results.txt
index 4baacbec53763..a02bedff5b676 100644
--- a/sql/core/benchmarks/MetricsAggregationBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/MetricsAggregationBenchmark-jdk21-results.txt
@@ -1,12 +1,12 @@
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
metrics aggregation (50 metrics, 100000 tasks per stage): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------------------
-1 stage(s) 471 621 140 0.0 470645682.0 1.0X
-2 stage(s) 990 1152 228 0.0 990437475.0 0.5X
-3 stage(s) 1685 1804 169 0.0 1684707956.0 0.3X
+1 stage(s) 510 673 113 0.0 510483174.0 1.0X
+2 stage(s) 1590 1640 71 0.0 1590439310.0 0.3X
+3 stage(s) 2512 2582 100 0.0 2511748396.0 0.2X
Stage Count Stage Proc. Time Aggreg. Time
- 1 376 58
- 2 288 131
- 3 352 157
+ 1 499 58
+ 2 563 167
+ 3 552 258
diff --git a/sql/core/benchmarks/MetricsAggregationBenchmark-results.txt b/sql/core/benchmarks/MetricsAggregationBenchmark-results.txt
index 5324f515b813c..f0df3ee4fe20d 100644
--- a/sql/core/benchmarks/MetricsAggregationBenchmark-results.txt
+++ b/sql/core/benchmarks/MetricsAggregationBenchmark-results.txt
@@ -1,12 +1,12 @@
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
metrics aggregation (50 metrics, 100000 tasks per stage): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
----------------------------------------------------------------------------------------------------------------------------------------
-1 stage(s) 635 682 45 0.0 635457575.0 1.0X
-2 stage(s) 1395 1406 16 0.0 1394972033.0 0.5X
-3 stage(s) 1935 2037 145 0.0 1934636284.0 0.3X
+1 stage(s) 383 418 71 0.0 383366848.0 1.0X
+2 stage(s) 1367 1383 22 0.0 1367425765.0 0.3X
+3 stage(s) 1732 1910 253 0.0 1731683484.0 0.2X
Stage Count Stage Proc. Time Aggreg. Time
- 1 402 36
- 2 367 179
- 3 390 174
+ 1 252 51
+ 2 400 148
+ 3 371 246
diff --git a/sql/core/benchmarks/MiscBenchmark-jdk21-results.txt b/sql/core/benchmarks/MiscBenchmark-jdk21-results.txt
index ff2335fc3d6e1..422b5fd109610 100644
--- a/sql/core/benchmarks/MiscBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/MiscBenchmark-jdk21-results.txt
@@ -2,126 +2,126 @@
filter & aggregate without group
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
range/filter/sum: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-range/filter/sum wholestage off 35430 36341 1288 59.2 16.9 1.0X
-range/filter/sum wholestage on 3511 3649 78 597.2 1.7 10.1X
+range/filter/sum wholestage off 36339 37196 1213 57.7 17.3 1.0X
+range/filter/sum wholestage on 3502 3770 300 598.8 1.7 10.4X
================================================================================================
range/limit/sum
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
range/limit/sum: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-range/limit/sum wholestage off 67 68 2 7864.7 0.1 1.0X
-range/limit/sum wholestage on 81 95 12 6502.2 0.2 0.8X
+range/limit/sum wholestage off 74 75 2 7125.3 0.1 1.0X
+range/limit/sum wholestage on 75 84 7 6972.8 0.1 1.0X
================================================================================================
sample
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
sample with replacement: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-sample with replacement wholestage off 7948 8022 104 16.5 60.6 1.0X
-sample with replacement wholestage on 5104 5183 45 25.7 38.9 1.6X
+sample with replacement wholestage off 8091 8152 87 16.2 61.7 1.0X
+sample with replacement wholestage on 5134 5156 21 25.5 39.2 1.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
sample without replacement: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
-sample without replacement wholestage off 1832 1842 14 71.6 14.0 1.0X
-sample without replacement wholestage on 661 671 8 198.4 5.0 2.8X
+sample without replacement wholestage off 1895 1896 2 69.2 14.5 1.0X
+sample without replacement wholestage on 657 677 13 199.6 5.0 2.9X
================================================================================================
collect
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collect: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-collect 1 million 154 214 58 6.8 147.0 1.0X
-collect 2 millions 329 422 86 3.2 313.6 0.5X
-collect 4 millions 741 821 83 1.4 706.9 0.2X
+collect 1 million 169 218 118 6.2 161.3 1.0X
+collect 2 millions 309 440 84 3.4 294.7 0.5X
+collect 4 millions 635 747 116 1.7 605.7 0.3X
================================================================================================
collect limit
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collect limit: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-collect limit 1 million 151 222 69 7.0 143.7 1.0X
-collect limit 2 millions 314 421 131 3.3 299.0 0.5X
+collect limit 1 million 149 230 95 7.1 141.8 1.0X
+collect limit 2 millions 288 409 70 3.6 274.5 0.5X
================================================================================================
generate explode
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
generate explode array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-generate explode array wholestage off 11858 11982 176 1.4 706.8 1.0X
-generate explode array wholestage on 3019 3063 86 5.6 179.9 3.9X
+generate explode array wholestage off 12491 12542 71 1.3 744.5 1.0X
+generate explode array wholestage on 2970 3145 175 5.6 177.0 4.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
generate explode map: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-generate explode map wholestage off 24900 24951 72 0.7 1484.1 1.0X
-generate explode map wholestage on 9324 9385 41 1.8 555.8 2.7X
+generate explode map wholestage off 24462 24644 257 0.7 1458.0 1.0X
+generate explode map wholestage on 9106 9313 230 1.8 542.7 2.7X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
generate posexplode array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-generate posexplode array wholestage off 12596 12671 105 1.3 750.8 1.0X
-generate posexplode array wholestage on 2984 3094 71 5.6 177.8 4.2X
+generate posexplode array wholestage off 12603 12606 4 1.3 751.2 1.0X
+generate posexplode array wholestage on 3011 3097 49 5.6 179.5 4.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
generate inline array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-generate inline array wholestage off 7072 7114 59 2.4 421.5 1.0X
-generate inline array wholestage on 2399 2468 74 7.0 143.0 2.9X
+generate inline array wholestage off 6943 6983 57 2.4 413.8 1.0X
+generate inline array wholestage on 2498 2566 75 6.7 148.9 2.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
generate big struct array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-generate big struct array wholestage off 208 209 1 0.3 3463.4 1.0X
-generate big struct array wholestage on 156 169 14 0.4 2593.2 1.3X
+generate big struct array wholestage off 192 198 8 0.3 3201.2 1.0X
+generate big struct array wholestage on 152 161 8 0.4 2526.6 1.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
generate big nested struct array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------
-generate big nested struct array wholestage off 18211 18237 36 0.0 303524.5 1.0X
-generate big nested struct array wholestage on 154 172 16 0.4 2559.5 118.6X
+generate big nested struct array wholestage off 18968 20994 2866 0.0 316136.8 1.0X
+generate big nested struct array wholestage on 153 161 9 0.4 2544.7 124.2X
================================================================================================
generate regular generator
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
generate stack: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-generate stack wholestage off 13727 13943 305 1.2 818.2 1.0X
-generate stack wholestage on 3152 3206 36 5.3 187.9 4.4X
+generate stack wholestage off 13409 13455 66 1.3 799.2 1.0X
+generate stack wholestage on 3050 3056 4 5.5 181.8 4.4X
diff --git a/sql/core/benchmarks/MiscBenchmark-results.txt b/sql/core/benchmarks/MiscBenchmark-results.txt
index f2d3eb1988b62..24b0ce572ddae 100644
--- a/sql/core/benchmarks/MiscBenchmark-results.txt
+++ b/sql/core/benchmarks/MiscBenchmark-results.txt
@@ -2,126 +2,126 @@
filter & aggregate without group
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
range/filter/sum: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-range/filter/sum wholestage off 36214 36785 807 57.9 17.3 1.0X
-range/filter/sum wholestage on 3468 3752 160 604.7 1.7 10.4X
+range/filter/sum wholestage off 36126 37069 1333 58.1 17.2 1.0X
+range/filter/sum wholestage on 3453 3747 166 607.3 1.6 10.5X
================================================================================================
range/limit/sum
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
range/limit/sum: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-range/limit/sum wholestage off 76 76 0 6926.9 0.1 1.0X
-range/limit/sum wholestage on 78 81 3 6685.5 0.1 1.0X
+range/limit/sum wholestage off 73 76 5 7206.6 0.1 1.0X
+range/limit/sum wholestage on 87 95 6 6028.2 0.2 0.8X
================================================================================================
sample
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
sample with replacement: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-sample with replacement wholestage off 8042 8116 106 16.3 61.4 1.0X
-sample with replacement wholestage on 5265 5351 52 24.9 40.2 1.5X
+sample with replacement wholestage off 7717 7759 60 17.0 58.9 1.0X
+sample with replacement wholestage on 5018 5026 7 26.1 38.3 1.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
sample without replacement: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
-sample without replacement wholestage off 1942 1946 6 67.5 14.8 1.0X
-sample without replacement wholestage on 713 740 29 183.8 5.4 2.7X
+sample without replacement wholestage off 1808 1814 8 72.5 13.8 1.0X
+sample without replacement wholestage on 701 743 77 186.9 5.3 2.6X
================================================================================================
collect
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collect: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-collect 1 million 158 216 72 6.7 150.2 1.0X
-collect 2 millions 298 407 91 3.5 284.3 0.5X
-collect 4 millions 722 827 132 1.5 688.9 0.2X
+collect 1 million 160 213 67 6.6 152.2 1.0X
+collect 2 millions 301 430 73 3.5 286.9 0.5X
+collect 4 millions 716 840 134 1.5 682.5 0.2X
================================================================================================
collect limit
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
collect limit: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-collect limit 1 million 152 235 119 6.9 145.4 1.0X
-collect limit 2 millions 317 433 117 3.3 302.2 0.5X
+collect limit 1 million 162 200 66 6.5 154.4 1.0X
+collect limit 2 millions 315 407 75 3.3 300.2 0.5X
================================================================================================
generate explode
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
generate explode array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-generate explode array wholestage off 11844 11877 48 1.4 705.9 1.0X
-generate explode array wholestage on 2946 2980 59 5.7 175.6 4.0X
+generate explode array wholestage off 11515 11594 112 1.5 686.3 1.0X
+generate explode array wholestage on 2936 3021 82 5.7 175.0 3.9X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
generate explode map: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-generate explode map wholestage off 23185 23254 98 0.7 1381.9 1.0X
-generate explode map wholestage on 9594 9844 196 1.7 571.8 2.4X
+generate explode map wholestage off 23775 23804 41 0.7 1417.1 1.0X
+generate explode map wholestage on 9691 9798 93 1.7 577.6 2.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
generate posexplode array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-generate posexplode array wholestage off 12281 12328 67 1.4 732.0 1.0X
-generate posexplode array wholestage on 2914 3004 76 5.8 173.7 4.2X
+generate posexplode array wholestage off 12212 12393 257 1.4 727.9 1.0X
+generate posexplode array wholestage on 2971 3034 40 5.6 177.1 4.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
generate inline array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-generate inline array wholestage off 6803 6881 112 2.5 405.5 1.0X
-generate inline array wholestage on 2506 2515 11 6.7 149.4 2.7X
+generate inline array wholestage off 6705 6754 69 2.5 399.7 1.0X
+generate inline array wholestage on 2490 2525 40 6.7 148.4 2.7X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
generate big struct array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-generate big struct array wholestage off 228 241 18 0.3 3799.8 1.0X
-generate big struct array wholestage on 175 181 5 0.3 2909.8 1.3X
+generate big struct array wholestage off 200 212 16 0.3 3339.2 1.0X
+generate big struct array wholestage on 180 188 14 0.3 2996.2 1.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
generate big nested struct array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------
-generate big nested struct array wholestage off 17588 18273 968 0.0 293139.3 1.0X
-generate big nested struct array wholestage on 169 179 8 0.4 2812.8 104.2X
+generate big nested struct array wholestage off 18810 19086 390 0.0 313505.9 1.0X
+generate big nested struct array wholestage on 173 186 13 0.3 2886.8 108.6X
================================================================================================
generate regular generator
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
generate stack: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-generate stack wholestage off 13013 13020 10 1.3 775.7 1.0X
-generate stack wholestage on 3085 3093 7 5.4 183.9 4.2X
+generate stack wholestage off 12822 12842 28 1.3 764.3 1.0X
+generate stack wholestage on 3056 3073 13 5.5 182.2 4.2X
diff --git a/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-jdk21-results.txt b/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-jdk21-results.txt
index 4fcfd547e40dd..f87f452a1b484 100644
--- a/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-jdk21-results.txt
@@ -2,52 +2,52 @@
Nested Schema Pruning Benchmark For ORC v1
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Selection: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 59 71 8 16.9 59.3 1.0X
-Nested column 58 66 7 17.3 57.7 1.0X
-Nested column in array 161 166 3 6.2 160.8 0.4X
+Top-level column 67 93 14 14.8 67.4 1.0X
+Nested column 61 72 8 16.4 60.9 1.1X
+Nested column in array 168 171 3 6.0 167.8 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Limiting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 227 240 12 4.4 227.0 1.0X
-Nested column 230 253 17 4.3 230.1 1.0X
-Nested column in array 500 522 14 2.0 500.1 0.5X
+Top-level column 239 255 16 4.2 239.2 1.0X
+Nested column 245 267 16 4.1 245.1 1.0X
+Nested column in array 539 562 20 1.9 539.0 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Repartitioning: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 206 218 9 4.9 206.1 1.0X
-Nested column 210 221 12 4.8 210.0 1.0X
-Nested column in array 448 469 12 2.2 447.7 0.5X
+Top-level column 215 229 11 4.7 214.7 1.0X
+Nested column 225 239 11 4.4 224.7 1.0X
+Nested column in array 481 488 5 2.1 480.8 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Repartitioning by exprs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 201 214 7 5.0 200.7 1.0X
-Nested column 224 237 8 4.5 223.8 0.9X
-Nested column in array 501 514 7 2.0 500.8 0.4X
+Top-level column 212 226 10 4.7 212.2 1.0X
+Nested column 234 248 8 4.3 234.2 0.9X
+Nested column in array 512 521 7 2.0 512.4 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Sample: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 67 79 10 15.0 66.8 1.0X
-Nested column 75 91 17 13.3 75.0 0.9X
-Nested column in array 214 232 11 4.7 213.6 0.3X
+Top-level column 67 82 13 14.9 66.9 1.0X
+Nested column 75 91 13 13.4 74.8 0.9X
+Nested column in array 215 240 24 4.6 215.2 0.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Sorting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 293 306 13 3.4 293.5 1.0X
-Nested column 340 349 10 2.9 340.4 0.9X
-Nested column in array 712 778 41 1.4 712.3 0.4X
+Top-level column 297 314 16 3.4 297.1 1.0X
+Nested column 341 351 14 2.9 341.5 0.9X
+Nested column in array 714 729 19 1.4 714.4 0.4X
diff --git a/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-results.txt b/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-results.txt
index a50f57fd5abbe..74179e5b6c504 100644
--- a/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-results.txt
+++ b/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-results.txt
@@ -2,52 +2,52 @@
Nested Schema Pruning Benchmark For ORC v1
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Selection: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 62 79 11 16.1 62.0 1.0X
-Nested column 58 66 7 17.2 58.2 1.1X
-Nested column in array 160 166 5 6.2 160.4 0.4X
+Top-level column 63 78 11 16.0 62.7 1.0X
+Nested column 58 67 7 17.2 58.3 1.1X
+Nested column in array 168 177 9 5.9 168.5 0.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Limiting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 226 246 12 4.4 226.0 1.0X
-Nested column 230 247 11 4.4 229.8 1.0X
-Nested column in array 488 507 22 2.0 488.0 0.5X
+Top-level column 247 263 22 4.0 247.4 1.0X
+Nested column 236 252 9 4.2 235.9 1.0X
+Nested column in array 490 512 14 2.0 489.8 0.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Repartitioning: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 205 209 5 4.9 204.5 1.0X
-Nested column 209 216 6 4.8 208.7 1.0X
-Nested column in array 447 454 5 2.2 447.3 0.5X
+Top-level column 207 214 5 4.8 207.5 1.0X
+Nested column 214 221 6 4.7 213.6 1.0X
+Nested column in array 454 460 4 2.2 453.7 0.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Repartitioning by exprs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 202 207 5 5.0 201.8 1.0X
-Nested column 225 233 4 4.4 225.1 0.9X
-Nested column in array 481 489 6 2.1 481.2 0.4X
+Top-level column 209 212 2 4.8 208.7 1.0X
+Nested column 231 238 7 4.3 230.6 0.9X
+Nested column in array 488 495 5 2.1 487.5 0.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Sample: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 67 80 12 15.0 66.9 1.0X
-Nested column 74 89 14 13.5 74.3 0.9X
-Nested column in array 193 252 48 5.2 193.5 0.3X
+Top-level column 70 77 6 14.2 70.3 1.0X
+Nested column 76 91 14 13.2 75.9 0.9X
+Nested column in array 208 243 26 4.8 208.3 0.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Sorting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 285 297 8 3.5 285.1 1.0X
-Nested column 342 349 5 2.9 341.9 0.8X
-Nested column in array 714 722 8 1.4 713.5 0.4X
+Top-level column 297 306 7 3.4 297.0 1.0X
+Nested column 351 359 6 2.9 350.9 0.8X
+Nested column in array 696 711 17 1.4 695.7 0.4X
diff --git a/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-jdk21-results.txt b/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-jdk21-results.txt
index 9478a15923122..3805177dde47b 100644
--- a/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-jdk21-results.txt
@@ -2,52 +2,52 @@
Nested Schema Pruning Benchmark For ORC v2
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Selection: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 59 75 10 17.0 58.8 1.0X
-Nested column 60 68 7 16.8 59.5 1.0X
-Nested column in array 160 164 4 6.3 160.0 0.4X
+Top-level column 59 73 12 16.9 59.1 1.0X
+Nested column 58 67 6 17.2 58.0 1.0X
+Nested column in array 167 172 5 6.0 167.1 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Limiting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 243 258 13 4.1 242.5 1.0X
-Nested column 234 256 15 4.3 234.4 1.0X
-Nested column in array 525 543 12 1.9 525.2 0.5X
+Top-level column 243 260 22 4.1 243.4 1.0X
+Nested column 244 263 15 4.1 243.6 1.0X
+Nested column in array 508 538 17 2.0 508.1 0.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Repartitioning: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 209 221 7 4.8 209.3 1.0X
-Nested column 217 229 8 4.6 217.2 1.0X
-Nested column in array 458 474 7 2.2 457.7 0.5X
+Top-level column 213 227 8 4.7 213.2 1.0X
+Nested column 221 233 7 4.5 221.3 1.0X
+Nested column in array 464 481 14 2.2 464.2 0.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Repartitioning by exprs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 205 217 8 4.9 205.4 1.0X
-Nested column 228 239 7 4.4 227.7 0.9X
-Nested column in array 498 509 6 2.0 498.3 0.4X
+Top-level column 212 224 8 4.7 212.1 1.0X
+Nested column 243 254 7 4.1 243.3 0.9X
+Nested column in array 514 522 6 1.9 514.4 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Sample: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 73 85 12 13.6 73.4 1.0X
-Nested column 76 92 12 13.1 76.3 1.0X
-Nested column in array 204 233 32 4.9 203.7 0.4X
+Top-level column 73 83 11 13.7 73.0 1.0X
+Nested column 76 89 11 13.1 76.3 1.0X
+Nested column in array 214 254 33 4.7 214.5 0.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Sorting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 296 307 18 3.4 295.9 1.0X
-Nested column 337 348 18 3.0 337.1 0.9X
-Nested column in array 683 711 34 1.5 682.6 0.4X
+Top-level column 307 326 25 3.3 306.9 1.0X
+Nested column 352 361 13 2.8 351.7 0.9X
+Nested column in array 708 728 15 1.4 708.3 0.4X
diff --git a/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-results.txt b/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-results.txt
index 9c608cdc104db..75c922e834670 100644
--- a/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-results.txt
+++ b/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-results.txt
@@ -2,52 +2,52 @@
Nested Schema Pruning Benchmark For ORC v2
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Selection: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 64 76 11 15.6 64.1 1.0X
-Nested column 57 64 4 17.7 56.6 1.1X
-Nested column in array 160 165 5 6.3 159.6 0.4X
+Top-level column 69 84 11 14.5 68.9 1.0X
+Nested column 59 67 5 16.8 59.5 1.2X
+Nested column in array 167 171 4 6.0 166.6 0.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Limiting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 231 248 15 4.3 230.8 1.0X
-Nested column 234 251 20 4.3 233.5 1.0X
-Nested column in array 498 523 16 2.0 498.0 0.5X
+Top-level column 233 259 20 4.3 233.1 1.0X
+Nested column 235 248 9 4.2 235.5 1.0X
+Nested column in array 493 511 12 2.0 493.0 0.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Repartitioning: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 203 211 6 4.9 202.5 1.0X
-Nested column 209 217 4 4.8 209.2 1.0X
-Nested column in array 447 455 6 2.2 446.9 0.5X
+Top-level column 207 214 6 4.8 206.7 1.0X
+Nested column 210 218 7 4.8 210.5 1.0X
+Nested column in array 445 452 4 2.2 445.1 0.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Repartitioning by exprs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 204 206 3 4.9 203.6 1.0X
-Nested column 225 230 5 4.4 225.2 0.9X
-Nested column in array 482 489 6 2.1 481.8 0.4X
+Top-level column 203 207 3 4.9 203.4 1.0X
+Nested column 229 236 6 4.4 228.5 0.9X
+Nested column in array 479 487 6 2.1 479.2 0.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Sample: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 60 81 10 16.7 60.0 1.0X
-Nested column 75 89 19 13.3 75.0 0.8X
-Nested column in array 197 236 21 5.1 197.4 0.3X
+Top-level column 69 79 7 14.4 69.3 1.0X
+Nested column 76 89 10 13.2 75.7 0.9X
+Nested column in array 199 218 16 5.0 199.0 0.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Sorting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 295 301 4 3.4 295.1 1.0X
-Nested column 340 348 7 2.9 339.7 0.9X
-Nested column in array 696 713 15 1.4 696.2 0.4X
+Top-level column 290 300 7 3.5 289.8 1.0X
+Nested column 347 352 5 2.9 346.9 0.8X
+Nested column in array 691 704 13 1.4 691.5 0.4X
diff --git a/sql/core/benchmarks/ParquetNestedPredicatePushDownBenchmark-jdk21-results.txt b/sql/core/benchmarks/ParquetNestedPredicatePushDownBenchmark-jdk21-results.txt
index 36df6288ae20c..d577a05c07562 100644
--- a/sql/core/benchmarks/ParquetNestedPredicatePushDownBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/ParquetNestedPredicatePushDownBenchmark-jdk21-results.txt
@@ -1,21 +1,21 @@
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Can skip all row groups: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Without nested predicate Pushdown 6432 6461 39 16.3 61.3 1.0X
-With nested predicate Pushdown 50 64 9 2098.5 0.5 128.7X
+Without nested predicate Pushdown 6444 6497 51 16.3 61.5 1.0X
+With nested predicate Pushdown 63 77 12 1661.1 0.6 102.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Can skip some row groups: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Without nested predicate Pushdown 6867 7113 88 15.3 65.5 1.0X
-With nested predicate Pushdown 50 66 12 2108.2 0.5 138.1X
+Without nested predicate Pushdown 6923 6941 15 15.1 66.0 1.0X
+With nested predicate Pushdown 56 71 12 1889.2 0.5 124.7X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Can skip no row groups: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Without nested predicate Pushdown 13364 13431 62 7.8 127.4 1.0X
-With nested predicate Pushdown 13367 13482 91 7.8 127.5 1.0X
+Without nested predicate Pushdown 13365 13428 32 7.8 127.5 1.0X
+With nested predicate Pushdown 13404 13469 40 7.8 127.8 1.0X
diff --git a/sql/core/benchmarks/ParquetNestedPredicatePushDownBenchmark-results.txt b/sql/core/benchmarks/ParquetNestedPredicatePushDownBenchmark-results.txt
index 3744a6c32b9e5..638672ca7b4c9 100644
--- a/sql/core/benchmarks/ParquetNestedPredicatePushDownBenchmark-results.txt
+++ b/sql/core/benchmarks/ParquetNestedPredicatePushDownBenchmark-results.txt
@@ -1,21 +1,21 @@
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Can skip all row groups: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Without nested predicate Pushdown 7205 7285 64 14.6 68.7 1.0X
-With nested predicate Pushdown 73 87 13 1429.5 0.7 98.2X
+Without nested predicate Pushdown 7237 7306 72 14.5 69.0 1.0X
+With nested predicate Pushdown 55 72 13 1904.2 0.5 131.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Can skip some row groups: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Without nested predicate Pushdown 7554 7851 109 13.9 72.0 1.0X
-With nested predicate Pushdown 54 74 7 1954.7 0.5 140.8X
+Without nested predicate Pushdown 7775 7852 63 13.5 74.1 1.0X
+With nested predicate Pushdown 63 74 6 1665.2 0.6 123.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Can skip no row groups: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Without nested predicate Pushdown 14165 14208 28 7.4 135.1 1.0X
-With nested predicate Pushdown 14195 14250 57 7.4 135.4 1.0X
+Without nested predicate Pushdown 14210 14261 37 7.4 135.5 1.0X
+With nested predicate Pushdown 14216 14254 33 7.4 135.6 1.0X
diff --git a/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-jdk21-results.txt b/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-jdk21-results.txt
index 03124ec591c2c..9351880547265 100644
--- a/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-jdk21-results.txt
@@ -2,52 +2,52 @@
Nested Schema Pruning Benchmark For Parquet
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Selection: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 63 77 12 15.8 63.4 1.0X
-Nested column 64 71 6 15.6 64.0 1.0X
-Nested column in array 215 220 5 4.6 215.2 0.3X
+Top-level column 65 78 16 15.3 65.4 1.0X
+Nested column 67 76 10 14.9 67.3 1.0X
+Nested column in array 229 237 4 4.4 229.1 0.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Limiting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 244 269 26 4.1 243.6 1.0X
-Nested column 247 267 15 4.0 247.1 1.0X
-Nested column in array 573 596 18 1.7 573.5 0.4X
+Top-level column 243 263 11 4.1 243.3 1.0X
+Nested column 252 273 13 4.0 251.6 1.0X
+Nested column in array 562 593 24 1.8 561.9 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Repartitioning: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 220 232 10 4.5 220.5 1.0X
-Nested column 231 243 10 4.3 230.7 1.0X
-Nested column in array 507 530 16 2.0 507.3 0.4X
+Top-level column 234 241 6 4.3 234.1 1.0X
+Nested column 239 249 6 4.2 239.5 1.0X
+Nested column in array 521 535 14 1.9 520.8 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Repartitioning by exprs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 216 234 25 4.6 216.4 1.0X
-Nested column 246 259 9 4.1 246.4 0.9X
-Nested column in array 558 563 5 1.8 558.0 0.4X
+Top-level column 222 229 12 4.5 221.6 1.0X
+Nested column 247 267 27 4.1 246.5 0.9X
+Nested column in array 558 571 10 1.8 558.4 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Sample: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 73 87 10 13.7 72.8 1.0X
-Nested column 81 91 7 12.4 80.7 0.9X
-Nested column in array 256 291 21 3.9 255.9 0.3X
+Top-level column 75 90 15 13.3 75.1 1.0X
+Nested column 81 98 19 12.3 81.1 0.9X
+Nested column in array 281 309 18 3.6 280.6 0.3X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Sorting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 300 316 25 3.3 299.6 1.0X
-Nested column 343 354 10 2.9 342.5 0.9X
-Nested column in array 755 778 25 1.3 754.7 0.4X
+Top-level column 308 324 17 3.2 308.4 1.0X
+Nested column 360 371 16 2.8 359.5 0.9X
+Nested column in array 756 773 17 1.3 755.6 0.4X
diff --git a/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-results.txt b/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-results.txt
index 5f9a95e6fa8cb..8729ffc3428f9 100644
--- a/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-results.txt
+++ b/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-results.txt
@@ -2,52 +2,52 @@
Nested Schema Pruning Benchmark For Parquet
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Selection: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 67 88 13 14.9 67.1 1.0X
-Nested column 67 75 6 14.9 67.3 1.0X
-Nested column in array 224 231 3 4.5 224.4 0.3X
+Top-level column 68 81 11 14.7 68.2 1.0X
+Nested column 69 80 9 14.6 68.6 1.0X
+Nested column in array 229 235 5 4.4 228.6 0.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Limiting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 245 268 30 4.1 244.9 1.0X
-Nested column 244 258 9 4.1 244.0 1.0X
-Nested column in array 551 582 14 1.8 551.4 0.4X
+Top-level column 255 272 12 3.9 254.8 1.0X
+Nested column 249 262 8 4.0 248.8 1.0X
+Nested column in array 565 587 14 1.8 564.8 0.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Repartitioning: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 215 220 4 4.7 214.6 1.0X
-Nested column 218 226 6 4.6 217.8 1.0X
-Nested column in array 511 516 5 2.0 511.2 0.4X
+Top-level column 223 227 4 4.5 222.8 1.0X
+Nested column 224 232 6 4.5 224.4 1.0X
+Nested column in array 520 524 4 1.9 519.6 0.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Repartitioning by exprs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 211 218 6 4.7 211.0 1.0X
-Nested column 237 244 9 4.2 237.1 0.9X
-Nested column in array 545 553 4 1.8 545.3 0.4X
+Top-level column 219 225 5 4.6 218.8 1.0X
+Nested column 240 248 6 4.2 240.4 0.9X
+Nested column in array 547 556 7 1.8 547.3 0.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Sample: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 77 87 8 12.9 77.5 1.0X
-Nested column 86 96 10 11.6 85.9 0.9X
-Nested column in array 260 283 13 3.8 259.9 0.3X
+Top-level column 78 92 13 12.9 77.6 1.0X
+Nested column 86 96 10 11.6 86.0 0.9X
+Nested column in array 267 297 19 3.7 267.0 0.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Sorting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Top-level column 302 311 9 3.3 301.6 1.0X
-Nested column 348 354 5 2.9 348.1 0.9X
-Nested column in array 754 775 21 1.3 754.5 0.4X
+Top-level column 305 312 3 3.3 305.0 1.0X
+Nested column 357 369 19 2.8 357.0 0.9X
+Nested column in array 765 775 13 1.3 764.6 0.4X
diff --git a/sql/core/benchmarks/PrimitiveArrayBenchmark-jdk21-results.txt b/sql/core/benchmarks/PrimitiveArrayBenchmark-jdk21-results.txt
index c4e4c1e48a965..8c4594cd37c70 100644
--- a/sql/core/benchmarks/PrimitiveArrayBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/PrimitiveArrayBenchmark-jdk21-results.txt
@@ -2,11 +2,11 @@
Write primitive arrays in dataset
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write an array in Dataset: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Int 182 226 39 46.2 21.7 1.0X
-Double 243 263 12 34.5 29.0 0.7X
+Int 180 204 13 46.5 21.5 1.0X
+Double 268 283 10 31.3 31.9 0.7X
diff --git a/sql/core/benchmarks/PrimitiveArrayBenchmark-results.txt b/sql/core/benchmarks/PrimitiveArrayBenchmark-results.txt
index 569b9714bed1e..ae32c6361729f 100644
--- a/sql/core/benchmarks/PrimitiveArrayBenchmark-results.txt
+++ b/sql/core/benchmarks/PrimitiveArrayBenchmark-results.txt
@@ -2,11 +2,11 @@
Write primitive arrays in dataset
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write an array in Dataset: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Int 205 226 21 40.9 24.4 1.0X
-Double 276 288 15 30.4 32.9 0.7X
+Int 212 226 8 39.6 25.3 1.0X
+Double 283 308 26 29.6 33.8 0.7X
diff --git a/sql/core/benchmarks/RangeBenchmark-jdk21-results.txt b/sql/core/benchmarks/RangeBenchmark-jdk21-results.txt
index 39b54801fedd2..f63e056208c29 100644
--- a/sql/core/benchmarks/RangeBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/RangeBenchmark-jdk21-results.txt
@@ -2,14 +2,14 @@
range
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
range: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-full scan 10188 10609 498 51.5 19.4 1.0X
-limit after range 51 53 2 10288.3 0.1 199.9X
-filter after range 1017 1035 21 515.5 1.9 10.0X
-count after range 351 355 5 1495.4 0.7 29.1X
-count after limit after range 60 62 2 8676.6 0.1 168.6X
+full scan 10038 10468 600 52.2 19.1 1.0X
+limit after range 46 50 8 11408.3 0.1 218.4X
+filter after range 1010 1018 8 519.0 1.9 9.9X
+count after range 352 361 9 1489.1 0.7 28.5X
+count after limit after range 53 59 4 9913.1 0.1 189.8X
diff --git a/sql/core/benchmarks/RangeBenchmark-results.txt b/sql/core/benchmarks/RangeBenchmark-results.txt
index c8aec61eb707d..eabc7f1298ca8 100644
--- a/sql/core/benchmarks/RangeBenchmark-results.txt
+++ b/sql/core/benchmarks/RangeBenchmark-results.txt
@@ -2,14 +2,14 @@
range
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
range: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-full scan 9612 10290 492 54.5 18.3 1.0X
-limit after range 50 60 9 10545.4 0.1 193.3X
-filter after range 1004 1012 7 522.1 1.9 9.6X
-count after range 351 358 8 1494.1 0.7 27.4X
-count after limit after range 64 65 1 8190.7 0.1 150.2X
+full scan 9860 10025 111 53.2 18.8 1.0X
+limit after range 61 63 3 8663.2 0.1 162.9X
+filter after range 1009 1025 18 519.6 1.9 9.8X
+count after range 352 355 4 1489.4 0.7 28.0X
+count after limit after range 63 66 3 8265.4 0.1 155.4X
diff --git a/sql/core/benchmarks/RecursiveCTEBenchmark-jdk21-results.txt b/sql/core/benchmarks/RecursiveCTEBenchmark-jdk21-results.txt
new file mode 100644
index 0000000000000..fce114d867b2f
--- /dev/null
+++ b/sql/core/benchmarks/RecursiveCTEBenchmark-jdk21-results.txt
@@ -0,0 +1,95 @@
+================================================================================================
+Recursive CTE with only LocalRelation
+================================================================================================
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
+AMD EPYC 7763 64-Core Processor
+First 10 integers: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+--------------------------------------------------------------------------------------------------------------------------------------
+First 10 integers using VALUES 136 150 16 0.0 13629654.3 1.0X
+First 10 integers using SELECT 86 93 8 0.0 8592266.5 1.6X
+First 10 integers using SELECT and LIMIT 63 70 9 0.0 6270977.7 2.2X
+First 10 integers referencing external table in anchor 115 141 23 0.0 11516912.5 1.2X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
+AMD EPYC 7763 64-Core Processor
+First 20 integers: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+--------------------------------------------------------------------------------------------------------------------------------------
+First 20 integers using VALUES 75 81 8 0.0 3767343.2 1.0X
+First 20 integers using SELECT 87 95 7 0.0 4358911.5 0.9X
+First 20 integers using SELECT and LIMIT 59 62 3 0.0 2967261.6 1.3X
+First 20 integers referencing external table in anchor 113 117 3 0.0 5659810.4 0.7X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
+AMD EPYC 7763 64-Core Processor
+First 30 integers: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+--------------------------------------------------------------------------------------------------------------------------------------
+First 30 integers using VALUES 77 82 5 0.0 2555962.6 1.0X
+First 30 integers using SELECT 75 83 9 0.0 2488825.0 1.0X
+First 30 integers using SELECT and LIMIT 64 68 6 0.0 2131034.1 1.2X
+First 30 integers referencing external table in anchor 116 131 20 0.0 3870932.1 0.7X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
+AMD EPYC 7763 64-Core Processor
+First 40 integers: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+--------------------------------------------------------------------------------------------------------------------------------------
+First 40 integers using VALUES 79 80 0 0.0 1986696.6 1.0X
+First 40 integers using SELECT 75 78 4 0.0 1879019.3 1.1X
+First 40 integers using SELECT and LIMIT 68 71 5 0.0 1702571.8 1.2X
+First 40 integers referencing external table in anchor 114 128 15 0.0 2856194.8 0.7X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
+AMD EPYC 7763 64-Core Processor
+First 50 integers: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+--------------------------------------------------------------------------------------------------------------------------------------
+First 50 integers using VALUES 82 93 18 0.0 1633748.5 1.0X
+First 50 integers using SELECT 84 87 6 0.0 1678713.9 1.0X
+First 50 integers using SELECT and LIMIT 75 90 16 0.0 1508240.7 1.1X
+First 50 integers referencing external table in anchor 123 127 4 0.0 2451603.8 0.7X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
+AMD EPYC 7763 64-Core Processor
+First 60 integers: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+--------------------------------------------------------------------------------------------------------------------------------------
+First 60 integers using VALUES 91 94 3 0.0 1512050.0 1.0X
+First 60 integers using SELECT 95 98 4 0.0 1576008.7 1.0X
+First 60 integers using SELECT and LIMIT 87 87 0 0.0 1443233.9 1.0X
+First 60 integers referencing external table in anchor 134 137 5 0.0 2227818.3 0.7X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
+AMD EPYC 7763 64-Core Processor
+First 70 integers: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+--------------------------------------------------------------------------------------------------------------------------------------
+First 70 integers using VALUES 107 114 7 0.0 1531580.2 1.0X
+First 70 integers using SELECT 108 113 5 0.0 1545424.7 1.0X
+First 70 integers using SELECT and LIMIT 99 103 4 0.0 1413035.3 1.1X
+First 70 integers referencing external table in anchor 140 141 1 0.0 1994987.9 0.8X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
+AMD EPYC 7763 64-Core Processor
+First 80 integers: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+--------------------------------------------------------------------------------------------------------------------------------------
+First 80 integers using VALUES 120 125 5 0.0 1495890.7 1.0X
+First 80 integers using SELECT 119 126 7 0.0 1493331.3 1.0X
+First 80 integers using SELECT and LIMIT 101 103 2 0.0 1268107.2 1.2X
+First 80 integers referencing external table in anchor 154 165 14 0.0 1923761.7 0.8X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
+AMD EPYC 7763 64-Core Processor
+First 90 integers: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+--------------------------------------------------------------------------------------------------------------------------------------
+First 90 integers using VALUES 123 131 8 0.0 1363274.5 1.0X
+First 90 integers using SELECT 121 127 9 0.0 1345079.3 1.0X
+First 90 integers using SELECT and LIMIT 118 122 4 0.0 1305706.1 1.0X
+First 90 integers referencing external table in anchor 163 169 5 0.0 1813308.8 0.8X
+
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
+AMD EPYC 7763 64-Core Processor
+First 100 integers: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+---------------------------------------------------------------------------------------------------------------------------------------
+First 100 integers using VALUES 140 143 2 0.0 1404497.4 1.0X
+First 100 integers using SELECT 133 136 4 0.0 1334375.7 1.1X
+First 100 integers using SELECT and LIMIT 125 129 4 0.0 1246067.9 1.1X
+First 100 integers referencing external table in anchor 170 175 4 0.0 1703882.3 0.8X
+
+
diff --git a/sql/core/benchmarks/RecursiveCTEBenchmark-results.txt b/sql/core/benchmarks/RecursiveCTEBenchmark-results.txt
index 13c1cadc32bcf..a77aafe11591e 100644
--- a/sql/core/benchmarks/RecursiveCTEBenchmark-results.txt
+++ b/sql/core/benchmarks/RecursiveCTEBenchmark-results.txt
@@ -2,94 +2,94 @@
Recursive CTE with only LocalRelation
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Mac OS X 15.4
-Apple M3 Max
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
+AMD EPYC 7763 64-Core Processor
First 10 integers: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------
-First 10 integers using VALUES 31 41 11 0.0 3132641.6 1.0X
-First 10 integers using SELECT 27 29 3 0.0 2674441.6 1.2X
-First 10 integers using SELECT and LIMIT 21 22 0 0.0 2133933.4 1.5X
-First 10 integers referencing external table in anchor 54 59 7 0.0 5351133.4 0.6X
+First 10 integers using VALUES 151 163 18 0.0 15065707.8 1.0X
+First 10 integers using SELECT 108 112 5 0.0 10772405.6 1.4X
+First 10 integers using SELECT and LIMIT 86 90 3 0.0 8620888.3 1.7X
+First 10 integers referencing external table in anchor 150 158 9 0.0 15013999.1 1.0X
-OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Mac OS X 15.4
-Apple M3 Max
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
+AMD EPYC 7763 64-Core Processor
First 20 integers: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------
-First 20 integers using VALUES 25 25 1 0.0 1245431.3 1.0X
-First 20 integers using SELECT 24 24 1 0.0 1183625.0 1.1X
-First 20 integers using SELECT and LIMIT 26 35 12 0.0 1288108.4 1.0X
-First 20 integers referencing external table in anchor 66 89 23 0.0 3291745.9 0.4X
+First 20 integers using VALUES 77 92 14 0.0 3869518.5 1.0X
+First 20 integers using SELECT 91 94 4 0.0 4571792.9 0.8X
+First 20 integers using SELECT and LIMIT 65 72 7 0.0 3236348.5 1.2X
+First 20 integers referencing external table in anchor 124 141 15 0.0 6207737.9 0.6X
-OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Mac OS X 15.4
-Apple M3 Max
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
+AMD EPYC 7763 64-Core Processor
First 30 integers: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------
-First 30 integers using VALUES 25 25 1 0.0 826962.5 1.0X
-First 30 integers using SELECT 25 25 0 0.0 824329.2 1.0X
-First 30 integers using SELECT and LIMIT 24 25 1 0.0 803851.4 1.0X
-First 30 integers referencing external table in anchor 67 70 3 0.0 2247233.3 0.4X
+First 30 integers using VALUES 77 93 16 0.0 2571754.4 1.0X
+First 30 integers using SELECT 96 99 5 0.0 3187083.6 0.8X
+First 30 integers using SELECT and LIMIT 68 76 12 0.0 2250803.4 1.1X
+First 30 integers referencing external table in anchor 123 132 7 0.0 4113510.3 0.6X
-OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Mac OS X 15.4
-Apple M3 Max
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
+AMD EPYC 7763 64-Core Processor
First 40 integers: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------
-First 40 integers using VALUES 29 30 0 0.0 734365.6 1.0X
-First 40 integers using SELECT 29 30 1 0.0 725761.5 1.0X
-First 40 integers using SELECT and LIMIT 29 33 8 0.0 721877.1 1.0X
-First 40 integers referencing external table in anchor 74 85 9 0.0 1860972.9 0.4X
+First 40 integers using VALUES 82 89 7 0.0 2041074.7 1.0X
+First 40 integers using SELECT 85 89 6 0.0 2131638.0 1.0X
+First 40 integers using SELECT and LIMIT 78 80 4 0.0 1947446.2 1.0X
+First 40 integers referencing external table in anchor 118 122 4 0.0 2957424.4 0.7X
-OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Mac OS X 15.4
-Apple M3 Max
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
+AMD EPYC 7763 64-Core Processor
First 50 integers: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------
-First 50 integers using VALUES 34 34 1 0.0 673504.2 1.0X
-First 50 integers using SELECT 35 35 0 0.0 707847.5 1.0X
-First 50 integers using SELECT and LIMIT 32 32 0 0.0 637159.2 1.1X
-First 50 integers referencing external table in anchor 61 62 1 0.0 1217748.3 0.6X
+First 50 integers using VALUES 96 111 13 0.0 1922421.5 1.0X
+First 50 integers using SELECT 86 89 3 0.0 1718552.2 1.1X
+First 50 integers using SELECT and LIMIT 81 90 14 0.0 1610883.5 1.2X
+First 50 integers referencing external table in anchor 140 147 6 0.0 2806679.9 0.7X
-OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Mac OS X 15.4
-Apple M3 Max
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
+AMD EPYC 7763 64-Core Processor
First 60 integers: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------
-First 60 integers using VALUES 38 38 0 0.0 628445.8 1.0X
-First 60 integers using SELECT 38 38 0 0.0 635657.0 1.0X
-First 60 integers using SELECT and LIMIT 35 35 0 0.0 584701.4 1.1X
-First 60 integers referencing external table in anchor 67 81 24 0.0 1109077.8 0.6X
+First 60 integers using VALUES 95 99 4 0.0 1588658.6 1.0X
+First 60 integers using SELECT 94 96 3 0.0 1567532.8 1.0X
+First 60 integers using SELECT and LIMIT 87 87 0 0.0 1449613.9 1.1X
+First 60 integers referencing external table in anchor 138 147 8 0.0 2297241.5 0.7X
-OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Mac OS X 15.4
-Apple M3 Max
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
+AMD EPYC 7763 64-Core Processor
First 70 integers: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------
-First 70 integers using VALUES 42 42 0 0.0 602191.1 1.0X
-First 70 integers using SELECT 42 43 1 0.0 598091.7 1.0X
-First 70 integers using SELECT and LIMIT 40 45 7 0.0 567091.1 1.1X
-First 70 integers referencing external table in anchor 72 90 18 0.0 1033307.7 0.6X
+First 70 integers using VALUES 107 113 7 0.0 1527431.7 1.0X
+First 70 integers using SELECT 107 107 1 0.0 1522253.3 1.0X
+First 70 integers using SELECT and LIMIT 96 100 5 0.0 1376351.6 1.1X
+First 70 integers referencing external table in anchor 152 153 2 0.0 2174213.6 0.7X
-OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Mac OS X 15.4
-Apple M3 Max
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
+AMD EPYC 7763 64-Core Processor
First 80 integers: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------
-First 80 integers using VALUES 53 56 5 0.0 664916.7 1.0X
-First 80 integers using SELECT 47 51 6 0.0 583047.9 1.1X
-First 80 integers using SELECT and LIMIT 52 54 2 0.0 645460.9 1.0X
-First 80 integers referencing external table in anchor 83 95 19 0.0 1042742.7 0.6X
+First 80 integers using VALUES 118 125 11 0.0 1471536.7 1.0X
+First 80 integers using SELECT 118 119 2 0.0 1470698.4 1.0X
+First 80 integers using SELECT and LIMIT 107 109 2 0.0 1335795.1 1.1X
+First 80 integers referencing external table in anchor 156 164 9 0.0 1945739.9 0.8X
-OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Mac OS X 15.4
-Apple M3 Max
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
+AMD EPYC 7763 64-Core Processor
First 90 integers: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------
-First 90 integers using VALUES 50 51 2 0.0 554144.4 1.0X
-First 90 integers using SELECT 51 55 6 0.0 563051.4 1.0X
-First 90 integers using SELECT and LIMIT 47 48 1 0.0 517157.9 1.1X
-First 90 integers referencing external table in anchor 147 148 1 0.0 1637719.4 0.3X
+First 90 integers using VALUES 128 132 5 0.0 1426374.7 1.0X
+First 90 integers using SELECT 129 133 6 0.0 1431482.2 1.0X
+First 90 integers using SELECT and LIMIT 116 120 4 0.0 1283439.3 1.1X
+First 90 integers referencing external table in anchor 168 173 4 0.0 1871741.3 0.8X
-OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Mac OS X 15.4
-Apple M3 Max
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
+AMD EPYC 7763 64-Core Processor
First 100 integers: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------------
-First 100 integers using VALUES 55 57 2 0.0 550062.9 1.0X
-First 100 integers using SELECT 54 55 0 0.0 544030.0 1.0X
-First 100 integers using SELECT and LIMIT 51 55 6 0.0 511913.8 1.1X
-First 100 integers referencing external table in anchor 99 121 34 0.0 987736.3 0.6X
+First 100 integers using VALUES 140 142 4 0.0 1396375.8 1.0X
+First 100 integers using SELECT 140 143 4 0.0 1395777.3 1.0X
+First 100 integers using SELECT and LIMIT 127 129 3 0.0 1265041.6 1.1X
+First 100 integers referencing external table in anchor 181 186 5 0.0 1809088.1 0.8X
diff --git a/sql/core/benchmarks/SetOperationsBenchmark-jdk21-results.txt b/sql/core/benchmarks/SetOperationsBenchmark-jdk21-results.txt
index aab7d77c3c5e8..37555720abf42 100644
--- a/sql/core/benchmarks/SetOperationsBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/SetOperationsBenchmark-jdk21-results.txt
@@ -2,12 +2,12 @@
Set Operations Benchmark
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Parsing + Analysis: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-UNION ALL 293 368 90 0.1 19561.0 1.0X
-EXCEPT ALL 248 258 6 0.1 16558.5 1.2X
-INTERSECT ALL 242 256 16 0.1 16119.8 1.2X
+UNION ALL 133 258 127 0.1 8834.8 1.0X
+EXCEPT ALL 95 100 6 0.2 6332.4 1.4X
+INTERSECT ALL 95 100 4 0.2 6341.7 1.4X
diff --git a/sql/core/benchmarks/SetOperationsBenchmark-results.txt b/sql/core/benchmarks/SetOperationsBenchmark-results.txt
index 7be9efd5dfd38..f1efa0b25910d 100644
--- a/sql/core/benchmarks/SetOperationsBenchmark-results.txt
+++ b/sql/core/benchmarks/SetOperationsBenchmark-results.txt
@@ -2,12 +2,12 @@
Set Operations Benchmark
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Parsing + Analysis: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-UNION ALL 371 502 142 0.0 24706.1 1.0X
-EXCEPT ALL 270 281 8 0.1 17967.1 1.4X
-INTERSECT ALL 267 276 7 0.1 17797.3 1.4X
+UNION ALL 171 298 142 0.1 11411.6 1.0X
+EXCEPT ALL 123 130 6 0.1 8185.4 1.4X
+INTERSECT ALL 123 129 6 0.1 8200.2 1.4X
diff --git a/sql/core/benchmarks/SortBenchmark-jdk21-results.txt b/sql/core/benchmarks/SortBenchmark-jdk21-results.txt
index 9003f22a818e8..2c27c2abb065a 100644
--- a/sql/core/benchmarks/SortBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/SortBenchmark-jdk21-results.txt
@@ -2,15 +2,15 @@
radix sort
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
radix sort 25000000: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-reference TimSort key prefix array 8419 8432 18 3.0 336.8 1.0X
-reference Arrays.sort 2175 2202 38 11.5 87.0 3.9X
-radix sort one byte 66 70 4 378.7 2.6 127.5X
-radix sort two bytes 122 130 13 204.5 4.9 68.9X
-radix sort eight bytes 488 492 7 51.2 19.5 17.2X
-radix sort key prefix array 569 579 13 43.9 22.8 14.8X
+reference TimSort key prefix array 8401 8446 64 3.0 336.0 1.0X
+reference Arrays.sort 2180 2204 35 11.5 87.2 3.9X
+radix sort one byte 65 67 2 384.8 2.6 129.3X
+radix sort two bytes 121 130 7 207.3 4.8 69.6X
+radix sort eight bytes 467 472 5 53.6 18.7 18.0X
+radix sort key prefix array 558 560 2 44.8 22.3 15.1X
diff --git a/sql/core/benchmarks/SortBenchmark-results.txt b/sql/core/benchmarks/SortBenchmark-results.txt
index b8a4f188bc201..7490a3bbf0ee2 100644
--- a/sql/core/benchmarks/SortBenchmark-results.txt
+++ b/sql/core/benchmarks/SortBenchmark-results.txt
@@ -2,15 +2,15 @@
radix sort
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
radix sort 25000000: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-reference TimSort key prefix array 8082 8117 50 3.1 323.3 1.0X
-reference Arrays.sort 2066 2103 53 12.1 82.6 3.9X
-radix sort one byte 63 69 5 396.0 2.5 128.0X
-radix sort two bytes 119 130 6 210.5 4.7 68.1X
-radix sort eight bytes 492 502 7 50.8 19.7 16.4X
-radix sort key prefix array 905 922 16 27.6 36.2 8.9X
+reference TimSort key prefix array 8320 8326 9 3.0 332.8 1.0X
+reference Arrays.sort 2062 2087 35 12.1 82.5 4.0X
+radix sort one byte 63 70 6 398.2 2.5 132.5X
+radix sort two bytes 118 130 6 211.5 4.7 70.4X
+radix sort eight bytes 453 480 17 55.2 18.1 18.4X
+radix sort key prefix array 916 919 3 27.3 36.7 9.1X
diff --git a/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk21-results.txt b/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk21-results.txt
index 1e3fc590644ad..ede4e88fe91d8 100644
--- a/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk21-results.txt
@@ -2,143 +2,143 @@
put rows
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
putting 10000 rows (10000 rows to overwrite - rate 100): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------------
-In-memory 8 9 1 1.2 815.6 1.0X
-RocksDB (trackTotalNumberOfRows: true) 46 47 2 0.2 4559.1 0.2X
-RocksDB (trackTotalNumberOfRows: false) 17 18 1 0.6 1678.7 0.5X
+In-memory 9 10 1 1.2 856.0 1.0X
+RocksDB (trackTotalNumberOfRows: true) 47 49 2 0.2 4703.5 0.2X
+RocksDB (trackTotalNumberOfRows: false) 18 19 1 0.6 1799.8 0.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
putting 10000 rows (5000 rows to overwrite - rate 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------
-In-memory 8 9 1 1.3 798.1 1.0X
-RocksDB (trackTotalNumberOfRows: true) 47 48 2 0.2 4659.8 0.2X
-RocksDB (trackTotalNumberOfRows: false) 17 17 1 0.6 1663.4 0.5X
+In-memory 8 9 1 1.2 833.3 1.0X
+RocksDB (trackTotalNumberOfRows: true) 48 50 1 0.2 4832.1 0.2X
+RocksDB (trackTotalNumberOfRows: false) 18 19 1 0.6 1791.8 0.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
putting 10000 rows (1000 rows to overwrite - rate 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------
-In-memory 8 9 1 1.3 794.9 1.0X
-RocksDB (trackTotalNumberOfRows: true) 46 48 1 0.2 4625.7 0.2X
-RocksDB (trackTotalNumberOfRows: false) 17 17 1 0.6 1660.7 0.5X
+In-memory 8 10 1 1.2 811.2 1.0X
+RocksDB (trackTotalNumberOfRows: true) 48 50 1 0.2 4799.0 0.2X
+RocksDB (trackTotalNumberOfRows: false) 17 19 1 0.6 1741.7 0.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
putting 10000 rows (0 rows to overwrite - rate 0): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------
-In-memory 8 8 1 1.3 788.6 1.0X
-RocksDB (trackTotalNumberOfRows: true) 46 47 1 0.2 4557.0 0.2X
-RocksDB (trackTotalNumberOfRows: false) 17 17 1 0.6 1650.3 0.5X
+In-memory 8 10 1 1.2 810.7 1.0X
+RocksDB (trackTotalNumberOfRows: true) 48 49 1 0.2 4781.2 0.2X
+RocksDB (trackTotalNumberOfRows: false) 18 19 1 0.6 1790.3 0.5X
================================================================================================
merge rows
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
merging 10000 rows with 10 values per key (10000 rows to overwrite - rate 100): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------------------------------
-RocksDB (trackTotalNumberOfRows: true) 574 585 6 0.0 57387.8 1.0X
-RocksDB (trackTotalNumberOfRows: false) 181 186 3 0.1 18065.2 3.2X
+RocksDB (trackTotalNumberOfRows: true) 581 597 7 0.0 58107.0 1.0X
+RocksDB (trackTotalNumberOfRows: false) 192 197 2 0.1 19209.3 3.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
merging 10000 rows with 10 values per key (5000 rows to overwrite - rate 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------------------------------------
-RocksDB (trackTotalNumberOfRows: true) 504 515 5 0.0 50382.4 1.0X
-RocksDB (trackTotalNumberOfRows: false) 179 185 3 0.1 17882.2 2.8X
+RocksDB (trackTotalNumberOfRows: true) 507 521 5 0.0 50704.7 1.0X
+RocksDB (trackTotalNumberOfRows: false) 191 196 2 0.1 19078.2 2.7X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
merging 10000 rows with 10 values per key (1000 rows to overwrite - rate 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------------------------------------
-RocksDB (trackTotalNumberOfRows: true) 442 455 6 0.0 44235.2 1.0X
-RocksDB (trackTotalNumberOfRows: false) 180 185 3 0.1 17971.5 2.5X
+RocksDB (trackTotalNumberOfRows: true) 447 459 5 0.0 44665.5 1.0X
+RocksDB (trackTotalNumberOfRows: false) 189 195 2 0.1 18919.4 2.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
merging 10000 rows with 10 values per key (0 rows to overwrite - rate 0): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------------------------
-RocksDB (trackTotalNumberOfRows: true) 424 436 5 0.0 42391.9 1.0X
-RocksDB (trackTotalNumberOfRows: false) 179 185 4 0.1 17923.5 2.4X
+RocksDB (trackTotalNumberOfRows: true) 435 448 5 0.0 43468.5 1.0X
+RocksDB (trackTotalNumberOfRows: false) 192 197 2 0.1 19164.7 2.3X
================================================================================================
delete rows
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
trying to delete 10000 rows from 10000 rows(10000 rows are non-existing - rate 100): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------------------------------
-In-memory 0 1 0 27.1 36.9 1.0X
-RocksDB (trackTotalNumberOfRows: true) 45 46 1 0.2 4470.0 0.0X
-RocksDB (trackTotalNumberOfRows: false) 16 17 1 0.6 1583.0 0.0X
+In-memory 1 1 0 16.9 59.1 1.0X
+RocksDB (trackTotalNumberOfRows: true) 46 47 1 0.2 4604.1 0.0X
+RocksDB (trackTotalNumberOfRows: false) 17 18 0 0.6 1720.5 0.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
trying to delete 10000 rows from 10000 rows(5000 rows are non-existing - rate 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------------------------------------------
-In-memory 7 7 0 1.5 651.4 1.0X
-RocksDB (trackTotalNumberOfRows: true) 46 47 1 0.2 4580.3 0.1X
-RocksDB (trackTotalNumberOfRows: false) 16 17 0 0.6 1582.7 0.4X
+In-memory 8 9 1 1.3 762.2 1.0X
+RocksDB (trackTotalNumberOfRows: true) 47 48 1 0.2 4666.1 0.2X
+RocksDB (trackTotalNumberOfRows: false) 17 18 1 0.6 1710.9 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
trying to delete 10000 rows from 10000 rows(1000 rows are non-existing - rate 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------------------------------------------
-In-memory 7 8 0 1.4 713.7 1.0X
-RocksDB (trackTotalNumberOfRows: true) 45 47 1 0.2 4538.6 0.2X
-RocksDB (trackTotalNumberOfRows: false) 16 16 0 0.6 1579.3 0.5X
+In-memory 7 10 1 1.3 748.9 1.0X
+RocksDB (trackTotalNumberOfRows: true) 46 47 1 0.2 4580.6 0.2X
+RocksDB (trackTotalNumberOfRows: false) 17 18 0 0.6 1721.2 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
trying to delete 10000 rows from 10000 rows(0 rows are non-existing - rate 0): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------------------------
-In-memory 7 8 0 1.4 716.9 1.0X
-RocksDB (trackTotalNumberOfRows: true) 45 46 1 0.2 4459.8 0.2X
-RocksDB (trackTotalNumberOfRows: false) 16 16 1 0.6 1580.7 0.5X
+In-memory 7 9 1 1.3 746.4 1.0X
+RocksDB (trackTotalNumberOfRows: true) 45 46 1 0.2 4479.1 0.2X
+RocksDB (trackTotalNumberOfRows: false) 17 18 0 0.6 1658.2 0.5X
================================================================================================
evict rows
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
evicting 10000 rows (maxTimestampToEvictInMillis: 9999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------------------
-In-memory 7 7 0 1.5 689.5 1.0X
-RocksDB (trackTotalNumberOfRows: true) 44 45 1 0.2 4424.0 0.2X
-RocksDB (trackTotalNumberOfRows: false) 18 18 0 0.6 1784.2 0.4X
+In-memory 7 7 0 1.4 693.6 1.0X
+RocksDB (trackTotalNumberOfRows: true) 43 44 1 0.2 4315.4 0.2X
+RocksDB (trackTotalNumberOfRows: false) 17 18 0 0.6 1724.8 0.4X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
evicting 5000 rows (maxTimestampToEvictInMillis: 4999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------------------------------
-In-memory 6 7 0 1.5 650.0 1.0X
-RocksDB (trackTotalNumberOfRows: true) 23 24 1 0.4 2347.8 0.3X
-RocksDB (trackTotalNumberOfRows: false) 10 11 0 1.0 1037.1 0.6X
+In-memory 7 7 0 1.5 650.5 1.0X
+RocksDB (trackTotalNumberOfRows: true) 23 24 1 0.4 2308.7 0.3X
+RocksDB (trackTotalNumberOfRows: false) 10 11 0 1.0 1037.6 0.6X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
evicting 1000 rows (maxTimestampToEvictInMillis: 999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------------------------------
-In-memory 6 6 0 1.7 585.4 1.0X
-RocksDB (trackTotalNumberOfRows: true) 8 8 0 1.3 766.5 0.8X
-RocksDB (trackTotalNumberOfRows: false) 5 5 0 2.0 503.2 1.2X
+In-memory 6 7 0 1.7 598.8 1.0X
+RocksDB (trackTotalNumberOfRows: true) 7 8 0 1.3 744.0 0.8X
+RocksDB (trackTotalNumberOfRows: false) 5 5 0 2.0 488.2 1.2X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
evicting 0 rows (maxTimestampToEvictInMillis: -1) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------------
-In-memory 0 0 0 25.0 40.1 1.0X
-RocksDB (trackTotalNumberOfRows: true) 4 4 0 2.8 359.1 0.1X
-RocksDB (trackTotalNumberOfRows: false) 4 4 0 2.8 359.9 0.1X
+In-memory 0 1 0 23.1 43.3 1.0X
+RocksDB (trackTotalNumberOfRows: true) 3 4 0 2.9 349.9 0.1X
+RocksDB (trackTotalNumberOfRows: false) 4 4 1 2.9 350.0 0.1X
diff --git a/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-results.txt b/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-results.txt
index a3688c16f100a..19264ccf4a1a8 100644
--- a/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-results.txt
+++ b/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-results.txt
@@ -2,143 +2,143 @@
put rows
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
putting 10000 rows (10000 rows to overwrite - rate 100): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------------
-In-memory 8 9 1 1.2 816.3 1.0X
-RocksDB (trackTotalNumberOfRows: true) 45 47 1 0.2 4514.1 0.2X
-RocksDB (trackTotalNumberOfRows: false) 17 18 1 0.6 1682.7 0.5X
+In-memory 8 9 1 1.2 806.5 1.0X
+RocksDB (trackTotalNumberOfRows: true) 46 48 1 0.2 4641.9 0.2X
+RocksDB (trackTotalNumberOfRows: false) 17 18 1 0.6 1739.9 0.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
putting 10000 rows (5000 rows to overwrite - rate 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------
-In-memory 8 10 1 1.2 811.7 1.0X
-RocksDB (trackTotalNumberOfRows: true) 47 49 1 0.2 4694.9 0.2X
-RocksDB (trackTotalNumberOfRows: false) 17 18 1 0.6 1680.2 0.5X
+In-memory 8 9 0 1.2 801.7 1.0X
+RocksDB (trackTotalNumberOfRows: true) 48 50 1 0.2 4800.8 0.2X
+RocksDB (trackTotalNumberOfRows: false) 17 18 1 0.6 1720.0 0.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
putting 10000 rows (1000 rows to overwrite - rate 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------
-In-memory 8 9 1 1.3 786.5 1.0X
-RocksDB (trackTotalNumberOfRows: true) 47 48 1 0.2 4679.7 0.2X
-RocksDB (trackTotalNumberOfRows: false) 17 18 1 0.6 1650.0 0.5X
+In-memory 8 8 0 1.3 782.4 1.0X
+RocksDB (trackTotalNumberOfRows: true) 46 47 1 0.2 4614.3 0.2X
+RocksDB (trackTotalNumberOfRows: false) 17 18 1 0.6 1702.2 0.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
putting 10000 rows (0 rows to overwrite - rate 0): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------
-In-memory 8 8 1 1.3 778.0 1.0X
-RocksDB (trackTotalNumberOfRows: true) 46 48 1 0.2 4629.4 0.2X
-RocksDB (trackTotalNumberOfRows: false) 17 17 1 0.6 1664.9 0.5X
+In-memory 8 8 0 1.3 771.9 1.0X
+RocksDB (trackTotalNumberOfRows: true) 46 47 1 0.2 4554.4 0.2X
+RocksDB (trackTotalNumberOfRows: false) 17 18 1 0.6 1698.3 0.5X
================================================================================================
merge rows
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
merging 10000 rows with 10 values per key (10000 rows to overwrite - rate 100): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------------------------------
-RocksDB (trackTotalNumberOfRows: true) 570 585 6 0.0 56996.2 1.0X
-RocksDB (trackTotalNumberOfRows: false) 184 190 3 0.1 18411.4 3.1X
+RocksDB (trackTotalNumberOfRows: true) 580 592 5 0.0 57963.9 1.0X
+RocksDB (trackTotalNumberOfRows: false) 194 199 3 0.1 19366.7 3.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
merging 10000 rows with 10 values per key (5000 rows to overwrite - rate 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------------------------------------
-RocksDB (trackTotalNumberOfRows: true) 493 505 5 0.0 49327.2 1.0X
-RocksDB (trackTotalNumberOfRows: false) 181 188 3 0.1 18140.8 2.7X
+RocksDB (trackTotalNumberOfRows: true) 509 522 7 0.0 50899.9 1.0X
+RocksDB (trackTotalNumberOfRows: false) 194 199 3 0.1 19417.5 2.6X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
merging 10000 rows with 10 values per key (1000 rows to overwrite - rate 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------------------------------------
-RocksDB (trackTotalNumberOfRows: true) 435 448 5 0.0 43484.3 1.0X
-RocksDB (trackTotalNumberOfRows: false) 183 188 3 0.1 18289.1 2.4X
+RocksDB (trackTotalNumberOfRows: true) 450 460 4 0.0 44950.2 1.0X
+RocksDB (trackTotalNumberOfRows: false) 192 197 2 0.1 19171.3 2.3X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
merging 10000 rows with 10 values per key (0 rows to overwrite - rate 0): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------------------------
-RocksDB (trackTotalNumberOfRows: true) 416 432 5 0.0 41606.2 1.0X
-RocksDB (trackTotalNumberOfRows: false) 183 189 3 0.1 18282.2 2.3X
+RocksDB (trackTotalNumberOfRows: true) 429 439 4 0.0 42875.5 1.0X
+RocksDB (trackTotalNumberOfRows: false) 191 197 2 0.1 19059.3 2.2X
================================================================================================
delete rows
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
trying to delete 10000 rows from 10000 rows(10000 rows are non-existing - rate 100): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------------------------------
-In-memory 0 1 0 26.6 37.7 1.0X
-RocksDB (trackTotalNumberOfRows: true) 45 47 1 0.2 4514.1 0.0X
-RocksDB (trackTotalNumberOfRows: false) 16 17 0 0.6 1587.8 0.0X
+In-memory 0 1 0 25.6 39.0 1.0X
+RocksDB (trackTotalNumberOfRows: true) 46 47 1 0.2 4575.3 0.0X
+RocksDB (trackTotalNumberOfRows: false) 17 18 0 0.6 1684.7 0.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
trying to delete 10000 rows from 10000 rows(5000 rows are non-existing - rate 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------------------------------------------
-In-memory 6 7 1 1.6 644.9 1.0X
-RocksDB (trackTotalNumberOfRows: true) 45 47 1 0.2 4524.6 0.1X
-RocksDB (trackTotalNumberOfRows: false) 16 17 1 0.6 1579.1 0.4X
+In-memory 6 7 0 1.6 630.0 1.0X
+RocksDB (trackTotalNumberOfRows: true) 47 48 1 0.2 4667.7 0.1X
+RocksDB (trackTotalNumberOfRows: false) 17 18 0 0.6 1697.3 0.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
trying to delete 10000 rows from 10000 rows(1000 rows are non-existing - rate 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------------------------------------------
-In-memory 7 8 1 1.4 698.2 1.0X
-RocksDB (trackTotalNumberOfRows: true) 45 46 1 0.2 4481.1 0.2X
-RocksDB (trackTotalNumberOfRows: false) 16 17 1 0.6 1585.3 0.4X
+In-memory 7 7 0 1.4 701.4 1.0X
+RocksDB (trackTotalNumberOfRows: true) 46 47 1 0.2 4590.1 0.2X
+RocksDB (trackTotalNumberOfRows: false) 17 18 0 0.6 1698.0 0.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
trying to delete 10000 rows from 10000 rows(0 rows are non-existing - rate 0): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------------------------
-In-memory 7 8 1 1.4 707.0 1.0X
-RocksDB (trackTotalNumberOfRows: true) 43 45 1 0.2 4326.6 0.2X
-RocksDB (trackTotalNumberOfRows: false) 16 17 1 0.6 1560.6 0.5X
+In-memory 7 7 0 1.4 716.7 1.0X
+RocksDB (trackTotalNumberOfRows: true) 46 47 1 0.2 4563.4 0.2X
+RocksDB (trackTotalNumberOfRows: false) 17 18 0 0.6 1665.2 0.4X
================================================================================================
evict rows
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
evicting 10000 rows (maxTimestampToEvictInMillis: 9999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------------------
-In-memory 7 7 0 1.4 693.7 1.0X
-RocksDB (trackTotalNumberOfRows: true) 43 44 1 0.2 4285.3 0.2X
-RocksDB (trackTotalNumberOfRows: false) 17 18 0 0.6 1726.3 0.4X
+In-memory 7 7 0 1.5 683.9 1.0X
+RocksDB (trackTotalNumberOfRows: true) 45 46 1 0.2 4454.0 0.2X
+RocksDB (trackTotalNumberOfRows: false) 18 18 0 0.6 1771.2 0.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
evicting 5000 rows (maxTimestampToEvictInMillis: 4999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------------------------------
-In-memory 6 7 0 1.5 646.3 1.0X
-RocksDB (trackTotalNumberOfRows: true) 24 24 0 0.4 2351.2 0.3X
-RocksDB (trackTotalNumberOfRows: false) 11 11 0 0.9 1062.9 0.6X
+In-memory 6 7 0 1.6 631.7 1.0X
+RocksDB (trackTotalNumberOfRows: true) 24 25 0 0.4 2414.0 0.3X
+RocksDB (trackTotalNumberOfRows: false) 11 11 0 0.9 1066.8 0.6X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
evicting 1000 rows (maxTimestampToEvictInMillis: 999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------------------------------
-In-memory 6 6 0 1.7 587.7 1.0X
-RocksDB (trackTotalNumberOfRows: true) 8 8 0 1.3 784.7 0.7X
-RocksDB (trackTotalNumberOfRows: false) 5 6 0 1.9 529.1 1.1X
+In-memory 6 6 0 1.7 574.0 1.0X
+RocksDB (trackTotalNumberOfRows: true) 8 8 0 1.3 780.8 0.7X
+RocksDB (trackTotalNumberOfRows: false) 5 5 0 2.0 507.7 1.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
evicting 0 rows (maxTimestampToEvictInMillis: -1) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------------
-In-memory 0 0 0 23.2 43.2 1.0X
-RocksDB (trackTotalNumberOfRows: true) 4 4 0 2.6 387.5 0.1X
-RocksDB (trackTotalNumberOfRows: false) 4 4 0 2.6 389.4 0.1X
+In-memory 0 0 0 22.6 44.3 1.0X
+RocksDB (trackTotalNumberOfRows: true) 4 4 0 2.8 360.5 0.1X
+RocksDB (trackTotalNumberOfRows: false) 4 4 0 2.8 360.8 0.1X
diff --git a/sql/core/benchmarks/StringFunctionsBenchmark-jdk21-results.txt b/sql/core/benchmarks/StringFunctionsBenchmark-jdk21-results.txt
index c1684fae10653..caff1265db41e 100644
--- a/sql/core/benchmarks/StringFunctionsBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/StringFunctionsBenchmark-jdk21-results.txt
@@ -2,10 +2,10 @@
SQL string functions
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
regexp_replace: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-regexp_replace('*-*', '(\\d+)', 'num') 500 515 11 2.0 499.9 1.0X
+regexp_replace('*-*', '(\\d+)', 'num') 491 512 24 2.0 491.4 1.0X
diff --git a/sql/core/benchmarks/StringFunctionsBenchmark-results.txt b/sql/core/benchmarks/StringFunctionsBenchmark-results.txt
index a6571007f7aac..ae35dc90be5e3 100644
--- a/sql/core/benchmarks/StringFunctionsBenchmark-results.txt
+++ b/sql/core/benchmarks/StringFunctionsBenchmark-results.txt
@@ -2,10 +2,10 @@
SQL string functions
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
regexp_replace: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-regexp_replace('*-*', '(\\d+)', 'num') 500 514 13 2.0 499.7 1.0X
+regexp_replace('*-*', '(\\d+)', 'num') 536 552 14 1.9 536.2 1.0X
diff --git a/sql/core/benchmarks/SubExprEliminationBenchmark-jdk21-results.txt b/sql/core/benchmarks/SubExprEliminationBenchmark-jdk21-results.txt
index 8cf0ba7224bad..ba1109cab5578 100644
--- a/sql/core/benchmarks/SubExprEliminationBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/SubExprEliminationBenchmark-jdk21-results.txt
@@ -3,23 +3,23 @@ Benchmark for performance of subexpression elimination
================================================================================================
Preparing data for benchmarking ...
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1021-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
from_json as subExpr in Project: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-subExprElimination false, codegen: true 6700 7047 301 0.0 67001649.1 1.0X
-subExprElimination false, codegen: false 6719 6837 118 0.0 67191470.6 1.0X
-subExprElimination true, codegen: true 1350 1489 122 0.0 13503842.8 5.0X
-subExprElimination true, codegen: false 1366 1444 96 0.0 13658823.9 4.9X
+subExprElimination false, codegen: true 6356 6511 154 0.0 63562019.6 1.0X
+subExprElimination false, codegen: false 6308 6384 66 0.0 63084255.4 1.0X
+subExprElimination true, codegen: true 1210 1265 93 0.0 12099184.7 5.3X
+subExprElimination true, codegen: false 1236 1299 99 0.0 12360076.2 5.1X
Preparing data for benchmarking ...
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1021-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
from_json as subExpr in Filter: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-subExprElimination false, codegen: true 7250 7520 384 0.0 72501549.6 1.0X
-subExprElimination false, codegen: false 7255 7366 114 0.0 72554716.3 1.0X
-subExprElimination true, codegen: true 1934 2024 79 0.0 19344228.2 3.7X
-subExprElimination true, codegen: false 1981 2015 29 0.0 19814306.4 3.7X
+subExprElimination false, codegen: true 6485 6649 174 0.0 64852199.5 1.0X
+subExprElimination false, codegen: false 6390 6489 87 0.0 63895276.1 1.0X
+subExprElimination true, codegen: true 1580 1633 52 0.0 15799544.7 4.1X
+subExprElimination true, codegen: false 1619 1649 27 0.0 16193983.0 4.0X
diff --git a/sql/core/benchmarks/SubExprEliminationBenchmark-results.txt b/sql/core/benchmarks/SubExprEliminationBenchmark-results.txt
index b689fef720f78..2a0da13bab2ec 100644
--- a/sql/core/benchmarks/SubExprEliminationBenchmark-results.txt
+++ b/sql/core/benchmarks/SubExprEliminationBenchmark-results.txt
@@ -3,23 +3,23 @@ Benchmark for performance of subexpression elimination
================================================================================================
Preparing data for benchmarking ...
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1021-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
from_json as subExpr in Project: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-subExprElimination false, codegen: true 6389 6498 173 0.0 63887225.6 1.0X
-subExprElimination false, codegen: false 6235 6292 81 0.0 62351284.2 1.0X
-subExprElimination true, codegen: true 1328 1368 47 0.0 13284825.9 4.8X
-subExprElimination true, codegen: false 1323 1368 73 0.0 13227629.0 4.8X
+subExprElimination false, codegen: true 6643 6703 53 0.0 66433286.4 1.0X
+subExprElimination false, codegen: false 6408 6436 26 0.0 64082328.5 1.0X
+subExprElimination true, codegen: true 1545 1596 44 0.0 15454792.9 4.3X
+subExprElimination true, codegen: false 1459 1548 79 0.0 14592926.1 4.6X
Preparing data for benchmarking ...
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1021-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
from_json as subExpr in Filter: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-subExprElimination false, codegen: true 7081 7177 86 0.0 70813603.9 1.0X
-subExprElimination false, codegen: false 6586 6720 139 0.0 65859888.8 1.1X
-subExprElimination true, codegen: true 1729 1827 117 0.0 17291697.7 4.1X
-subExprElimination true, codegen: false 1726 1789 57 0.0 17255779.7 4.1X
+subExprElimination false, codegen: true 6726 6789 69 0.0 67259727.5 1.0X
+subExprElimination false, codegen: false 6800 6851 77 0.0 68003862.4 1.0X
+subExprElimination true, codegen: true 1833 1906 74 0.0 18325029.8 3.7X
+subExprElimination true, codegen: false 1834 1901 64 0.0 18341410.4 3.7X
diff --git a/sql/core/benchmarks/TPCDSQueryBenchmark-jdk21-results.txt b/sql/core/benchmarks/TPCDSQueryBenchmark-jdk21-results.txt
index c92e2b6d44980..808031f4085cb 100644
--- a/sql/core/benchmarks/TPCDSQueryBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/TPCDSQueryBenchmark-jdk21-results.txt
@@ -1,810 +1,810 @@
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q1 646 812 151 0.7 1400.6 1.0X
+q1 702 806 129 0.7 1521.4 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q2 889 901 21 2.5 398.3 1.0X
+q2 882 961 72 2.5 395.4 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q3 230 279 41 12.9 77.5 1.0X
+q3 241 276 33 12.3 81.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q4 5117 5164 67 1.0 981.8 1.0X
+q4 4969 5281 442 1.0 953.3 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q5 1140 1339 282 4.9 202.5 1.0X
+q5 1126 1294 239 5.0 200.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q6 1125 1137 17 2.8 360.4 1.0X
+q6 1103 1185 115 2.8 353.6 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q7 604 636 33 8.1 123.4 1.0X
+q7 591 630 45 8.3 120.7 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q8 489 517 32 6.3 157.5 1.0X
+q8 435 509 93 7.1 140.3 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q9 900 925 23 0.0 25708761.0 1.0X
+q9 878 926 41 0.0 25082543.9 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q10 1997 2109 159 1.0 964.3 1.0X
+q10 2047 2064 24 1.0 988.5 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q11 1857 2054 279 2.0 492.4 1.0X
+q11 1758 2001 343 2.1 466.2 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q12 163 199 39 5.0 201.1 1.0X
+q12 173 203 31 4.7 213.9 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q13 817 855 39 6.0 165.7 1.0X
+q13 775 821 61 6.4 157.2 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q14a 5011 5171 226 1.0 976.8 1.0X
+q14a 4554 4759 290 1.1 887.8 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q14b 4082 4106 34 1.3 795.7 1.0X
+q14b 3718 3760 60 1.4 724.7 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q15 391 410 34 4.3 235.2 1.0X
+q15 404 431 41 4.1 242.7 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q16 595 666 49 2.6 380.5 1.0X
+q16 606 653 59 2.6 387.5 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q17 1243 1338 133 3.8 264.6 1.0X
+q17 1341 1361 29 3.5 285.4 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q18 1051 1247 277 3.4 291.8 1.0X
+q18 1128 1281 216 3.2 313.2 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q19 303 321 21 10.3 97.1 1.0X
+q19 305 328 28 10.2 97.9 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q20 183 207 32 8.4 119.6 1.0X
+q20 186 208 49 8.2 121.4 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q21 660 683 39 17.9 55.7 1.0X
+q21 667 693 39 17.7 56.4 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q22 3373 3418 64 3.5 285.0 1.0X
+q22 3318 3393 106 3.6 280.3 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q23a 5880 6049 239 0.9 1124.3 1.0X
+q23a 5982 6138 220 0.9 1143.8 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q23b 5821 5955 189 0.9 1113.1 1.0X
+q23b 6036 6134 139 0.9 1154.2 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q24a 212 266 43 15.7 63.7 1.0X
+q24a 221 255 44 15.1 66.1 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q24b 223 261 48 15.0 66.7 1.0X
+q24b 215 245 23 15.5 64.5 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q25 1315 1343 40 3.6 279.8 1.0X
+q25 1279 1346 95 3.7 272.1 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q26 357 386 36 9.7 103.3 1.0X
+q26 348 381 32 9.9 100.7 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q27 577 611 27 8.5 117.9 1.0X
+q27 574 611 42 8.5 117.4 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q28 1373 1414 57 2.1 476.8 1.0X
+q28 1319 1411 130 2.2 458.1 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q29 1235 1300 93 3.8 262.7 1.0X
+q29 1311 1368 80 3.6 279.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q30 390 438 38 0.8 1323.4 1.0X
+q30 410 435 28 0.7 1390.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q31 811 1114 429 4.6 217.9 1.0X
+q31 805 920 186 4.6 216.4 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q32 221 234 9 6.9 144.7 1.0X
+q32 206 242 59 7.4 134.9 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q33 429 467 39 12.1 82.8 1.0X
+q33 386 419 38 13.4 74.6 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q34 371 407 25 8.3 121.2 1.0X
+q34 359 391 36 8.5 117.4 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q35 1252 1302 71 1.7 604.6 1.0X
+q35 1234 1281 66 1.7 596.1 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q36 533 560 41 5.6 179.4 1.0X
+q36 529 565 40 5.6 178.2 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q37 798 827 26 16.6 60.1 1.0X
+q37 841 875 35 15.8 63.3 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q38 641 711 65 8.1 123.1 1.0X
+q38 657 703 62 7.9 126.1 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q39a 1349 1486 193 8.8 114.0 1.0X
+q39a 1502 1556 77 7.9 126.9 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q39b 1360 1447 123 8.7 114.9 1.0X
+q39b 1406 1428 32 8.4 118.8 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q40 309 401 71 5.4 184.6 1.0X
+q40 318 349 35 5.3 190.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q41 137 158 27 0.1 7598.4 1.0X
+q41 137 157 19 0.1 7635.1 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q42 149 161 17 19.9 50.1 1.0X
+q42 150 172 30 19.8 50.5 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q43 316 347 34 9.3 107.2 1.0X
+q43 314 339 31 9.4 106.4 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q44 351 381 29 8.3 121.1 1.0X
+q44 341 369 28 8.5 117.5 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q45 211 231 15 4.5 220.1 1.0X
+q45 186 218 34 5.2 194.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q46 482 503 25 6.5 154.9 1.0X
+q46 458 493 37 6.8 147.3 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q47 1486 1605 168 2.0 500.1 1.0X
+q47 1658 1673 22 1.8 557.9 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q48 943 947 4 5.2 191.6 1.0X
+q48 878 919 42 5.6 178.2 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q49 585 670 127 9.6 104.1 1.0X
+q49 537 587 34 10.5 95.6 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q50 641 693 50 5.1 197.8 1.0X
+q50 627 686 62 5.2 193.4 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q51 2650 2698 68 1.4 721.7 1.0X
+q51 2585 2765 254 1.4 704.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q52 150 182 27 19.8 50.6 1.0X
+q52 152 165 18 19.5 51.3 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q53 282 322 37 10.5 94.9 1.0X
+q53 278 308 28 10.7 93.6 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q54 1220 1248 40 4.3 231.0 1.0X
+q54 1205 1236 44 4.4 228.2 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q55 146 160 19 20.3 49.2 1.0X
+q55 151 166 20 19.7 50.8 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q56 402 447 43 12.9 77.7 1.0X
+q56 386 433 55 13.4 74.6 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q57 759 820 71 2.0 495.7 1.0X
+q57 743 840 107 2.1 485.4 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q58 403 428 30 12.7 78.6 1.0X
+q58 462 522 108 11.1 90.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q59 702 715 14 4.2 237.6 1.0X
+q59 671 689 23 4.4 227.1 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q60 422 468 46 12.3 81.6 1.0X
+q60 401 416 15 12.9 77.3 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q61 537 571 35 5.8 172.2 1.0X
+q61 523 575 85 6.0 167.6 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q62 180 194 21 4.4 227.4 1.0X
+q62 180 203 28 4.4 227.5 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q63 276 296 22 10.8 92.9 1.0X
+q63 283 302 32 10.5 95.1 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q64 2183 2459 391 3.2 315.4 1.0X
+q64 2215 2452 336 3.1 320.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q65 594 646 66 5.0 200.0 1.0X
+q65 598 637 41 5.0 201.1 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q66 515 612 121 4.5 222.2 1.0X
+q66 464 503 46 5.0 200.3 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q67 5498 5512 19 0.5 1850.8 1.0X
+q67 5499 5604 149 0.5 1850.9 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q68 469 509 36 6.6 150.9 1.0X
+q68 430 476 44 7.2 138.4 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q69 1706 1714 11 1.2 824.0 1.0X
+q69 1662 1673 15 1.2 802.7 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q70 566 598 42 5.2 191.8 1.0X
+q70 586 612 41 5.0 198.5 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q71 374 408 30 14.0 71.7 1.0X
+q71 361 382 35 14.5 69.2 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q72 106812 113451 9389 0.1 6959.4 1.0X
+q72 119812 123190 4778 0.1 7806.4 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q73 349 371 36 8.8 113.9 1.0X
+q73 332 360 35 9.2 108.6 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q74 1195 1502 434 3.2 316.9 1.0X
+q74 1242 1393 214 3.0 329.4 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q75 1487 1666 252 3.8 264.0 1.0X
+q75 1269 1615 489 4.4 225.3 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q76 295 316 21 17.4 57.5 1.0X
+q76 291 321 26 17.6 56.8 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q77 538 719 163 10.4 95.7 1.0X
+q77 643 792 142 8.7 114.5 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q78 1963 2055 129 2.9 349.6 1.0X
+q78 2160 2354 275 2.6 384.6 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q79 418 451 26 7.3 136.7 1.0X
+q79 418 444 27 7.3 136.6 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q80 1551 1845 416 3.6 274.7 1.0X
+q80 1333 1458 177 4.2 236.1 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q81 339 390 39 1.1 924.9 1.0X
+q81 277 387 80 1.3 756.4 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q82 1101 1102 1 13.4 74.8 1.0X
+q82 1074 1103 41 13.7 73.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q83 244 283 39 2.4 409.7 1.0X
+q83 239 283 36 2.5 402.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q84 615 671 48 3.8 259.9 1.0X
+q84 646 686 53 3.7 273.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q85 1636 1926 410 1.7 577.2 1.0X
+q85 1638 1768 185 1.7 577.7 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q86 187 208 22 4.3 231.1 1.0X
+q86 185 205 20 4.4 228.9 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q87 652 719 81 8.0 125.0 1.0X
+q87 669 752 73 7.8 128.4 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q88 1207 1396 268 2.5 405.9 1.0X
+q88 1259 1351 131 2.4 423.3 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q89 317 365 43 9.4 106.8 1.0X
+q89 317 347 43 9.4 106.7 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q90 113 139 20 7.2 138.5 1.0X
+q90 117 139 22 6.9 144.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q91 335 381 45 6.9 145.8 1.0X
+q91 323 368 55 7.1 140.6 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q92 134 153 23 6.0 165.8 1.0X
+q92 135 158 21 6.0 166.6 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q93 402 441 42 7.9 126.9 1.0X
+q93 450 533 91 7.0 142.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q94 313 353 45 2.7 371.3 1.0X
+q94 331 360 31 2.5 393.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q95 5220 5267 66 0.2 6199.7 1.0X
+q95 5243 5383 197 0.2 6227.2 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q96 165 183 23 18.1 55.4 1.0X
+q96 172 187 20 17.2 58.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q97 1090 1139 70 4.0 248.2 1.0X
+q97 1177 1190 19 3.7 267.9 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q98 254 288 29 11.7 85.5 1.0X
+q98 276 296 30 10.8 92.9 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q99 267 288 24 5.7 176.7 1.0X
+q99 269 292 32 5.6 178.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q5a-v2.7 1315 1472 221 4.3 233.7 1.0X
+q5a-v2.7 1159 1215 80 4.9 206.0 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q6-v2.7 892 954 62 3.5 285.7 1.0X
+q6-v2.7 959 1002 60 3.3 307.4 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q10a-v2.7 1836 1927 129 1.1 886.5 1.0X
+q10a-v2.7 1899 1941 59 1.1 917.3 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q11-v2.7 1673 1998 460 2.3 443.5 1.0X
+q11-v2.7 1529 1934 573 2.5 405.5 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q12-v2.7 126 143 22 6.4 155.7 1.0X
+q12-v2.7 131 149 22 6.2 161.4 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q14-v2.7 3412 3660 351 1.5 665.1 1.0X
+q14-v2.7 3360 3619 366 1.5 655.1 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q14a-v2.7 7118 7330 301 0.7 1387.6 1.0X
+q14a-v2.7 7535 7653 167 0.7 1468.9 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q18a-v2.7 1954 2086 188 1.8 542.4 1.0X
+q18a-v2.7 2009 2283 388 1.8 557.8 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q20-v2.7 161 190 47 9.5 105.2 1.0X
+q20-v2.7 165 174 11 9.3 107.6 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q22-v2.7 13538 13568 42 0.9 1143.8 1.0X
+q22-v2.7 13296 13422 178 0.9 1123.4 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q22a-v2.7 1993 2057 90 5.9 168.4 1.0X
+q22a-v2.7 1956 2080 176 6.1 165.3 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q24-v2.7 202 233 25 16.5 60.6 1.0X
+q24-v2.7 197 225 27 16.9 59.1 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q27a-v2.7 1219 1552 470 4.0 249.2 1.0X
+q27a-v2.7 1351 1699 492 3.6 276.1 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q34-v2.7 347 377 28 8.8 113.4 1.0X
+q34-v2.7 344 378 50 8.9 112.3 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q35-v2.7 1211 1240 42 1.7 584.7 1.0X
+q35-v2.7 1196 1249 76 1.7 577.5 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q35a-v2.7 1225 1263 54 1.7 591.7 1.0X
+q35a-v2.7 1166 1223 82 1.8 562.9 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q36a-v2.7 492 525 38 6.0 165.7 1.0X
+q36a-v2.7 494 511 20 6.0 166.4 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q47-v2.7 1560 1705 205 1.9 525.2 1.0X
+q47-v2.7 1632 1657 36 1.8 549.2 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q49-v2.7 541 624 81 10.4 96.3 1.0X
+q49-v2.7 503 538 55 11.2 89.6 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q51a-v2.7 15126 15337 298 0.2 4119.6 1.0X
+q51a-v2.7 14959 15258 423 0.2 4074.1 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q57-v2.7 747 823 67 2.1 487.6 1.0X
+q57-v2.7 759 816 78 2.0 495.8 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q64-v2.7 2071 2354 400 3.3 299.3 1.0X
+q64-v2.7 2234 2554 453 3.1 322.8 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q67a-v2.7 6815 7072 363 0.4 2294.0 1.0X
+q67a-v2.7 7609 7799 270 0.4 2561.2 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q70a-v2.7 619 636 13 4.8 209.8 1.0X
+q70a-v2.7 626 674 57 4.7 212.1 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q72-v2.7 101776 104972 4519 0.2 6631.3 1.0X
+q72-v2.7 123476 124060 826 0.1 8045.1 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q74-v2.7 1219 1398 254 3.1 323.1 1.0X
+q74-v2.7 1378 1388 15 2.7 365.3 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q75-v2.7 1236 1484 351 4.6 219.5 1.0X
+q75-v2.7 1303 1621 450 4.3 231.3 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q77a-v2.7 780 895 156 7.2 138.9 1.0X
+q77a-v2.7 699 810 96 8.0 124.6 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q78-v2.7 1939 2304 516 2.9 345.2 1.0X
+q78-v2.7 2490 2537 67 2.3 443.4 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q80a-v2.7 1689 1700 14 3.3 299.3 1.0X
+q80a-v2.7 1751 1915 231 3.2 310.2 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q86a-v2.7 227 267 36 3.6 280.2 1.0X
+q86a-v2.7 236 267 36 3.4 291.8 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q98-v2.7 255 283 37 11.6 85.9 1.0X
+q98-v2.7 270 294 25 11.0 90.8 1.0X
diff --git a/sql/core/benchmarks/TPCDSQueryBenchmark-results.txt b/sql/core/benchmarks/TPCDSQueryBenchmark-results.txt
index 1a4205ac535ae..aa67c8b71436b 100644
--- a/sql/core/benchmarks/TPCDSQueryBenchmark-results.txt
+++ b/sql/core/benchmarks/TPCDSQueryBenchmark-results.txt
@@ -1,810 +1,810 @@
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q1 597 789 167 0.8 1294.9 1.0X
+q1 663 797 219 0.7 1437.1 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q2 800 887 76 2.8 358.5 1.0X
+q2 1027 1149 173 2.2 459.9 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q3 221 251 23 13.5 74.3 1.0X
+q3 245 280 23 12.1 82.4 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q4 4538 4797 367 1.1 870.7 1.0X
+q4 4521 4868 491 1.2 867.4 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q5 1210 1277 96 4.7 214.9 1.0X
+q5 1156 1372 306 4.9 205.3 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q6 1030 1052 31 3.0 330.1 1.0X
+q6 1128 1152 34 2.8 361.4 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q7 584 603 18 8.4 119.3 1.0X
+q7 706 753 70 6.9 144.3 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q8 455 490 33 6.8 146.5 1.0X
+q8 649 667 22 4.8 209.1 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q9 829 878 48 0.0 23682965.2 1.0X
+q9 976 1039 89 0.0 27895478.6 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q10 1969 2095 178 1.1 950.8 1.0X
+q10 1940 2072 187 1.1 936.7 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q11 1966 2226 367 1.9 521.3 1.0X
+q11 2074 2322 350 1.8 549.9 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q12 218 263 34 3.7 268.7 1.0X
+q12 218 267 26 3.7 268.7 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q13 854 892 42 5.8 173.1 1.0X
+q13 864 928 64 5.7 175.2 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q14a 5110 5424 444 1.0 996.1 1.0X
+q14a 4917 5335 592 1.0 958.5 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q14b 3621 3785 232 1.4 706.0 1.0X
+q14b 3663 3816 216 1.4 714.0 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q15 390 411 32 4.3 234.6 1.0X
+q15 415 439 26 4.0 249.6 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q16 709 748 48 2.2 453.5 1.0X
+q16 721 776 60 2.2 461.1 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q17 1340 1515 247 3.5 285.2 1.0X
+q17 1598 1735 194 2.9 340.0 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q18 1295 1421 178 2.8 359.6 1.0X
+q18 1385 1390 7 2.6 384.6 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q19 370 402 35 8.4 118.7 1.0X
+q19 381 401 20 8.2 122.2 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q20 223 254 35 6.9 145.9 1.0X
+q20 200 237 44 7.7 130.6 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q21 687 695 14 17.2 58.0 1.0X
+q21 724 743 17 16.4 61.2 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q22 3422 3542 171 3.5 289.1 1.0X
+q22 3529 3631 144 3.4 298.2 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q23a 5917 6315 563 0.9 1131.5 1.0X
+q23a 6128 6160 46 0.9 1171.7 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q23b 5882 5930 67 0.9 1124.8 1.0X
+q23b 5923 5963 56 0.9 1132.6 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q24a 232 295 61 14.4 69.6 1.0X
+q24a 273 320 46 12.2 81.8 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q24b 146 239 48 22.8 43.9 1.0X
+q24b 209 253 38 16.0 62.7 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q25 1270 1289 27 3.7 270.4 1.0X
+q25 1281 1291 15 3.7 272.5 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q26 390 419 26 8.8 113.1 1.0X
+q26 405 420 18 8.5 117.3 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q27 599 619 17 8.2 122.5 1.0X
+q27 592 614 25 8.3 121.1 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q28 1437 1530 131 2.0 499.1 1.0X
+q28 1481 1542 87 1.9 514.2 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q29 1438 1566 181 3.3 306.0 1.0X
+q29 1446 1547 142 3.2 307.7 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q30 471 514 36 0.6 1597.0 1.0X
+q30 450 521 62 0.7 1526.9 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q31 1152 1411 366 3.2 309.6 1.0X
+q31 1369 1443 104 2.7 367.9 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q32 238 273 49 6.4 155.5 1.0X
+q32 250 288 31 6.1 163.5 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q33 498 552 55 10.4 96.2 1.0X
+q33 463 547 138 11.2 89.5 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q34 436 465 39 7.0 142.6 1.0X
+q34 423 443 12 7.2 138.2 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q35 1494 1500 8 1.4 721.7 1.0X
+q35 1375 1418 62 1.5 663.8 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q36 590 599 14 5.0 198.5 1.0X
+q36 571 586 15 5.2 192.3 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q37 883 920 33 15.0 66.5 1.0X
+q37 906 912 6 14.6 68.3 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q38 953 1211 365 5.5 182.8 1.0X
+q38 812 1068 362 6.4 155.9 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q39a 1453 1643 269 8.1 122.7 1.0X
+q39a 1516 1733 307 7.8 128.1 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q39b 1440 1540 142 8.2 121.6 1.0X
+q39b 1564 1683 168 7.6 132.2 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q40 325 352 22 5.2 193.8 1.0X
+q40 343 357 17 4.9 204.7 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q41 152 185 27 0.1 8439.8 1.0X
+q41 154 179 18 0.1 8582.9 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q42 164 207 39 18.1 55.3 1.0X
+q42 172 193 24 17.3 57.9 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q43 354 369 14 8.4 119.8 1.0X
+q43 345 359 14 8.6 116.9 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q44 336 347 8 8.6 116.0 1.0X
+q44 340 373 31 8.5 117.4 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q45 190 226 39 5.1 197.9 1.0X
+q45 214 253 20 4.5 222.9 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q46 489 522 23 6.4 157.1 1.0X
+q46 517 560 49 6.0 166.2 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q47 1612 1816 288 1.8 542.6 1.0X
+q47 1564 1607 62 1.9 526.4 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q48 913 935 22 5.4 185.5 1.0X
+q48 813 873 56 6.1 165.1 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q49 715 829 190 7.8 127.4 1.0X
+q49 533 571 36 10.5 95.0 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q50 715 734 17 4.5 220.6 1.0X
+q50 755 763 9 4.3 232.9 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q51 2685 2797 158 1.4 731.3 1.0X
+q51 2352 2471 167 1.6 640.7 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q52 158 182 21 18.8 53.3 1.0X
+q52 151 158 8 19.7 50.7 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q53 298 338 38 10.0 100.4 1.0X
+q53 280 287 8 10.6 94.2 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q54 1280 1337 81 4.1 242.4 1.0X
+q54 1157 1204 68 4.6 219.1 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q55 166 189 25 17.9 55.7 1.0X
+q55 148 158 4 20.1 49.8 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q56 490 561 98 10.6 94.6 1.0X
+q56 473 529 54 11.0 91.2 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q57 770 882 166 2.0 502.8 1.0X
+q57 905 1198 414 1.7 591.2 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q58 415 440 29 12.3 81.0 1.0X
+q58 514 535 25 10.0 100.3 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q59 643 675 42 4.6 217.8 1.0X
+q59 652 688 31 4.5 220.7 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q60 574 594 33 9.0 110.9 1.0X
+q60 412 419 6 12.6 79.5 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q61 569 598 47 5.5 182.1 1.0X
+q61 628 686 52 5.0 201.3 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q62 181 187 7 4.4 228.0 1.0X
+q62 168 180 12 4.7 211.6 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q63 296 345 37 10.0 99.5 1.0X
+q63 279 293 18 10.7 93.8 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q64 2546 2784 338 2.7 367.8 1.0X
+q64 2533 2795 371 2.7 365.9 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q65 804 822 20 3.7 270.6 1.0X
+q65 727 730 4 4.1 244.6 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q66 611 660 75 3.8 263.7 1.0X
+q66 516 562 68 4.5 222.6 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q67 5472 5599 180 0.5 1841.9 1.0X
+q67 5325 5412 123 0.6 1792.5 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q68 526 545 16 5.9 169.1 1.0X
+q68 508 545 48 6.1 163.3 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q69 1772 1813 58 1.2 855.8 1.0X
+q69 1734 1799 92 1.2 837.1 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q70 668 679 10 4.4 226.3 1.0X
+q70 671 678 8 4.4 227.1 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q71 412 431 17 12.7 79.0 1.0X
+q71 426 440 16 12.2 81.7 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q72 118708 121669 4188 0.1 7734.4 1.0X
+q72 108371 110305 2735 0.1 7061.0 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q73 410 432 19 7.5 134.0 1.0X
+q73 393 400 9 7.8 128.5 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q74 1615 1807 271 2.3 428.2 1.0X
+q74 1490 1742 357 2.5 394.9 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q75 1702 1934 328 3.3 302.1 1.0X
+q75 1696 1895 282 3.3 301.0 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q76 344 361 16 14.9 67.1 1.0X
+q76 329 346 15 15.6 64.2 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q77 1025 1076 71 5.5 182.6 1.0X
+q77 581 742 222 9.7 103.4 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q78 2602 2762 226 2.2 463.4 1.0X
+q78 2032 2334 427 2.8 361.9 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q79 460 468 6 6.6 150.5 1.0X
+q79 408 416 5 7.5 133.2 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q80 1722 1769 66 3.3 305.0 1.0X
+q80 1592 1598 9 3.5 281.9 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q81 412 456 47 0.9 1124.0 1.0X
+q81 379 431 61 1.0 1034.2 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q82 1181 1197 24 12.5 80.2 1.0X
+q82 1185 1185 0 12.4 80.5 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q83 316 336 17 1.9 531.0 1.0X
+q83 305 343 31 2.0 512.4 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q84 776 780 6 3.1 327.7 1.0X
+q84 777 796 28 3.0 328.3 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q85 2164 2318 218 1.3 763.4 1.0X
+q85 1792 2035 343 1.6 632.2 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q86 217 242 19 3.7 267.9 1.0X
+q86 211 234 17 3.8 260.5 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q87 802 850 48 6.5 153.9 1.0X
+q87 949 1160 298 5.5 182.1 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q88 1373 1670 420 2.2 461.6 1.0X
+q88 1194 1352 223 2.5 401.5 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q89 364 421 51 8.2 122.5 1.0X
+q89 352 427 66 8.4 118.6 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q90 138 163 20 5.9 170.2 1.0X
+q90 121 132 9 6.7 148.7 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q91 350 395 41 6.6 152.3 1.0X
+q91 320 356 44 7.2 139.4 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q92 138 160 20 5.9 170.1 1.0X
+q92 130 149 14 6.2 160.1 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q93 493 522 22 6.4 155.6 1.0X
+q93 451 494 51 7.0 142.3 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q94 384 410 33 2.2 456.3 1.0X
+q94 354 382 26 2.4 420.9 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q95 5272 5368 134 0.2 6261.8 1.0X
+q95 5246 5366 169 0.2 6230.7 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q96 184 211 28 16.2 61.7 1.0X
+q96 180 200 18 16.5 60.5 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q97 1291 1397 149 3.4 294.0 1.0X
+q97 1207 1268 86 3.6 274.8 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q98 323 345 22 9.2 108.8 1.0X
+q98 278 301 32 10.7 93.5 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q99 315 329 14 4.8 208.1 1.0X
+q99 275 302 17 5.5 182.0 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q5a-v2.7 1597 1683 121 3.5 283.8 1.0X
+q5a-v2.7 1579 1853 387 3.6 280.7 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q6-v2.7 1047 1108 86 3.0 335.6 1.0X
+q6-v2.7 1003 1003 0 3.1 321.4 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q10a-v2.7 1788 1920 186 1.2 863.6 1.0X
+q10a-v2.7 1797 1879 116 1.2 867.6 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q11-v2.7 1888 2202 443 2.0 500.7 1.0X
+q11-v2.7 1834 2280 630 2.1 486.3 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q12-v2.7 141 161 17 5.8 173.7 1.0X
+q12-v2.7 157 184 20 5.2 193.3 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q14-v2.7 3971 4128 222 1.3 774.1 1.0X
+q14-v2.7 3900 4113 302 1.3 760.2 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q14a-v2.7 7072 7411 480 0.7 1378.5 1.0X
+q14a-v2.7 8068 8102 48 0.6 1572.9 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q18a-v2.7 1893 2189 419 1.9 525.6 1.0X
+q18a-v2.7 1755 2323 803 2.1 487.1 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q20-v2.7 159 178 14 9.6 104.1 1.0X
+q20-v2.7 162 173 6 9.4 106.1 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q22-v2.7 14036 14248 299 0.8 1185.9 1.0X
+q22-v2.7 14196 14538 483 0.8 1199.4 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q22a-v2.7 2096 2192 137 5.6 177.1 1.0X
+q22a-v2.7 2107 2195 124 5.6 178.0 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q24-v2.7 252 272 16 13.3 75.4 1.0X
+q24-v2.7 203 233 18 16.4 60.8 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q27a-v2.7 1357 1569 299 3.6 277.4 1.0X
+q27a-v2.7 1275 1711 617 3.8 260.7 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q34-v2.7 405 425 24 7.6 132.3 1.0X
+q34-v2.7 350 370 19 8.7 114.4 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q35-v2.7 1343 1424 115 1.5 648.5 1.0X
+q35-v2.7 1276 1323 66 1.6 616.3 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q35a-v2.7 1343 1429 122 1.5 648.4 1.0X
+q35a-v2.7 1234 1348 161 1.7 595.8 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q36a-v2.7 542 570 30 5.5 182.3 1.0X
+q36a-v2.7 557 588 26 5.3 187.3 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q47-v2.7 1533 1731 279 1.9 516.1 1.0X
+q47-v2.7 1756 1929 244 1.7 591.2 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q49-v2.7 627 644 31 9.0 111.6 1.0X
+q49-v2.7 573 646 95 9.8 102.0 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q51a-v2.7 14685 14924 338 0.3 3999.5 1.0X
+q51a-v2.7 14343 14818 673 0.3 3906.2 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q57-v2.7 786 837 44 1.9 513.5 1.0X
+q57-v2.7 803 861 99 1.9 524.6 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q64-v2.7 2399 2642 343 2.9 346.6 1.0X
+q64-v2.7 2618 3021 571 2.6 378.2 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q67a-v2.7 7749 7811 87 0.4 2608.4 1.0X
+q67a-v2.7 7111 7314 287 0.4 2393.6 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q70a-v2.7 696 704 8 4.2 235.8 1.0X
+q70a-v2.7 739 750 13 4.0 250.4 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q72-v2.7 97059 97107 67 0.2 6323.9 1.0X
+q72-v2.7 115607 117956 3321 0.1 7532.4 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q74-v2.7 1404 1675 383 2.7 372.2 1.0X
+q74-v2.7 1551 1714 230 2.4 411.3 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q75-v2.7 1434 1878 628 3.9 254.6 1.0X
+q75-v2.7 1705 1883 252 3.3 302.7 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q77a-v2.7 816 1004 283 6.9 145.2 1.0X
+q77a-v2.7 1262 1277 21 4.5 224.7 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q78-v2.7 2024 2365 482 2.8 360.5 1.0X
+q78-v2.7 2239 2684 630 2.5 398.8 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q80a-v2.7 1822 1984 230 3.1 322.6 1.0X
+q80a-v2.7 2166 2242 108 2.6 383.6 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q86a-v2.7 268 296 20 3.0 331.4 1.0X
+q86a-v2.7 285 311 19 2.8 352.0 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TPCDS: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-q98-v2.7 291 312 19 10.2 98.1 1.0X
+q98-v2.7 288 333 40 10.3 97.1 1.0X
diff --git a/sql/core/benchmarks/TakeOrderedAndProjectBenchmark-jdk21-results.txt b/sql/core/benchmarks/TakeOrderedAndProjectBenchmark-jdk21-results.txt
index 01299cee6fb90..6e3e10b804e81 100644
--- a/sql/core/benchmarks/TakeOrderedAndProjectBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/TakeOrderedAndProjectBenchmark-jdk21-results.txt
@@ -2,11 +2,11 @@
TakeOrderedAndProject
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TakeOrderedAndProject with SMJ: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------
-TakeOrderedAndProject with SMJ for doExecute 173 201 30 0.1 17333.6 1.0X
-TakeOrderedAndProject with SMJ for executeCollect 112 117 5 0.1 11175.1 1.6X
+TakeOrderedAndProject with SMJ for doExecute 188 208 18 0.1 18765.1 1.0X
+TakeOrderedAndProject with SMJ for executeCollect 95 104 12 0.1 9544.5 2.0X
diff --git a/sql/core/benchmarks/TakeOrderedAndProjectBenchmark-results.txt b/sql/core/benchmarks/TakeOrderedAndProjectBenchmark-results.txt
index efc97286cf50c..a641d73a1e043 100644
--- a/sql/core/benchmarks/TakeOrderedAndProjectBenchmark-results.txt
+++ b/sql/core/benchmarks/TakeOrderedAndProjectBenchmark-results.txt
@@ -2,11 +2,11 @@
TakeOrderedAndProject
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
TakeOrderedAndProject with SMJ: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------
-TakeOrderedAndProject with SMJ for doExecute 206 212 5 0.0 20631.3 1.0X
-TakeOrderedAndProject with SMJ for executeCollect 111 121 10 0.1 11106.3 1.9X
+TakeOrderedAndProject with SMJ for doExecute 215 216 1 0.0 21490.5 1.0X
+TakeOrderedAndProject with SMJ for executeCollect 105 110 4 0.1 10469.7 2.1X
diff --git a/sql/core/benchmarks/TopKBenchmark-jdk21-results.txt b/sql/core/benchmarks/TopKBenchmark-jdk21-results.txt
index e5c669c53da3d..c15157e9d2362 100644
--- a/sql/core/benchmarks/TopKBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/TopKBenchmark-jdk21-results.txt
@@ -2,21 +2,21 @@
Top-K Computation
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Benchmark Top-K: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------------------------
-ROW_NUMBER (PARTITION: , WindowGroupLimit: false) 8967 9181 174 2.3 427.6 1.0X
-ROW_NUMBER (PARTITION: , WindowGroupLimit: true) 1868 1909 23 11.2 89.1 4.8X
-ROW_NUMBER (PARTITION: PARTITION BY b, WindowGroupLimit: false) 11459 11734 120 1.8 546.4 0.8X
-ROW_NUMBER (PARTITION: PARTITION BY b, WindowGroupLimit: true) 4913 5018 51 4.3 234.3 1.8X
-RANK (PARTITION: , WindowGroupLimit: false) 9519 9767 113 2.2 453.9 0.9X
-RANK (PARTITION: , WindowGroupLimit: true) 1785 1855 48 11.8 85.1 5.0X
-RANK (PARTITION: PARTITION BY b, WindowGroupLimit: false) 12177 12421 96 1.7 580.7 0.7X
-RANK (PARTITION: PARTITION BY b, WindowGroupLimit: true) 4907 5011 56 4.3 234.0 1.8X
-DENSE_RANK (PARTITION: , WindowGroupLimit: false) 9496 9681 150 2.2 452.8 0.9X
-DENSE_RANK (PARTITION: , WindowGroupLimit: true) 1756 1788 32 11.9 83.7 5.1X
-DENSE_RANK (PARTITION: PARTITION BY b, WindowGroupLimit: false) 12084 12385 158 1.7 576.2 0.7X
-DENSE_RANK (PARTITION: PARTITION BY b, WindowGroupLimit: true) 4905 5035 69 4.3 233.9 1.8X
+ROW_NUMBER (PARTITION: , WindowGroupLimit: false) 8842 8986 82 2.4 421.6 1.0X
+ROW_NUMBER (PARTITION: , WindowGroupLimit: true) 1947 1970 12 10.8 92.8 4.5X
+ROW_NUMBER (PARTITION: PARTITION BY b, WindowGroupLimit: false) 11322 11472 78 1.9 539.9 0.8X
+ROW_NUMBER (PARTITION: PARTITION BY b, WindowGroupLimit: true) 4914 5102 88 4.3 234.3 1.8X
+RANK (PARTITION: , WindowGroupLimit: false) 9422 9698 196 2.2 449.3 0.9X
+RANK (PARTITION: , WindowGroupLimit: true) 1720 1752 20 12.2 82.0 5.1X
+RANK (PARTITION: PARTITION BY b, WindowGroupLimit: false) 11841 11996 120 1.8 564.6 0.7X
+RANK (PARTITION: PARTITION BY b, WindowGroupLimit: true) 4790 4991 130 4.4 228.4 1.8X
+DENSE_RANK (PARTITION: , WindowGroupLimit: false) 9185 9356 136 2.3 438.0 1.0X
+DENSE_RANK (PARTITION: , WindowGroupLimit: true) 1756 1917 121 11.9 83.7 5.0X
+DENSE_RANK (PARTITION: PARTITION BY b, WindowGroupLimit: false) 11572 11776 154 1.8 551.8 0.8X
+DENSE_RANK (PARTITION: PARTITION BY b, WindowGroupLimit: true) 4749 4917 118 4.4 226.5 1.9X
diff --git a/sql/core/benchmarks/TopKBenchmark-results.txt b/sql/core/benchmarks/TopKBenchmark-results.txt
index b07a395f9e9ca..3e71741956878 100644
--- a/sql/core/benchmarks/TopKBenchmark-results.txt
+++ b/sql/core/benchmarks/TopKBenchmark-results.txt
@@ -2,21 +2,21 @@
Top-K Computation
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Benchmark Top-K: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------------------------
-ROW_NUMBER (PARTITION: , WindowGroupLimit: false) 9240 9454 213 2.3 440.6 1.0X
-ROW_NUMBER (PARTITION: , WindowGroupLimit: true) 1612 1633 18 13.0 76.8 5.7X
-ROW_NUMBER (PARTITION: PARTITION BY b, WindowGroupLimit: false) 11354 11429 50 1.8 541.4 0.8X
-ROW_NUMBER (PARTITION: PARTITION BY b, WindowGroupLimit: true) 5117 5204 83 4.1 244.0 1.8X
-RANK (PARTITION: , WindowGroupLimit: false) 9589 9766 239 2.2 457.2 1.0X
-RANK (PARTITION: , WindowGroupLimit: true) 1848 1867 17 11.3 88.1 5.0X
-RANK (PARTITION: PARTITION BY b, WindowGroupLimit: false) 11752 11930 211 1.8 560.4 0.8X
-RANK (PARTITION: PARTITION BY b, WindowGroupLimit: true) 5197 5255 32 4.0 247.8 1.8X
-DENSE_RANK (PARTITION: , WindowGroupLimit: false) 9604 9645 50 2.2 457.9 1.0X
-DENSE_RANK (PARTITION: , WindowGroupLimit: true) 1843 1883 39 11.4 87.9 5.0X
-DENSE_RANK (PARTITION: PARTITION BY b, WindowGroupLimit: false) 11681 11993 569 1.8 557.0 0.8X
-DENSE_RANK (PARTITION: PARTITION BY b, WindowGroupLimit: true) 5098 5191 72 4.1 243.1 1.8X
+ROW_NUMBER (PARTITION: , WindowGroupLimit: false) 8866 8994 143 2.4 422.8 1.0X
+ROW_NUMBER (PARTITION: , WindowGroupLimit: true) 1612 1628 18 13.0 76.9 5.5X
+ROW_NUMBER (PARTITION: PARTITION BY b, WindowGroupLimit: false) 11230 11313 118 1.9 535.5 0.8X
+ROW_NUMBER (PARTITION: PARTITION BY b, WindowGroupLimit: true) 4829 4961 94 4.3 230.3 1.8X
+RANK (PARTITION: , WindowGroupLimit: false) 9412 9445 36 2.2 448.8 0.9X
+RANK (PARTITION: , WindowGroupLimit: true) 1835 1976 51 11.4 87.5 4.8X
+RANK (PARTITION: PARTITION BY b, WindowGroupLimit: false) 11637 11818 148 1.8 554.9 0.8X
+RANK (PARTITION: PARTITION BY b, WindowGroupLimit: true) 5005 5048 28 4.2 238.6 1.8X
+DENSE_RANK (PARTITION: , WindowGroupLimit: false) 9288 9331 36 2.3 442.9 1.0X
+DENSE_RANK (PARTITION: , WindowGroupLimit: true) 1808 1973 61 11.6 86.2 4.9X
+DENSE_RANK (PARTITION: PARTITION BY b, WindowGroupLimit: false) 11553 11734 149 1.8 550.9 0.8X
+DENSE_RANK (PARTITION: PARTITION BY b, WindowGroupLimit: true) 4876 5021 81 4.3 232.5 1.8X
diff --git a/sql/core/benchmarks/UDFBenchmark-jdk21-results.txt b/sql/core/benchmarks/UDFBenchmark-jdk21-results.txt
index 8b0870813056c..f0cd4591f60c8 100644
--- a/sql/core/benchmarks/UDFBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/UDFBenchmark-jdk21-results.txt
@@ -2,58 +2,58 @@
UDF with mixed input types
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
long/nullable int/string to string: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------
-long/nullable int/string to string wholestage off 140 164 33 0.7 1402.0 1.0X
-long/nullable int/string to string wholestage on 65 72 8 1.5 650.3 2.2X
+long/nullable int/string to string wholestage off 102 117 20 1.0 1022.8 1.0X
+long/nullable int/string to string wholestage on 56 67 9 1.8 562.4 1.8X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
long/nullable int/string to option: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------
-long/nullable int/string to option wholestage off 56 62 8 1.8 557.2 1.0X
-long/nullable int/string to option wholestage on 39 44 4 2.6 386.4 1.4X
+long/nullable int/string to option wholestage off 41 41 1 2.5 406.0 1.0X
+long/nullable int/string to option wholestage on 38 44 5 2.6 381.5 1.1X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
long/nullable int/string to primitive: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------------
-long/nullable int/string to primitive wholestage off 35 35 0 2.8 352.7 1.0X
-long/nullable int/string to primitive wholestage on 29 32 2 3.4 291.4 1.2X
+long/nullable int/string to primitive wholestage off 32 32 1 3.2 315.4 1.0X
+long/nullable int/string to primitive wholestage on 27 32 5 3.8 265.5 1.2X
================================================================================================
UDF with primitive types
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
long/nullable int to string: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------
-long/nullable int to string wholestage off 31 32 1 3.2 313.6 1.0X
-long/nullable int to string wholestage on 32 36 4 3.2 315.7 1.0X
+long/nullable int to string wholestage off 29 29 1 3.5 286.4 1.0X
+long/nullable int to string wholestage on 29 32 4 3.5 288.9 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
long/nullable int to option: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------
-long/nullable int to option wholestage off 23 24 2 4.4 226.0 1.0X
-long/nullable int to option wholestage on 26 32 6 3.9 258.2 0.9X
+long/nullable int to option wholestage off 21 24 4 4.7 213.9 1.0X
+long/nullable int to option wholestage on 21 23 3 4.8 208.1 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
long/nullable int to primitive: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------
-long/nullable int to primitive wholestage off 19 22 5 5.3 188.3 1.0X
-long/nullable int to primitive wholestage on 18 18 0 5.6 180.2 1.0X
+long/nullable int to primitive wholestage off 17 23 9 6.1 165.1 1.0X
+long/nullable int to primitive wholestage on 17 22 4 5.8 172.9 1.0X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
UDF identity overhead: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Baseline 12 14 2 8.2 122.4 1.0X
-With identity UDF 16 18 2 6.4 157.2 0.8X
+Baseline 12 14 1 8.4 118.8 1.0X
+With identity UDF 14 15 1 7.2 139.5 0.9X
diff --git a/sql/core/benchmarks/UDFBenchmark-results.txt b/sql/core/benchmarks/UDFBenchmark-results.txt
index ca92c3305295f..746babff4bf4f 100644
--- a/sql/core/benchmarks/UDFBenchmark-results.txt
+++ b/sql/core/benchmarks/UDFBenchmark-results.txt
@@ -2,58 +2,58 @@
UDF with mixed input types
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
long/nullable int/string to string: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------
-long/nullable int/string to string wholestage off 118 129 16 0.8 1182.2 1.0X
-long/nullable int/string to string wholestage on 70 88 12 1.4 704.4 1.7X
+long/nullable int/string to string wholestage off 124 149 34 0.8 1243.3 1.0X
+long/nullable int/string to string wholestage on 67 81 9 1.5 667.5 1.9X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
long/nullable int/string to option: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------
-long/nullable int/string to option wholestage off 53 55 3 1.9 531.1 1.0X
-long/nullable int/string to option wholestage on 39 43 4 2.5 392.5 1.4X
+long/nullable int/string to option wholestage off 43 50 9 2.3 431.5 1.0X
+long/nullable int/string to option wholestage on 38 42 3 2.6 378.6 1.1X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
long/nullable int/string to primitive: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------------
-long/nullable int/string to primitive wholestage off 30 32 3 3.4 298.1 1.0X
-long/nullable int/string to primitive wholestage on 30 34 4 3.3 303.9 1.0X
+long/nullable int/string to primitive wholestage off 30 31 1 3.3 304.0 1.0X
+long/nullable int/string to primitive wholestage on 31 37 5 3.2 308.7 1.0X
================================================================================================
UDF with primitive types
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
long/nullable int to string: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------
-long/nullable int to string wholestage off 36 41 8 2.8 355.3 1.0X
-long/nullable int to string wholestage on 31 32 2 3.3 305.2 1.2X
+long/nullable int to string wholestage off 31 31 1 3.3 306.2 1.0X
+long/nullable int to string wholestage on 30 33 2 3.3 303.4 1.0X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
long/nullable int to option: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------
-long/nullable int to option wholestage off 31 32 1 3.2 314.5 1.0X
-long/nullable int to option wholestage on 22 24 2 4.5 220.3 1.4X
+long/nullable int to option wholestage off 32 32 0 3.1 320.4 1.0X
+long/nullable int to option wholestage on 21 25 5 4.7 213.7 1.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
long/nullable int to primitive: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------
-long/nullable int to primitive wholestage off 18 19 1 5.4 183.9 1.0X
-long/nullable int to primitive wholestage on 19 20 0 5.2 193.8 0.9X
+long/nullable int to primitive wholestage off 20 22 2 4.9 202.8 1.0X
+long/nullable int to primitive wholestage on 17 18 1 5.8 172.3 1.2X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
UDF identity overhead: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Baseline 12 13 1 8.3 120.0 1.0X
-With identity UDF 16 25 5 6.1 164.7 0.7X
+Baseline 11 13 3 9.1 109.9 1.0X
+With identity UDF 16 17 1 6.1 164.9 0.7X
diff --git a/sql/core/benchmarks/UnsafeArrayDataBenchmark-jdk21-results.txt b/sql/core/benchmarks/UnsafeArrayDataBenchmark-jdk21-results.txt
index 5f447a39039fa..4b93de5445f14 100644
--- a/sql/core/benchmarks/UnsafeArrayDataBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/UnsafeArrayDataBenchmark-jdk21-results.txt
@@ -2,32 +2,32 @@
Benchmark UnsafeArrayData
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read UnsafeArrayData: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Int 80 80 1 2091.7 0.5 1.0X
+Int 80 80 0 2094.8 0.5 1.0X
Double 159 159 0 1056.0 0.9 0.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write UnsafeArrayData: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Int 12 16 2 1699.1 0.6 1.0X
-Double 28 34 4 738.0 1.4 0.4X
+Int 14 16 3 1545.3 0.6 1.0X
+Double 29 33 3 721.4 1.4 0.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Get primitive array from UnsafeArrayData: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Int 19 21 1 3255.6 0.3 1.0X
-Double 40 43 2 1572.0 0.6 0.5X
+Int 20 22 1 3202.8 0.3 1.0X
+Double 41 45 2 1522.6 0.7 0.5X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Create UnsafeArrayData from primitive array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-Int 20 22 1 3206.7 0.3 1.0X
-Double 42 46 2 1496.8 0.7 0.5X
+Int 21 24 1 2937.5 0.3 1.0X
+Double 45 47 2 1411.0 0.7 0.5X
diff --git a/sql/core/benchmarks/UnsafeArrayDataBenchmark-results.txt b/sql/core/benchmarks/UnsafeArrayDataBenchmark-results.txt
index 3ddb3bdb3f0b4..689931d9b4aa1 100644
--- a/sql/core/benchmarks/UnsafeArrayDataBenchmark-results.txt
+++ b/sql/core/benchmarks/UnsafeArrayDataBenchmark-results.txt
@@ -2,32 +2,32 @@
Benchmark UnsafeArrayData
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Read UnsafeArrayData: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Int 76 77 1 2200.4 0.5 1.0X
-Double 158 159 0 1059.1 0.9 0.5X
+Int 76 76 0 2204.0 0.5 1.0X
+Double 159 159 0 1055.2 0.9 0.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Write UnsafeArrayData: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Int 12 13 1 1777.3 0.6 1.0X
-Double 28 33 4 754.2 1.3 0.4X
+Int 14 15 1 1501.6 0.7 1.0X
+Double 32 38 3 647.1 1.5 0.4X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Get primitive array from UnsafeArrayData: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-Int 19 22 2 3276.8 0.3 1.0X
-Double 41 43 2 1546.7 0.6 0.5X
+Int 21 24 1 2956.5 0.3 1.0X
+Double 45 48 1 1390.6 0.7 0.5X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Create UnsafeArrayData from primitive array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------
-Int 20 23 1 3189.4 0.3 1.0X
-Double 45 47 2 1400.8 0.7 0.4X
+Int 22 25 1 2802.3 0.4 1.0X
+Double 46 49 1 1357.5 0.7 0.5X
diff --git a/sql/core/benchmarks/UpdateFieldsBenchmark-jdk21-results.txt b/sql/core/benchmarks/UpdateFieldsBenchmark-jdk21-results.txt
index fdda101722dfc..1f33913b1038d 100644
--- a/sql/core/benchmarks/UpdateFieldsBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/UpdateFieldsBenchmark-jdk21-results.txt
@@ -2,25 +2,25 @@
Add 2 columns and drop 2 columns at 3 different depths of nesting
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Add 2 columns and drop 2 columns at 3 different depths of nesting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------------
To non-nullable StructTypes using performant method 2 3 1 0.0 Infinity 1.0X
-To nullable StructTypes using performant method 1 1 0 0.0 Infinity 1.4X
-To non-nullable StructTypes using non-performant method 25 27 4 0.0 Infinity 0.1X
-To nullable StructTypes using non-performant method 563 618 41 0.0 Infinity 0.0X
+To nullable StructTypes using performant method 1 2 1 0.0 Infinity 1.4X
+To non-nullable StructTypes using non-performant method 33 36 4 0.0 Infinity 0.1X
+To nullable StructTypes using non-performant method 723 738 18 0.0 Infinity 0.0X
================================================================================================
Add 50 columns and drop 50 columns at 100 different depths of nesting
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Add 50 columns and drop 50 columns at 100 different depths of nesting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------------------------------
-To non-nullable StructTypes using performant method 426 430 6 0.0 Infinity 1.0X
-To nullable StructTypes using performant method 455 471 20 0.0 Infinity 0.9X
+To non-nullable StructTypes using performant method 714 731 20 0.0 Infinity 1.0X
+To nullable StructTypes using performant method 731 762 28 0.0 Infinity 1.0X
diff --git a/sql/core/benchmarks/UpdateFieldsBenchmark-results.txt b/sql/core/benchmarks/UpdateFieldsBenchmark-results.txt
index e6f50a6b410fa..f716aef1f4d52 100644
--- a/sql/core/benchmarks/UpdateFieldsBenchmark-results.txt
+++ b/sql/core/benchmarks/UpdateFieldsBenchmark-results.txt
@@ -2,25 +2,25 @@
Add 2 columns and drop 2 columns at 3 different depths of nesting
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Add 2 columns and drop 2 columns at 3 different depths of nesting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------------
To non-nullable StructTypes using performant method 2 3 1 0.0 Infinity 1.0X
-To nullable StructTypes using performant method 1 2 0 0.0 Infinity 1.4X
-To non-nullable StructTypes using non-performant method 27 28 2 0.0 Infinity 0.1X
-To nullable StructTypes using non-performant method 632 671 41 0.0 Infinity 0.0X
+To nullable StructTypes using performant method 2 2 0 0.0 Infinity 1.3X
+To non-nullable StructTypes using non-performant method 43 45 3 0.0 Infinity 0.0X
+To nullable StructTypes using non-performant method 762 789 23 0.0 Infinity 0.0X
================================================================================================
Add 50 columns and drop 50 columns at 100 different depths of nesting
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
Add 50 columns and drop 50 columns at 100 different depths of nesting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-----------------------------------------------------------------------------------------------------------------------------------------------------
-To non-nullable StructTypes using performant method 523 528 4 0.0 Infinity 1.0X
-To nullable StructTypes using performant method 558 560 1 0.0 Infinity 0.9X
+To non-nullable StructTypes using performant method 1058 1061 5 0.0 Infinity 1.0X
+To nullable StructTypes using performant method 1055 1061 8 0.0 Infinity 1.0X
diff --git a/sql/core/benchmarks/V2FunctionBenchmark-jdk21-results.txt b/sql/core/benchmarks/V2FunctionBenchmark-jdk21-results.txt
index 5057d398891b0..dc84a0f3a54a6 100644
--- a/sql/core/benchmarks/V2FunctionBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/V2FunctionBenchmark-jdk21-results.txt
@@ -1,44 +1,44 @@
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
scalar function (long + long) -> long, result_nullable = true codegen = true: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------------------------------------
-native_long_add 10089 10493 661 49.6 20.2 1.0X
-java_long_add_default 22867 22890 37 21.9 45.7 0.4X
-java_long_add_magic 11600 11644 57 43.1 23.2 0.9X
-java_long_add_static_magic 11562 11720 228 43.2 23.1 0.9X
-scala_long_add_default 23965 24147 265 20.9 47.9 0.4X
-scala_long_add_magic 11591 11820 356 43.1 23.2 0.9X
+native_long_add 9835 10320 826 50.8 19.7 1.0X
+java_long_add_default 21945 22529 922 22.8 43.9 0.4X
+java_long_add_magic 11566 11633 112 43.2 23.1 0.9X
+java_long_add_static_magic 11509 11536 25 43.4 23.0 0.9X
+scala_long_add_default 23210 23294 133 21.5 46.4 0.4X
+scala_long_add_magic 11543 11601 99 43.3 23.1 0.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
scalar function (long + long) -> long, result_nullable = false codegen = true: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------------------------
-native_long_add 10907 11657 732 45.8 21.8 1.0X
-java_long_add_default 22513 22805 495 22.2 45.0 0.5X
-java_long_add_magic 12082 12151 67 41.4 24.2 0.9X
-java_long_add_static_magic 10022 10193 174 49.9 20.0 1.1X
-scala_long_add_default 22328 22403 83 22.4 44.7 0.5X
-scala_long_add_magic 11624 11873 229 43.0 23.2 0.9X
+native_long_add 10152 10364 183 49.3 20.3 1.0X
+java_long_add_default 22197 22314 193 22.5 44.4 0.5X
+java_long_add_magic 11567 11620 60 43.2 23.1 0.9X
+java_long_add_static_magic 9994 10035 37 50.0 20.0 1.0X
+scala_long_add_default 22078 22110 30 22.6 44.2 0.5X
+scala_long_add_magic 11575 11702 130 43.2 23.1 0.9X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
scalar function (long + long) -> long, result_nullable = true codegen = false: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------------------------
-native_long_add 22932 23181 407 21.8 45.9 1.0X
-java_long_add_default 26925 27143 353 18.6 53.8 0.9X
-java_long_add_magic 32820 32934 125 15.2 65.6 0.7X
-java_long_add_static_magic 30895 31058 156 16.2 61.8 0.7X
-scala_long_add_default 26669 26864 237 18.7 53.3 0.9X
-scala_long_add_magic 32976 33334 407 15.2 66.0 0.7X
+native_long_add 22677 22692 18 22.0 45.4 1.0X
+java_long_add_default 26749 26801 52 18.7 53.5 0.8X
+java_long_add_magic 32489 32582 100 15.4 65.0 0.7X
+java_long_add_static_magic 30791 30824 37 16.2 61.6 0.7X
+scala_long_add_default 26460 26510 44 18.9 52.9 0.9X
+scala_long_add_magic 32668 32724 50 15.3 65.3 0.7X
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
scalar function (long + long) -> long, result_nullable = false codegen = false: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------------------------------
-native_long_add 21803 21918 134 22.9 43.6 1.0X
-java_long_add_default 25534 25962 715 19.6 51.1 0.9X
-java_long_add_magic 32086 32180 123 15.6 64.2 0.7X
-java_long_add_static_magic 29945 30086 149 16.7 59.9 0.7X
-scala_long_add_default 25475 25523 42 19.6 51.0 0.9X
-scala_long_add_magic 32125 32175 44 15.6 64.3 0.7X
+native_long_add 22890 23100 291 21.8 45.8 1.0X
+java_long_add_default 26587 26639 45 18.8 53.2 0.9X
+java_long_add_magic 32885 32954 65 15.2 65.8 0.7X
+java_long_add_static_magic 31021 31268 224 16.1 62.0 0.7X
+scala_long_add_default 26612 26893 261 18.8 53.2 0.9X
+scala_long_add_magic 33112 33351 339 15.1 66.2 0.7X
diff --git a/sql/core/benchmarks/V2FunctionBenchmark-results.txt b/sql/core/benchmarks/V2FunctionBenchmark-results.txt
index 81726a34bcbba..966b8434ddb08 100644
--- a/sql/core/benchmarks/V2FunctionBenchmark-results.txt
+++ b/sql/core/benchmarks/V2FunctionBenchmark-results.txt
@@ -1,44 +1,44 @@
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
scalar function (long + long) -> long, result_nullable = true codegen = true: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------------------------------------
-native_long_add 9267 9290 37 54.0 18.5 1.0X
-java_long_add_default 22321 22462 232 22.4 44.6 0.4X
-java_long_add_magic 10925 10945 18 45.8 21.8 0.8X
-java_long_add_static_magic 11068 11192 142 45.2 22.1 0.8X
-scala_long_add_default 22271 22289 16 22.5 44.5 0.4X
-scala_long_add_magic 10930 10998 66 45.7 21.9 0.8X
+native_long_add 9191 9247 50 54.4 18.4 1.0X
+java_long_add_default 22939 23001 57 21.8 45.9 0.4X
+java_long_add_magic 10694 10751 81 46.8 21.4 0.9X
+java_long_add_static_magic 10567 11615 998 47.3 21.1 0.9X
+scala_long_add_default 23064 23144 104 21.7 46.1 0.4X
+scala_long_add_magic 10726 10797 62 46.6 21.5 0.9X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
scalar function (long + long) -> long, result_nullable = false codegen = true: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------------------------
-native_long_add 9944 10134 286 50.3 19.9 1.0X
-java_long_add_default 21517 21538 19 23.2 43.0 0.5X
-java_long_add_magic 11487 11535 59 43.5 23.0 0.9X
-java_long_add_static_magic 10276 10343 64 48.7 20.6 1.0X
-scala_long_add_default 21555 21610 51 23.2 43.1 0.5X
-scala_long_add_magic 11027 11154 187 45.3 22.1 0.9X
+native_long_add 10042 10098 93 49.8 20.1 1.0X
+java_long_add_default 22278 22326 64 22.4 44.6 0.5X
+java_long_add_magic 10647 10660 13 47.0 21.3 0.9X
+java_long_add_static_magic 9944 10014 88 50.3 19.9 1.0X
+scala_long_add_default 22176 22516 586 22.5 44.4 0.5X
+scala_long_add_magic 10690 10745 79 46.8 21.4 0.9X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
scalar function (long + long) -> long, result_nullable = true codegen = false: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------------------------------------------
-native_long_add 22916 23049 122 21.8 45.8 1.0X
-java_long_add_default 28358 28408 80 17.6 56.7 0.8X
-java_long_add_magic 31588 31709 134 15.8 63.2 0.7X
-java_long_add_static_magic 31198 31452 337 16.0 62.4 0.7X
-scala_long_add_default 26362 26390 36 19.0 52.7 0.9X
-scala_long_add_magic 31958 32000 37 15.6 63.9 0.7X
+native_long_add 22784 22953 185 21.9 45.6 1.0X
+java_long_add_default 26064 26088 38 19.2 52.1 0.9X
+java_long_add_magic 32566 32610 76 15.4 65.1 0.7X
+java_long_add_static_magic 32014 32099 81 15.6 64.0 0.7X
+scala_long_add_default 27099 27195 92 18.5 54.2 0.8X
+scala_long_add_magic 32697 33739 915 15.3 65.4 0.7X
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
scalar function (long + long) -> long, result_nullable = false codegen = false: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------------------------------
-native_long_add 22121 22162 68 22.6 44.2 1.0X
-java_long_add_default 25925 26037 98 19.3 51.8 0.9X
-java_long_add_magic 31642 31668 24 15.8 63.3 0.7X
-java_long_add_static_magic 30902 30948 55 16.2 61.8 0.7X
-scala_long_add_default 25906 25930 25 19.3 51.8 0.9X
-scala_long_add_magic 31693 32012 374 15.8 63.4 0.7X
+native_long_add 21805 21817 10 22.9 43.6 1.0X
+java_long_add_default 25863 26093 239 19.3 51.7 0.8X
+java_long_add_magic 31630 31668 55 15.8 63.3 0.7X
+java_long_add_static_magic 31125 31300 263 16.1 62.3 0.7X
+scala_long_add_default 25823 26135 463 19.4 51.6 0.8X
+scala_long_add_magic 31689 31881 330 15.8 63.4 0.7X
diff --git a/sql/core/benchmarks/WideSchemaBenchmark-jdk21-results.txt b/sql/core/benchmarks/WideSchemaBenchmark-jdk21-results.txt
index 534aa70e59af2..9963575f763ff 100644
--- a/sql/core/benchmarks/WideSchemaBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/WideSchemaBenchmark-jdk21-results.txt
@@ -2,157 +2,157 @@
parsing large select expressions
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
parsing large select: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-1 select expressions 1 1 0 0.0 671823.0 1.0X
-100 select expressions 3 3 0 0.0 3220274.0 0.2X
-2500 select expressions 70 72 2 0.0 69682180.0 0.0X
+1 select expressions 1 1 0 0.0 632990.0 1.0X
+100 select expressions 3 3 0 0.0 2850444.0 0.2X
+2500 select expressions 63 65 3 0.0 62915152.0 0.0X
================================================================================================
optimize large select expressions
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
optimize large select: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-100 columns 3 4 1 0.0 2961331.0 1.0X
-1000 columns 16 18 3 0.0 15853499.0 0.2X
-10000 columns 150 170 14 0.0 150229609.0 0.0X
+100 columns 3 4 1 0.0 2952203.0 1.0X
+1000 columns 17 18 1 0.0 17106048.0 0.2X
+10000 columns 162 177 18 0.0 162356332.0 0.0X
================================================================================================
many column field read and write
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
many column field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-1 cols x 100000 rows (read in-mem) 14 21 4 6.9 144.2 1.0X
-1 cols x 100000 rows (exec in-mem) 15 18 4 6.9 145.6 1.0X
-1 cols x 100000 rows (read parquet) 27 37 10 3.7 272.7 0.5X
-1 cols x 100000 rows (write parquet) 88 97 11 1.1 878.8 0.2X
-100 cols x 1000 rows (read in-mem) 11 15 4 9.0 111.2 1.3X
-100 cols x 1000 rows (exec in-mem) 13 18 5 7.4 134.6 1.1X
-100 cols x 1000 rows (read parquet) 23 28 8 4.4 228.8 0.6X
-100 cols x 1000 rows (write parquet) 85 92 11 1.2 845.1 0.2X
-2500 cols x 40 rows (read in-mem) 44 51 10 2.3 435.1 0.3X
-2500 cols x 40 rows (exec in-mem) 69 85 21 1.4 692.4 0.2X
-2500 cols x 40 rows (read parquet) 263 268 6 0.4 2627.9 0.1X
-2500 cols x 40 rows (write parquet) 119 136 17 0.8 1193.7 0.1X
+1 cols x 100000 rows (read in-mem) 17 23 5 5.9 168.2 1.0X
+1 cols x 100000 rows (exec in-mem) 16 21 4 6.3 159.8 1.1X
+1 cols x 100000 rows (read parquet) 29 36 6 3.4 293.5 0.6X
+1 cols x 100000 rows (write parquet) 92 97 6 1.1 915.5 0.2X
+100 cols x 1000 rows (read in-mem) 13 17 4 8.0 125.5 1.3X
+100 cols x 1000 rows (exec in-mem) 15 18 4 6.7 149.0 1.1X
+100 cols x 1000 rows (read parquet) 24 29 6 4.1 244.8 0.7X
+100 cols x 1000 rows (write parquet) 87 95 8 1.1 870.1 0.2X
+2500 cols x 40 rows (read in-mem) 47 49 4 2.1 472.7 0.4X
+2500 cols x 40 rows (exec in-mem) 74 79 9 1.3 741.1 0.2X
+2500 cols x 40 rows (read parquet) 316 326 7 0.3 3160.6 0.1X
+2500 cols x 40 rows (write parquet) 122 131 9 0.8 1220.3 0.1X
================================================================================================
wide shallowly nested struct field read and write
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
wide shallowly nested struct field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-1 wide x 100000 rows (read in-mem) 20 24 6 5.0 198.2 1.0X
-1 wide x 100000 rows (exec in-mem) 21 26 6 4.7 213.3 0.9X
-1 wide x 100000 rows (read parquet) 22 26 6 4.6 217.9 0.9X
-1 wide x 100000 rows (write parquet) 94 102 11 1.1 944.8 0.2X
-100 wide x 1000 rows (read in-mem) 15 19 6 6.6 150.9 1.3X
-100 wide x 1000 rows (exec in-mem) 16 18 4 6.4 157.4 1.3X
-100 wide x 1000 rows (read parquet) 22 25 5 4.6 215.3 0.9X
-100 wide x 1000 rows (write parquet) 89 103 11 1.1 889.6 0.2X
-2500 wide x 40 rows (read in-mem) 21 23 4 4.9 205.9 1.0X
-2500 wide x 40 rows (exec in-mem) 22 25 5 4.5 219.9 0.9X
-2500 wide x 40 rows (read parquet) 65 69 6 1.5 647.1 0.3X
-2500 wide x 40 rows (write parquet) 94 110 7 1.1 944.7 0.2X
+1 wide x 100000 rows (read in-mem) 21 28 7 4.8 208.2 1.0X
+1 wide x 100000 rows (exec in-mem) 22 27 5 4.5 224.1 0.9X
+1 wide x 100000 rows (read parquet) 23 29 8 4.3 230.1 0.9X
+1 wide x 100000 rows (write parquet) 96 104 8 1.0 962.3 0.2X
+100 wide x 1000 rows (read in-mem) 16 20 6 6.4 157.3 1.3X
+100 wide x 1000 rows (exec in-mem) 17 21 6 6.0 166.2 1.3X
+100 wide x 1000 rows (read parquet) 22 26 5 4.5 223.2 0.9X
+100 wide x 1000 rows (write parquet) 90 99 12 1.1 896.8 0.2X
+2500 wide x 40 rows (read in-mem) 21 24 5 4.8 209.2 1.0X
+2500 wide x 40 rows (exec in-mem) 22 27 6 4.5 223.7 0.9X
+2500 wide x 40 rows (read parquet) 65 70 6 1.5 654.5 0.3X
+2500 wide x 40 rows (write parquet) 96 102 9 1.0 957.8 0.2X
================================================================================================
deeply nested struct field read and write
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
deeply nested struct field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-1 deep x 100000 rows (read in-mem) 16 19 5 6.4 155.5 1.0X
-1 deep x 100000 rows (exec in-mem) 18 21 5 5.6 177.4 0.9X
-1 deep x 100000 rows (read parquet) 18 21 5 5.5 182.1 0.9X
-1 deep x 100000 rows (write parquet) 90 94 3 1.1 899.4 0.2X
-100 deep x 1000 rows (read in-mem) 45 47 3 2.2 448.2 0.3X
-100 deep x 1000 rows (exec in-mem) 455 463 6 0.2 4548.0 0.0X
-100 deep x 1000 rows (read parquet) 462 471 6 0.2 4618.6 0.0X
-100 deep x 1000 rows (write parquet) 127 137 6 0.8 1271.2 0.1X
-250 deep x 400 rows (read in-mem) 194 197 2 0.5 1940.9 0.1X
-250 deep x 400 rows (exec in-mem) 2758 2764 9 0.0 27576.6 0.0X
-250 deep x 400 rows (read parquet) 2752 2754 2 0.0 27523.9 0.0X
-250 deep x 400 rows (write parquet) 268 274 5 0.4 2676.3 0.1X
+1 deep x 100000 rows (read in-mem) 16 21 6 6.2 160.4 1.0X
+1 deep x 100000 rows (exec in-mem) 18 23 6 5.5 183.0 0.9X
+1 deep x 100000 rows (read parquet) 18 22 5 5.4 184.9 0.9X
+1 deep x 100000 rows (write parquet) 90 98 7 1.1 902.9 0.2X
+100 deep x 1000 rows (read in-mem) 44 47 4 2.3 440.2 0.4X
+100 deep x 1000 rows (exec in-mem) 469 475 7 0.2 4690.1 0.0X
+100 deep x 1000 rows (read parquet) 476 478 2 0.2 4761.2 0.0X
+100 deep x 1000 rows (write parquet) 119 124 4 0.8 1191.8 0.1X
+250 deep x 400 rows (read in-mem) 192 196 4 0.5 1918.4 0.1X
+250 deep x 400 rows (exec in-mem) 2876 2879 5 0.0 28758.0 0.0X
+250 deep x 400 rows (read parquet) 2853 2856 4 0.0 28534.9 0.0X
+250 deep x 400 rows (write parquet) 265 274 7 0.4 2651.7 0.1X
================================================================================================
bushy struct field read and write
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
bushy struct field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
-1 x 1 deep x 100000 rows (read in-mem) 13 15 4 7.6 131.1 1.0X
-1 x 1 deep x 100000 rows (exec in-mem) 15 17 4 6.6 151.7 0.9X
-1 x 1 deep x 100000 rows (read parquet) 18 20 5 5.7 175.1 0.7X
-1 x 1 deep x 100000 rows (write parquet) 87 91 4 1.1 873.2 0.2X
-128 x 8 deep x 1000 rows (read in-mem) 14 16 4 7.4 135.3 1.0X
-128 x 8 deep x 1000 rows (exec in-mem) 15 17 4 6.8 146.2 0.9X
-128 x 8 deep x 1000 rows (read parquet) 21 23 5 4.8 208.9 0.6X
-128 x 8 deep x 1000 rows (write parquet) 87 94 7 1.2 868.4 0.2X
-1024 x 11 deep x 100 rows (read in-mem) 16 19 4 6.1 164.9 0.8X
-1024 x 11 deep x 100 rows (exec in-mem) 18 20 4 5.6 180.1 0.7X
-1024 x 11 deep x 100 rows (read parquet) 34 38 5 2.9 342.1 0.4X
-1024 x 11 deep x 100 rows (write parquet) 91 101 9 1.1 906.6 0.1X
+1 x 1 deep x 100000 rows (read in-mem) 14 16 4 7.3 137.7 1.0X
+1 x 1 deep x 100000 rows (exec in-mem) 15 18 4 6.5 153.5 0.9X
+1 x 1 deep x 100000 rows (read parquet) 18 20 5 5.5 180.9 0.8X
+1 x 1 deep x 100000 rows (write parquet) 89 94 8 1.1 887.7 0.2X
+128 x 8 deep x 1000 rows (read in-mem) 13 16 5 7.5 134.0 1.0X
+128 x 8 deep x 1000 rows (exec in-mem) 15 17 4 6.8 146.1 0.9X
+128 x 8 deep x 1000 rows (read parquet) 21 25 5 4.7 212.1 0.6X
+128 x 8 deep x 1000 rows (write parquet) 87 92 7 1.1 873.3 0.2X
+1024 x 11 deep x 100 rows (read in-mem) 17 20 5 5.8 172.0 0.8X
+1024 x 11 deep x 100 rows (exec in-mem) 19 24 6 5.4 186.9 0.7X
+1024 x 11 deep x 100 rows (read parquet) 35 39 6 2.9 349.6 0.4X
+1024 x 11 deep x 100 rows (write parquet) 91 97 7 1.1 909.3 0.2X
================================================================================================
wide array field read and write
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
wide array field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-1 wide x 100000 rows (read in-mem) 15 17 4 6.7 150.3 1.0X
-1 wide x 100000 rows (exec in-mem) 17 20 4 5.8 172.0 0.9X
-1 wide x 100000 rows (read parquet) 18 20 4 5.6 177.3 0.8X
-1 wide x 100000 rows (write parquet) 89 97 7 1.1 889.4 0.2X
-100 wide x 1000 rows (read in-mem) 11 13 4 8.9 112.0 1.3X
-100 wide x 1000 rows (exec in-mem) 13 15 4 7.8 128.7 1.2X
-100 wide x 1000 rows (read parquet) 17 20 5 5.8 173.5 0.9X
-100 wide x 1000 rows (write parquet) 85 95 7 1.2 850.1 0.2X
-2500 wide x 40 rows (read in-mem) 11 13 4 9.2 109.1 1.4X
-2500 wide x 40 rows (exec in-mem) 12 15 4 8.0 124.8 1.2X
-2500 wide x 40 rows (read parquet) 17 19 4 5.9 170.3 0.9X
-2500 wide x 40 rows (write parquet) 84 89 4 1.2 844.3 0.2X
+1 wide x 100000 rows (read in-mem) 15 18 4 6.7 149.3 1.0X
+1 wide x 100000 rows (exec in-mem) 17 19 4 5.9 170.3 0.9X
+1 wide x 100000 rows (read parquet) 18 20 5 5.7 176.6 0.8X
+1 wide x 100000 rows (write parquet) 88 94 7 1.1 884.9 0.2X
+100 wide x 1000 rows (read in-mem) 11 14 4 8.8 113.4 1.3X
+100 wide x 1000 rows (exec in-mem) 13 16 4 7.7 129.4 1.2X
+100 wide x 1000 rows (read parquet) 17 20 6 5.8 173.6 0.9X
+100 wide x 1000 rows (write parquet) 85 90 7 1.2 847.2 0.2X
+2500 wide x 40 rows (read in-mem) 11 12 3 9.1 109.8 1.4X
+2500 wide x 40 rows (exec in-mem) 12 14 4 8.1 124.2 1.2X
+2500 wide x 40 rows (read parquet) 17 19 5 5.9 170.5 0.9X
+2500 wide x 40 rows (write parquet) 84 91 6 1.2 841.5 0.2X
================================================================================================
wide map field read and write
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
wide map field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-1 wide x 100000 rows (read in-mem) 12 14 2 8.0 124.5 1.0X
-1 wide x 100000 rows (exec in-mem) 16 19 3 6.1 164.2 0.8X
-1 wide x 100000 rows (read parquet) 21 24 5 4.7 213.1 0.6X
-1 wide x 100000 rows (write parquet) 86 92 6 1.2 855.4 0.1X
-100 wide x 1000 rows (read in-mem) 8 9 2 13.0 76.7 1.6X
-100 wide x 1000 rows (exec in-mem) 10 11 2 10.3 97.0 1.3X
-100 wide x 1000 rows (read parquet) 19 22 5 5.3 187.1 0.7X
-100 wide x 1000 rows (write parquet) 81 87 6 1.2 806.6 0.2X
-2500 wide x 40 rows (read in-mem) 10 11 3 10.5 95.7 1.3X
-2500 wide x 40 rows (exec in-mem) 11 13 3 8.8 113.7 1.1X
-2500 wide x 40 rows (read parquet) 19 21 5 5.4 186.2 0.7X
-2500 wide x 40 rows (write parquet) 83 89 6 1.2 826.0 0.2X
+1 wide x 100000 rows (read in-mem) 12 14 3 8.1 123.4 1.0X
+1 wide x 100000 rows (exec in-mem) 16 18 3 6.2 160.2 0.8X
+1 wide x 100000 rows (read parquet) 21 25 5 4.7 213.2 0.6X
+1 wide x 100000 rows (write parquet) 85 90 6 1.2 850.3 0.1X
+100 wide x 1000 rows (read in-mem) 7 9 3 13.6 73.6 1.7X
+100 wide x 1000 rows (exec in-mem) 9 11 3 10.6 94.0 1.3X
+100 wide x 1000 rows (read parquet) 18 22 6 5.4 184.6 0.7X
+100 wide x 1000 rows (write parquet) 80 84 5 1.2 802.4 0.2X
+2500 wide x 40 rows (read in-mem) 9 10 3 11.1 90.0 1.4X
+2500 wide x 40 rows (exec in-mem) 11 12 3 9.1 109.6 1.1X
+2500 wide x 40 rows (read parquet) 19 21 4 5.3 189.2 0.7X
+2500 wide x 40 rows (write parquet) 82 88 8 1.2 818.2 0.2X
diff --git a/sql/core/benchmarks/WideSchemaBenchmark-results.txt b/sql/core/benchmarks/WideSchemaBenchmark-results.txt
index fe8a25b6e918d..d2d2a4b9f084d 100644
--- a/sql/core/benchmarks/WideSchemaBenchmark-results.txt
+++ b/sql/core/benchmarks/WideSchemaBenchmark-results.txt
@@ -2,157 +2,157 @@
parsing large select expressions
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
parsing large select: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-1 select expressions 1 1 0 0.0 641958.0 1.0X
-100 select expressions 3 3 1 0.0 3194529.0 0.2X
-2500 select expressions 70 71 2 0.0 69525822.0 0.0X
+1 select expressions 1 1 0 0.0 650133.0 1.0X
+100 select expressions 3 3 0 0.0 3239833.0 0.2X
+2500 select expressions 71 73 2 0.0 70970829.0 0.0X
================================================================================================
optimize large select expressions
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
optimize large select: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-100 columns 3 4 1 0.0 2941387.0 1.0X
-1000 columns 16 18 3 0.0 16490130.0 0.2X
-10000 columns 152 165 17 0.0 151554960.0 0.0X
+100 columns 3 4 1 0.0 3227010.0 1.0X
+1000 columns 18 20 2 0.0 18320597.0 0.2X
+10000 columns 182 198 18 0.0 182328011.0 0.0X
================================================================================================
many column field read and write
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
many column field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-1 cols x 100000 rows (read in-mem) 19 28 7 5.4 186.4 1.0X
-1 cols x 100000 rows (exec in-mem) 16 25 6 6.1 162.9 1.1X
-1 cols x 100000 rows (read parquet) 33 48 8 3.1 325.9 0.6X
-1 cols x 100000 rows (write parquet) 93 119 13 1.1 932.5 0.2X
-100 cols x 1000 rows (read in-mem) 11 15 3 8.7 114.6 1.6X
-100 cols x 1000 rows (exec in-mem) 14 19 5 7.2 139.1 1.3X
-100 cols x 1000 rows (read parquet) 23 28 4 4.3 231.6 0.8X
-100 cols x 1000 rows (write parquet) 85 91 7 1.2 854.0 0.2X
-2500 cols x 40 rows (read in-mem) 44 47 4 2.3 442.7 0.4X
-2500 cols x 40 rows (exec in-mem) 71 76 5 1.4 710.2 0.3X
-2500 cols x 40 rows (read parquet) 260 264 3 0.4 2598.0 0.1X
-2500 cols x 40 rows (write parquet) 119 127 6 0.8 1193.2 0.2X
+1 cols x 100000 rows (read in-mem) 18 24 3 5.5 180.9 1.0X
+1 cols x 100000 rows (exec in-mem) 17 22 3 5.9 170.1 1.1X
+1 cols x 100000 rows (read parquet) 31 37 5 3.3 306.0 0.6X
+1 cols x 100000 rows (write parquet) 93 100 6 1.1 926.7 0.2X
+100 cols x 1000 rows (read in-mem) 13 17 3 7.4 134.3 1.3X
+100 cols x 1000 rows (exec in-mem) 16 19 3 6.3 160.0 1.1X
+100 cols x 1000 rows (read parquet) 25 30 5 3.9 253.6 0.7X
+100 cols x 1000 rows (write parquet) 88 94 6 1.1 884.9 0.2X
+2500 cols x 40 rows (read in-mem) 49 51 6 2.0 489.9 0.4X
+2500 cols x 40 rows (exec in-mem) 75 82 11 1.3 747.3 0.2X
+2500 cols x 40 rows (read parquet) 284 292 9 0.4 2843.7 0.1X
+2500 cols x 40 rows (write parquet) 125 135 8 0.8 1248.2 0.1X
================================================================================================
wide shallowly nested struct field read and write
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
wide shallowly nested struct field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-1 wide x 100000 rows (read in-mem) 20 26 5 5.0 199.2 1.0X
-1 wide x 100000 rows (exec in-mem) 21 28 6 4.7 214.2 0.9X
-1 wide x 100000 rows (read parquet) 22 29 6 4.5 220.0 0.9X
-1 wide x 100000 rows (write parquet) 94 102 8 1.1 944.6 0.2X
-100 wide x 1000 rows (read in-mem) 15 18 3 6.7 148.6 1.3X
-100 wide x 1000 rows (exec in-mem) 16 18 3 6.3 159.2 1.3X
-100 wide x 1000 rows (read parquet) 21 24 4 4.7 214.7 0.9X
-100 wide x 1000 rows (write parquet) 90 97 12 1.1 899.6 0.2X
-2500 wide x 40 rows (read in-mem) 21 22 3 4.9 206.0 1.0X
-2500 wide x 40 rows (exec in-mem) 22 24 3 4.6 217.2 0.9X
-2500 wide x 40 rows (read parquet) 65 70 4 1.5 652.8 0.3X
-2500 wide x 40 rows (write parquet) 96 101 6 1.0 960.8 0.2X
+1 wide x 100000 rows (read in-mem) 21 27 5 4.8 209.3 1.0X
+1 wide x 100000 rows (exec in-mem) 23 27 3 4.3 232.4 0.9X
+1 wide x 100000 rows (read parquet) 24 29 6 4.2 235.5 0.9X
+1 wide x 100000 rows (write parquet) 96 104 7 1.0 956.0 0.2X
+100 wide x 1000 rows (read in-mem) 16 19 3 6.3 159.5 1.3X
+100 wide x 1000 rows (exec in-mem) 17 20 3 5.9 169.9 1.2X
+100 wide x 1000 rows (read parquet) 22 25 3 4.5 224.4 0.9X
+100 wide x 1000 rows (write parquet) 89 95 5 1.1 894.0 0.2X
+2500 wide x 40 rows (read in-mem) 22 24 3 4.6 216.2 1.0X
+2500 wide x 40 rows (exec in-mem) 23 26 4 4.4 229.5 0.9X
+2500 wide x 40 rows (read parquet) 66 69 4 1.5 661.6 0.3X
+2500 wide x 40 rows (write parquet) 96 103 8 1.0 964.4 0.2X
================================================================================================
deeply nested struct field read and write
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
deeply nested struct field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-1 deep x 100000 rows (read in-mem) 15 17 2 6.5 153.7 1.0X
-1 deep x 100000 rows (exec in-mem) 17 20 3 5.8 172.2 0.9X
-1 deep x 100000 rows (read parquet) 18 20 3 5.5 183.1 0.8X
-1 deep x 100000 rows (write parquet) 90 97 5 1.1 901.8 0.2X
-100 deep x 1000 rows (read in-mem) 28 30 3 3.5 283.9 0.5X
-100 deep x 1000 rows (exec in-mem) 419 424 6 0.2 4192.5 0.0X
-100 deep x 1000 rows (read parquet) 430 439 7 0.2 4304.2 0.0X
-100 deep x 1000 rows (write parquet) 105 110 6 1.0 1045.3 0.1X
-250 deep x 400 rows (read in-mem) 106 111 6 0.9 1060.8 0.1X
-250 deep x 400 rows (exec in-mem) 2507 2517 14 0.0 25067.5 0.0X
-250 deep x 400 rows (read parquet) 2585 2589 5 0.0 25851.9 0.0X
-250 deep x 400 rows (write parquet) 180 186 7 0.6 1799.1 0.1X
+1 deep x 100000 rows (read in-mem) 16 19 4 6.1 163.9 1.0X
+1 deep x 100000 rows (exec in-mem) 18 22 4 5.5 182.9 0.9X
+1 deep x 100000 rows (read parquet) 19 22 4 5.4 185.2 0.9X
+1 deep x 100000 rows (write parquet) 89 95 7 1.1 889.2 0.2X
+100 deep x 1000 rows (read in-mem) 30 31 3 3.4 297.8 0.6X
+100 deep x 1000 rows (exec in-mem) 404 410 8 0.2 4042.1 0.0X
+100 deep x 1000 rows (read parquet) 418 434 23 0.2 4178.6 0.0X
+100 deep x 1000 rows (write parquet) 103 108 5 1.0 1025.2 0.2X
+250 deep x 400 rows (read in-mem) 105 110 5 1.0 1050.3 0.2X
+250 deep x 400 rows (exec in-mem) 2463 2480 23 0.0 24634.0 0.0X
+250 deep x 400 rows (read parquet) 2537 2539 2 0.0 25372.5 0.0X
+250 deep x 400 rows (write parquet) 177 182 4 0.6 1765.6 0.1X
================================================================================================
bushy struct field read and write
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
bushy struct field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------------------------------------------------
-1 x 1 deep x 100000 rows (read in-mem) 13 15 3 7.7 129.7 1.0X
-1 x 1 deep x 100000 rows (exec in-mem) 15 16 3 6.9 145.7 0.9X
-1 x 1 deep x 100000 rows (read parquet) 17 19 4 5.8 171.3 0.8X
-1 x 1 deep x 100000 rows (write parquet) 88 92 5 1.1 877.2 0.1X
-128 x 8 deep x 1000 rows (read in-mem) 13 14 2 7.9 126.5 1.0X
-128 x 8 deep x 1000 rows (exec in-mem) 14 15 2 7.1 140.8 0.9X
-128 x 8 deep x 1000 rows (read parquet) 20 23 3 5.0 201.6 0.6X
-128 x 8 deep x 1000 rows (write parquet) 87 96 9 1.1 870.6 0.1X
-1024 x 11 deep x 100 rows (read in-mem) 16 18 3 6.1 164.3 0.8X
-1024 x 11 deep x 100 rows (exec in-mem) 18 20 3 5.7 175.6 0.7X
-1024 x 11 deep x 100 rows (read parquet) 34 38 4 2.9 341.2 0.4X
-1024 x 11 deep x 100 rows (write parquet) 91 95 6 1.1 906.0 0.1X
+1 x 1 deep x 100000 rows (read in-mem) 14 16 4 7.2 138.9 1.0X
+1 x 1 deep x 100000 rows (exec in-mem) 16 18 3 6.4 156.5 0.9X
+1 x 1 deep x 100000 rows (read parquet) 18 20 3 5.5 181.8 0.8X
+1 x 1 deep x 100000 rows (write parquet) 87 92 5 1.1 874.0 0.2X
+128 x 8 deep x 1000 rows (read in-mem) 13 15 3 7.6 132.4 1.0X
+128 x 8 deep x 1000 rows (exec in-mem) 15 16 3 6.8 147.2 0.9X
+128 x 8 deep x 1000 rows (read parquet) 21 23 4 4.9 205.7 0.7X
+128 x 8 deep x 1000 rows (write parquet) 85 89 4 1.2 854.3 0.2X
+1024 x 11 deep x 100 rows (read in-mem) 18 20 3 5.7 176.0 0.8X
+1024 x 11 deep x 100 rows (exec in-mem) 19 22 4 5.3 190.1 0.7X
+1024 x 11 deep x 100 rows (read parquet) 35 37 4 2.9 346.4 0.4X
+1024 x 11 deep x 100 rows (write parquet) 91 95 5 1.1 909.0 0.2X
================================================================================================
wide array field read and write
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
wide array field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-1 wide x 100000 rows (read in-mem) 15 17 3 6.7 150.1 1.0X
-1 wide x 100000 rows (exec in-mem) 17 19 3 6.0 167.9 0.9X
-1 wide x 100000 rows (read parquet) 17 21 3 5.8 173.9 0.9X
-1 wide x 100000 rows (write parquet) 90 97 6 1.1 902.5 0.2X
-100 wide x 1000 rows (read in-mem) 11 13 3 9.0 111.4 1.3X
-100 wide x 1000 rows (exec in-mem) 12 14 3 8.0 124.8 1.2X
-100 wide x 1000 rows (read parquet) 17 18 3 6.0 167.5 0.9X
-100 wide x 1000 rows (write parquet) 85 91 6 1.2 852.9 0.2X
-2500 wide x 40 rows (read in-mem) 11 13 3 9.0 111.4 1.3X
-2500 wide x 40 rows (exec in-mem) 13 15 3 8.0 125.7 1.2X
-2500 wide x 40 rows (read parquet) 18 19 3 5.6 178.1 0.8X
-2500 wide x 40 rows (write parquet) 85 89 4 1.2 848.2 0.2X
+1 wide x 100000 rows (read in-mem) 15 18 3 6.5 154.1 1.0X
+1 wide x 100000 rows (exec in-mem) 18 21 4 5.4 185.0 0.8X
+1 wide x 100000 rows (read parquet) 18 20 3 5.7 175.5 0.9X
+1 wide x 100000 rows (write parquet) 87 92 5 1.2 866.2 0.2X
+100 wide x 1000 rows (read in-mem) 12 13 3 8.6 116.8 1.3X
+100 wide x 1000 rows (exec in-mem) 13 15 3 7.5 133.2 1.2X
+100 wide x 1000 rows (read parquet) 17 20 3 5.9 169.6 0.9X
+100 wide x 1000 rows (write parquet) 85 88 4 1.2 851.1 0.2X
+2500 wide x 40 rows (read in-mem) 12 14 3 8.5 117.8 1.3X
+2500 wide x 40 rows (exec in-mem) 13 15 3 7.5 133.7 1.2X
+2500 wide x 40 rows (read parquet) 17 20 3 5.7 174.8 0.9X
+2500 wide x 40 rows (write parquet) 84 88 6 1.2 835.1 0.2X
================================================================================================
wide map field read and write
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
wide map field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-1 wide x 100000 rows (read in-mem) 13 14 2 7.8 127.5 1.0X
-1 wide x 100000 rows (exec in-mem) 16 17 2 6.3 159.9 0.8X
-1 wide x 100000 rows (read parquet) 21 23 3 4.8 206.7 0.6X
-1 wide x 100000 rows (write parquet) 87 92 5 1.2 865.5 0.1X
-100 wide x 1000 rows (read in-mem) 8 9 2 12.8 78.1 1.6X
-100 wide x 1000 rows (exec in-mem) 10 11 2 10.2 97.9 1.3X
-100 wide x 1000 rows (read parquet) 18 20 3 5.5 181.5 0.7X
-100 wide x 1000 rows (write parquet) 81 85 4 1.2 808.5 0.2X
-2500 wide x 40 rows (read in-mem) 10 10 1 10.2 97.6 1.3X
-2500 wide x 40 rows (exec in-mem) 11 12 2 8.7 114.5 1.1X
-2500 wide x 40 rows (read parquet) 19 20 3 5.4 185.3 0.7X
-2500 wide x 40 rows (write parquet) 84 89 6 1.2 837.2 0.2X
+1 wide x 100000 rows (read in-mem) 14 16 2 7.1 140.8 1.0X
+1 wide x 100000 rows (exec in-mem) 18 20 2 5.5 180.9 0.8X
+1 wide x 100000 rows (read parquet) 21 23 4 4.8 208.7 0.7X
+1 wide x 100000 rows (write parquet) 85 89 3 1.2 851.5 0.2X
+100 wide x 1000 rows (read in-mem) 8 10 2 13.3 75.3 1.9X
+100 wide x 1000 rows (exec in-mem) 9 11 2 10.7 93.0 1.5X
+100 wide x 1000 rows (read parquet) 18 21 3 5.5 182.0 0.8X
+100 wide x 1000 rows (write parquet) 79 84 4 1.3 793.4 0.2X
+2500 wide x 40 rows (read in-mem) 10 11 2 10.3 97.4 1.4X
+2500 wide x 40 rows (exec in-mem) 11 13 2 8.7 114.6 1.2X
+2500 wide x 40 rows (read parquet) 19 21 3 5.3 188.5 0.7X
+2500 wide x 40 rows (write parquet) 80 86 6 1.2 804.0 0.2X
diff --git a/sql/core/benchmarks/WideTableBenchmark-jdk21-results.txt b/sql/core/benchmarks/WideTableBenchmark-jdk21-results.txt
index 9cf6f38fccc00..52708bcd12a70 100644
--- a/sql/core/benchmarks/WideTableBenchmark-jdk21-results.txt
+++ b/sql/core/benchmarks/WideTableBenchmark-jdk21-results.txt
@@ -2,16 +2,16 @@
projection on wide table
================================================================================================
-OpenJDK 64-Bit Server VM 21.0.6+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 21.0.8+9-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
projection on wide table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-split threshold 10 2647 2714 60 0.4 2524.1 1.0X
-split threshold 100 2171 2198 47 0.5 2070.3 1.2X
-split threshold 1024 1654 1673 12 0.6 1577.3 1.6X
-split threshold 2048 1614 1633 13 0.6 1539.5 1.6X
-split threshold 4096 1737 1747 9 0.6 1656.7 1.5X
-split threshold 8192 2309 2355 59 0.5 2202.2 1.1X
-split threshold 65536 21274 21439 163 0.0 20288.2 0.1X
+split threshold 10 2651 2709 47 0.4 2527.9 1.0X
+split threshold 100 2175 2187 19 0.5 2074.3 1.2X
+split threshold 1024 1660 1680 13 0.6 1583.5 1.6X
+split threshold 2048 1613 1628 18 0.7 1538.2 1.6X
+split threshold 4096 1713 1745 21 0.6 1634.0 1.5X
+split threshold 8192 2277 2288 13 0.5 2171.1 1.2X
+split threshold 65536 21265 21793 328 0.0 20279.6 0.1X
diff --git a/sql/core/benchmarks/WideTableBenchmark-results.txt b/sql/core/benchmarks/WideTableBenchmark-results.txt
index 89b0c2490a0b5..62845b83a624d 100644
--- a/sql/core/benchmarks/WideTableBenchmark-results.txt
+++ b/sql/core/benchmarks/WideTableBenchmark-results.txt
@@ -2,16 +2,16 @@
projection on wide table
================================================================================================
-OpenJDK 64-Bit Server VM 17.0.14+7-LTS on Linux 6.8.0-1020-azure
+OpenJDK 64-Bit Server VM 17.0.16+8-LTS on Linux 6.11.0-1018-azure
AMD EPYC 7763 64-Core Processor
projection on wide table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------
-split threshold 10 2555 2620 67 0.4 2437.1 1.0X
-split threshold 100 2044 2069 22 0.5 1949.7 1.2X
-split threshold 1024 1653 1669 18 0.6 1576.0 1.5X
-split threshold 2048 1617 1649 24 0.6 1541.7 1.6X
-split threshold 4096 1687 1691 6 0.6 1608.7 1.5X
-split threshold 8192 2129 2146 24 0.5 2030.3 1.2X
-split threshold 65536 21564 21770 151 0.0 20565.0 0.1X
+split threshold 10 2546 2630 65 0.4 2428.4 1.0X
+split threshold 100 2080 2092 8 0.5 1984.0 1.2X
+split threshold 1024 1660 1685 19 0.6 1583.6 1.5X
+split threshold 2048 1632 1636 5 0.6 1556.0 1.6X
+split threshold 4096 1678 1696 15 0.6 1600.6 1.5X
+split threshold 8192 2133 2141 9 0.5 2034.0 1.2X
+split threshold 65536 21981 22102 128 0.0 20962.3 0.1X
diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java
index 90b55a8586de7..66637ac2bbd9f 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java
@@ -120,6 +120,8 @@ private UnsafeExternalRowSorter(
pageSizeBytes,
(int) SparkEnv.get().conf().get(
package$.MODULE$.SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD()),
+ (long) SparkEnv.get().conf().get(
+ package$.MODULE$.SHUFFLE_SPILL_MAX_SIZE_FORCE_SPILL_THRESHOLD()),
canUseRadixSort
);
}
diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java
index 8587d9290078b..af8d5a4610f64 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java
@@ -242,6 +242,8 @@ public UnsafeKVExternalSorter destructAndCreateExternalSorter() throws IOExcepti
map.getPageSizeBytes(),
(int) SparkEnv.get().conf().get(
package$.MODULE$.SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD()),
+ (long) SparkEnv.get().conf().get(
+ package$.MODULE$.SHUFFLE_SPILL_MAX_SIZE_FORCE_SPILL_THRESHOLD()),
map);
}
}
diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java
index 6f2d12e6b790a..6affcb61b8d69 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java
@@ -60,9 +60,10 @@ public UnsafeKVExternalSorter(
BlockManager blockManager,
SerializerManager serializerManager,
long pageSizeBytes,
- int numElementsForSpillThreshold) throws IOException {
+ int numElementsForSpillThreshold,
+ long maxRecordsSizeForSpillThreshold) throws IOException {
this(keySchema, valueSchema, blockManager, serializerManager, pageSizeBytes,
- numElementsForSpillThreshold, null);
+ numElementsForSpillThreshold, maxRecordsSizeForSpillThreshold, null);
}
public UnsafeKVExternalSorter(
@@ -72,6 +73,7 @@ public UnsafeKVExternalSorter(
SerializerManager serializerManager,
long pageSizeBytes,
int numElementsForSpillThreshold,
+ long maxRecordsSizeForSpillThreshold,
@Nullable BytesToBytesMap map) throws IOException {
this.keySchema = keySchema;
this.valueSchema = valueSchema;
@@ -98,6 +100,7 @@ public UnsafeKVExternalSorter(
(int) (long) SparkEnv.get().conf().get(package$.MODULE$.SHUFFLE_SORT_INIT_BUFFER_SIZE()),
pageSizeBytes,
numElementsForSpillThreshold,
+ maxRecordsSizeForSpillThreshold,
canUseRadixSort);
} else {
// During spilling, the pointer array in `BytesToBytesMap` will not be used, so we can borrow
@@ -165,6 +168,7 @@ public UnsafeKVExternalSorter(
(int) (long) SparkEnv.get().conf().get(package$.MODULE$.SHUFFLE_SORT_INIT_BUFFER_SIZE()),
pageSizeBytes,
numElementsForSpillThreshold,
+ maxRecordsSizeForSpillThreshold,
inMemSorter,
map.getTotalMemoryConsumption());
diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/HadoopLineRecordReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/HadoopLineRecordReader.java
new file mode 100644
index 0000000000000..9a015cc740e42
--- /dev/null
+++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/HadoopLineRecordReader.java
@@ -0,0 +1,298 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 java.io.IOException;
+import java.io.InputStream;
+
+import scala.Option;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FutureDataInputStreamBuilder;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.compress.CodecPool;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.SplitCompressionInputStream;
+import org.apache.hadoop.io.compress.SplittableCompressionCodec;
+import org.apache.hadoop.io.compress.Decompressor;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.hadoop.mapreduce.lib.input.SplitLineReader;
+import org.apache.hadoop.mapreduce.lib.input.CompressedSplitLineReader;
+import org.apache.hadoop.mapreduce.lib.input.UncompressedSplitLineReader;
+import org.apache.hadoop.util.functional.FutureIO;
+import org.apache.spark.internal.SparkLogger;
+import org.apache.spark.internal.SparkLoggerFactory;
+import org.apache.spark.io.HadoopCodecStreams;
+
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_END;
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_START;
+
+/**
+ * Inlined from Hadoop's LineRecordReader to add support for passing compression option
+ * and also add support for other codecs like ZSTD.
+ * Specifically, it calls in HadoopCodecStreams.getDecompressionCodec to get the codec
+ * and calls HadoopCodecStreams.createZstdInputStream when the codec fails to create the
+ * InputStream.
+ */
+
+/**
+ * Treats keys as offset in file and value as line.
+ */
+public class HadoopLineRecordReader extends RecordReader {
+ public static final String MAX_LINE_LENGTH =
+ "mapreduce.input.linerecordreader.line.maxlength";
+ private static final SparkLogger LOG =
+ SparkLoggerFactory.getLogger(HadoopLineRecordReader.class);
+
+ private long start;
+ private long pos;
+ private long end;
+ private SplitLineReader in;
+ private FSDataInputStream fileIn;
+ private Seekable filePosition;
+ private int maxLineLength;
+ private LongWritable key;
+ private Text value;
+ private boolean isCompressedInput;
+ private Decompressor decompressor;
+ private byte[] recordDelimiterBytes;
+
+ public HadoopLineRecordReader() {
+ }
+
+ public HadoopLineRecordReader(byte[] recordDelimiter) {
+ this.recordDelimiterBytes = recordDelimiter;
+ }
+
+ public void initialize(InputSplit genericSplit,
+ TaskAttemptContext context) throws IOException {
+ FileSplit split = (FileSplit) genericSplit;
+ Configuration job = context.getConfiguration();
+ this.maxLineLength = job.getInt(MAX_LINE_LENGTH, Integer.MAX_VALUE);
+ start = split.getStart();
+ end = start + split.getLength();
+ final Path file = split.getPath();
+
+ // open the file and seek to the start of the split
+ final FutureDataInputStreamBuilder builder =
+ file.getFileSystem(job).openFile(file);
+ // the start and end of the split may be used to build
+ // an input strategy.
+ builder.optLong(FS_OPTION_OPENFILE_SPLIT_START, start);
+ builder.optLong(FS_OPTION_OPENFILE_SPLIT_END, end);
+ FutureIO.propagateOptions(builder, job,
+ MRJobConfig.INPUT_FILE_OPTION_PREFIX,
+ MRJobConfig.INPUT_FILE_MANDATORY_PREFIX);
+ fileIn = FutureIO.awaitFuture(builder.build());
+
+ try {
+ Option codecOpt = HadoopCodecStreams.getDecompressionCodec(job, file);
+ if (codecOpt.isDefined()) {
+ CompressionCodec codec = codecOpt.get();
+ isCompressedInput = true;
+ try {
+ decompressor = CodecPool.getDecompressor(codec);
+ if (codec instanceof SplittableCompressionCodec) {
+ final SplitCompressionInputStream cIn =
+ ((SplittableCompressionCodec) codec).createInputStream(
+ fileIn, decompressor, start, end,
+ SplittableCompressionCodec.READ_MODE.BYBLOCK);
+ in = new CompressedSplitLineReader(cIn, job,
+ this.recordDelimiterBytes);
+ start = cIn.getAdjustedStart();
+ end = cIn.getAdjustedEnd();
+ filePosition = cIn;
+ } else {
+ if (start != 0) {
+ // So we have a split that is only part of a file stored using
+ // a Compression codec that cannot be split.
+ throw new IOException("Cannot seek in " +
+ codec.getClass().getSimpleName() + " compressed stream");
+ }
+
+ in = new SplitLineReader(codec.createInputStream(fileIn,
+ decompressor), job, this.recordDelimiterBytes);
+ filePosition = fileIn;
+ }
+ } catch (RuntimeException e) {
+ // Try Spark's ZSTD decompression support. This is not available in Hadoop's
+ // version of LineRecordReader.
+ Option decompressedStreamOpt =
+ HadoopCodecStreams.createZstdInputStream(file, fileIn);
+ if (decompressedStreamOpt.isEmpty()) {
+ // File is either not ZSTD compressed or ZSTD codec is not available.
+ throw e;
+ }
+ InputStream decompressedStream = decompressedStreamOpt.get();
+ if (start != 0) {
+ decompressedStream.close();
+ throw new IOException("Cannot seek in "+ file.getName() +
+ " compressed stream");
+ }
+
+ isCompressedInput = true;
+ in = new SplitLineReader(decompressedStream, job, this.recordDelimiterBytes);
+ filePosition = fileIn;
+ }
+ } else {
+ fileIn.seek(start);
+ in = new UncompressedSplitLineReader(
+ fileIn, job, this.recordDelimiterBytes, split.getLength());
+ filePosition = fileIn;
+ }
+ // If this is not the first split, we always throw away first record
+ // because we always (except the last split) read one extra line in
+ // next() method.
+ if (start != 0) {
+ start += in.readLine(new Text(), 0, maxBytesToConsume(start));
+ }
+ this.pos = start;
+ } catch (Exception e) {
+ fileIn.close();
+ throw e;
+ }
+ }
+
+
+ private int maxBytesToConsume(long pos) {
+ return isCompressedInput
+ ? Integer.MAX_VALUE
+ : (int) Math.max(Math.min(Integer.MAX_VALUE, end - pos), maxLineLength);
+ }
+
+ private long getFilePosition() throws IOException {
+ long retVal;
+ if (isCompressedInput && null != filePosition) {
+ retVal = filePosition.getPos();
+ } else {
+ retVal = pos;
+ }
+ return retVal;
+ }
+
+ private int skipUtfByteOrderMark() throws IOException {
+ // Strip BOM(Byte Order Mark)
+ // Text only support UTF-8, we only need to check UTF-8 BOM
+ // (0xEF,0xBB,0xBF) at the start of the text stream.
+ int newMaxLineLength = (int) Math.min(3L + (long) maxLineLength,
+ Integer.MAX_VALUE);
+ int newSize = in.readLine(value, newMaxLineLength, maxBytesToConsume(pos));
+ // Even we read 3 extra bytes for the first line,
+ // we won't alter existing behavior (no backwards incompat issue).
+ // Because the newSize is less than maxLineLength and
+ // the number of bytes copied to Text is always no more than newSize.
+ // If the return size from readLine is not less than maxLineLength,
+ // we will discard the current line and read the next line.
+ pos += newSize;
+ int textLength = value.getLength();
+ byte[] textBytes = value.getBytes();
+ if ((textLength >= 3) && (textBytes[0] == (byte)0xEF) &&
+ (textBytes[1] == (byte)0xBB) && (textBytes[2] == (byte)0xBF)) {
+ // find UTF-8 BOM, strip it.
+ LOG.info("Found UTF-8 BOM and skipped it");
+ textLength -= 3;
+ newSize -= 3;
+ if (textLength > 0) {
+ // It may work to use the same buffer and not do the copyBytes
+ textBytes = value.copyBytes();
+ value.set(textBytes, 3, textLength);
+ } else {
+ value.clear();
+ }
+ }
+ return newSize;
+ }
+
+ public boolean nextKeyValue() throws IOException {
+ if (key == null) {
+ key = new LongWritable();
+ }
+ key.set(pos);
+ if (value == null) {
+ value = new Text();
+ }
+ int newSize = 0;
+ // We always read one extra line, which lies outside the upper
+ // split limit i.e. (end - 1)
+ while (getFilePosition() <= end || in.needAdditionalRecordAfterSplit()) {
+ if (pos == 0) {
+ newSize = skipUtfByteOrderMark();
+ } else {
+ newSize = in.readLine(value, maxLineLength, maxBytesToConsume(pos));
+ pos += newSize;
+ }
+
+ if ((newSize == 0) || (newSize < maxLineLength)) {
+ break;
+ }
+
+ // line too long. try again
+ LOG.info("Skipped line of size " + newSize + " at pos " +
+ (pos - newSize));
+ }
+ if (newSize == 0) {
+ key = null;
+ value = null;
+ return false;
+ } else {
+ return true;
+ }
+ }
+
+ @Override
+ public LongWritable getCurrentKey() {
+ return key;
+ }
+
+ @Override
+ public Text getCurrentValue() {
+ return value;
+ }
+
+ /**
+ * Get the progress within the split
+ */
+ public float getProgress() throws IOException {
+ if (start == end) {
+ return 0.0f;
+ } else {
+ return Math.min(1.0f, (getFilePosition() - start) / (float)(end - start));
+ }
+ }
+
+ public synchronized void close() throws IOException {
+ try {
+ if (in != null) {
+ in.close();
+ }
+ } finally {
+ if (decompressor != null) {
+ CodecPool.returnDecompressor(decompressor);
+ decompressor = null;
+ }
+ }
+ }
+}
diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetColumnVector.java
index 7fb8be7caf286..3331c8dfd8f5c 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetColumnVector.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetColumnVector.java
@@ -21,9 +21,8 @@
import java.util.List;
import java.util.Set;
-import com.google.common.base.Preconditions;
-
import org.apache.spark.memory.MemoryMode;
+import org.apache.spark.network.util.JavaUtils;
import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
@@ -131,7 +130,8 @@ final class ParquetColumnVector {
definitionLevels = allocateLevelsVector(capacity, memoryMode);
}
} else {
- Preconditions.checkArgument(column.children().size() == vector.getNumChildren());
+ JavaUtils.checkArgument(column.children().size() == vector.getNumChildren(),
+ "The number of column children is different from the number of vector children");
boolean allChildrenAreMissing = true;
for (int i = 0; i < column.children().size(); i++) {
diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java
index 889f11e119730..eb6c84b8113b8 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java
@@ -24,6 +24,7 @@
import org.apache.parquet.schema.LogicalTypeAnnotation.IntLogicalTypeAnnotation;
import org.apache.parquet.schema.LogicalTypeAnnotation.DateLogicalTypeAnnotation;
import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
+import org.apache.parquet.schema.LogicalTypeAnnotation.TimeLogicalTypeAnnotation;
import org.apache.parquet.schema.LogicalTypeAnnotation.TimestampLogicalTypeAnnotation;
import org.apache.parquet.schema.PrimitiveType;
@@ -159,7 +160,7 @@ public ParquetVectorUpdater getUpdater(ColumnDescriptor descriptor, DataType spa
} else if (canReadAsDecimal(descriptor, sparkType)) {
return new LongToDecimalUpdater(descriptor, (DecimalType) sparkType);
} else if (sparkType instanceof TimeType) {
- return new LongUpdater();
+ return new LongAsNanosUpdater();
}
}
case FLOAT -> {
@@ -233,6 +234,11 @@ boolean isTimestampTypeMatched(LogicalTypeAnnotation.TimeUnit unit) {
annotation.getUnit() == unit;
}
+ boolean isTimeTypeMatched(LogicalTypeAnnotation.TimeUnit unit) {
+ return logicalTypeAnnotation instanceof TimeLogicalTypeAnnotation annotation &&
+ annotation.getUnit() == unit;
+ }
+
boolean isUnsignedIntTypeMatched(int bitWidth) {
return logicalTypeAnnotation instanceof IntLogicalTypeAnnotation annotation &&
!annotation.isSigned() && annotation.getBitWidth() == bitWidth;
@@ -825,6 +831,42 @@ public void decodeSingleDictionaryId(
}
}
+ private static class LongAsNanosUpdater implements ParquetVectorUpdater {
+ @Override
+ public void readValues(
+ int total,
+ int offset,
+ WritableColumnVector values,
+ VectorizedValuesReader valuesReader) {
+ for (int i = 0; i < total; ++i) {
+ readValue(offset + i, values, valuesReader);
+ }
+ }
+
+ @Override
+ public void skipValues(int total, VectorizedValuesReader valuesReader) {
+ valuesReader.skipLongs(total);
+ }
+
+ @Override
+ public void readValue(
+ int offset,
+ WritableColumnVector values,
+ VectorizedValuesReader valuesReader) {
+ values.putLong(offset, DateTimeUtils.microsToNanos(valuesReader.readLong()));
+ }
+
+ @Override
+ public void decodeSingleDictionaryId(
+ int offset,
+ WritableColumnVector values,
+ WritableColumnVector dictionaryIds,
+ Dictionary dictionary) {
+ long micros = dictionary.decodeToLong(dictionaryIds.getDictId(offset));
+ values.putLong(offset, DateTimeUtils.microsToNanos(micros));
+ }
+ }
+
private static class FloatUpdater implements ParquetVectorUpdater {
@Override
public void readValues(
diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java
index 731c78cf94503..6e1660dc8c873 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java
@@ -165,7 +165,8 @@ private boolean isLazyDecodingSupported(
case INT64: {
boolean isDecimal = sparkType instanceof DecimalType;
boolean needsUpcast = (isDecimal && !DecimalType.is64BitDecimalType(sparkType)) ||
- updaterFactory.isTimestampTypeMatched(TimeUnit.MILLIS);
+ updaterFactory.isTimestampTypeMatched(TimeUnit.MILLIS) ||
+ updaterFactory.isTimeTypeMatched(TimeUnit.MICROS);
boolean needsRebase = updaterFactory.isTimestampTypeMatched(TimeUnit.MICROS) &&
!"CORRECTED".equals(datetimeRebaseMode);
isSupported = !needsUpcast && !needsRebase && !needsDecimalScaleRebase(sparkType);
diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedDeltaBinaryPackedReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedDeltaBinaryPackedReader.java
index 071720d4dbfc7..7b56e1ebf2394 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedDeltaBinaryPackedReader.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedDeltaBinaryPackedReader.java
@@ -21,12 +21,13 @@
import java.nio.ByteBuffer;
import java.util.Arrays;
-import org.apache.parquet.Preconditions;
import org.apache.parquet.bytes.ByteBufferInputStream;
import org.apache.parquet.bytes.BytesUtils;
import org.apache.parquet.column.values.bitpacking.BytePackerForLong;
import org.apache.parquet.column.values.bitpacking.Packer;
import org.apache.parquet.io.ParquetDecodingException;
+
+import org.apache.spark.network.util.JavaUtils;
import org.apache.spark.sql.catalyst.util.RebaseDateTime;
import org.apache.spark.sql.execution.datasources.DataSourceUtils;
import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
@@ -80,14 +81,14 @@ public class VectorizedDeltaBinaryPackedReader extends VectorizedReaderBase {
@Override
public void initFromPage(int valueCount, ByteBufferInputStream in) throws IOException {
- Preconditions.checkArgument(valueCount >= 1,
+ JavaUtils.checkArgument(valueCount >= 1,
"Page must have at least one value, but it has " + valueCount);
this.in = in;
// Read the header
this.blockSizeInValues = BytesUtils.readUnsignedVarInt(in);
this.miniBlockNumInABlock = BytesUtils.readUnsignedVarInt(in);
double miniSize = (double) blockSizeInValues / miniBlockNumInABlock;
- Preconditions.checkArgument(miniSize % 8 == 0,
+ JavaUtils.checkArgument(miniSize % 8 == 0,
"miniBlockSize must be multiple of 8, but it's " + miniSize);
this.miniBlockSizeInValues = (int) miniSize;
// True value count. May be less than valueCount because of nulls
diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedRleValuesReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedRleValuesReader.java
index 0d380997fd5b6..60544665409df 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedRleValuesReader.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedRleValuesReader.java
@@ -20,7 +20,6 @@
import java.io.IOException;
import java.nio.ByteBuffer;
-import org.apache.parquet.Preconditions;
import org.apache.parquet.bytes.ByteBufferInputStream;
import org.apache.parquet.bytes.BytesUtils;
import org.apache.parquet.column.values.ValuesReader;
@@ -30,6 +29,7 @@
import org.apache.parquet.io.api.Binary;
import org.apache.spark.SparkUnsupportedOperationException;
+import org.apache.spark.network.util.JavaUtils;
import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
/**
@@ -122,7 +122,7 @@ public void initFromPage(int valueCount, ByteBufferInputStream in) throws IOExce
* Initializes the internal state for decoding ints of `bitWidth`.
*/
private void init(int bitWidth) {
- Preconditions.checkArgument(bitWidth >= 0 && bitWidth <= 32, "bitWidth must be >= 0 and <= 32");
+ JavaUtils.checkArgument(bitWidth >= 0 && bitWidth <= 32, "bitWidth must be >= 0 and <= 32");
this.bitWidth = bitWidth;
this.bytesWidth = BytesUtils.paddedByteCountFromBits(bitWidth);
this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala
index 40779c66600fd..85f59c282ff55 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala
@@ -23,10 +23,10 @@ import java.nio.channels.{Channels, SocketChannel}
import net.razorvine.pickle.{Pickler, Unpickler}
import org.apache.spark.api.python.DechunkedInputStream
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.CLASS_LOADER
import org.apache.spark.security.SocketAuthServer
-import org.apache.spark.sql.{internal, Column, DataFrame, Row, SparkSession}
+import org.apache.spark.sql.{internal, Column, DataFrame, Row, SparkSession, TableArg}
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, TableFunctionRegistry}
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
@@ -182,6 +182,9 @@ private[sql] object PythonSQLUtils extends Logging {
def namedArgumentExpression(name: String, e: Column): Column =
Column(NamedArgumentExpression(name, expression(e)))
+ def namedArgumentExpression(name: String, e: TableArg): Column =
+ Column(NamedArgumentExpression(name, e.expression))
+
@scala.annotation.varargs
def fn(name: String, arguments: Column*): Column = Column.fn(name, arguments: _*)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala
index 1efd8f9e32208..3fc3cf698df9b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala
@@ -27,7 +27,7 @@ import org.apache.spark.TaskContext
import org.apache.spark.api.java.{JavaRDD, JavaSparkContext}
import org.apache.spark.api.r.SerDe
import org.apache.spark.broadcast.Broadcast
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.CONFIG
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.Row
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala
index 35e8e5c6000ae..de91e5e8a44b2 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala
@@ -25,14 +25,15 @@ import java.nio.file.{CopyOption, Files, Path, Paths, StandardCopyOption}
import java.util.concurrent.CopyOnWriteArrayList
import java.util.concurrent.atomic.AtomicBoolean
+import scala.collection.mutable.ListBuffer
import scala.jdk.CollectionConverters._
import scala.reflect.ClassTag
-import org.apache.commons.io.{FilenameUtils, FileUtils}
+import org.apache.commons.io.FilenameUtils
import org.apache.hadoop.fs.{LocalFileSystem, Path => FSPath}
import org.apache.spark.{JobArtifactSet, JobArtifactState, SparkContext, SparkEnv, SparkException, SparkRuntimeException, SparkUnsupportedOperationException}
-import org.apache.spark.internal.{Logging, LogKeys, MDC}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.internal.config.{CONNECT_SCALA_UDF_STUB_PREFIXES, EXECUTOR_USER_CLASS_PATH_FIRST}
import org.apache.spark.sql.Artifact
import org.apache.spark.sql.classic.SparkSession
@@ -213,7 +214,7 @@ class ArtifactManager(session: SparkSession) extends AutoCloseable with Logging
// Disallow overwriting with modified version
if (Files.exists(target)) {
// makes the query idempotent
- if (FileUtils.contentEquals(target.toFile, serverLocalStagingPath.toFile)) {
+ if (Utils.contentEquals(target.toFile, serverLocalStagingPath.toFile)) {
return
}
@@ -266,28 +267,39 @@ class ArtifactManager(session: SparkSession) extends AutoCloseable with Logging
* they are from a permanent location.
*/
private[sql] def addLocalArtifacts(artifacts: Seq[Artifact]): Unit = {
+ val failedArtifactExceptions = ListBuffer[SparkRuntimeException]()
+
artifacts.foreach { artifact =>
- artifact.storage match {
- case d: Artifact.LocalFile =>
- addArtifact(
- artifact.path,
- d.path,
- fragment = None,
- deleteStagedFile = false)
- case d: Artifact.InMemory =>
- val tempDir = Utils.createTempDir().toPath
- val tempFile = tempDir.resolve(artifact.path.getFileName)
- val outStream = Files.newOutputStream(tempFile)
- Utils.tryWithSafeFinallyAndFailureCallbacks {
- d.stream.transferTo(outStream)
- addArtifact(artifact.path, tempFile, fragment = None)
- }(finallyBlock = {
- outStream.close()
- })
- case _ =>
- throw SparkException.internalError(s"Unsupported artifact storage: ${artifact.storage}")
+ try {
+ artifact.storage match {
+ case d: Artifact.LocalFile =>
+ addArtifact(
+ artifact.path,
+ d.path,
+ fragment = None,
+ deleteStagedFile = false)
+ case d: Artifact.InMemory =>
+ val tempDir = Utils.createTempDir().toPath
+ val tempFile = tempDir.resolve(artifact.path.getFileName)
+ val outStream = Files.newOutputStream(tempFile)
+ Utils.tryWithSafeFinallyAndFailureCallbacks {
+ d.stream.transferTo(outStream)
+ addArtifact(artifact.path, tempFile, fragment = None)
+ }(finallyBlock = {
+ outStream.close()
+ })
+ case _ =>
+ throw SparkException.internalError(s"Unsupported artifact storage: ${artifact.storage}")
+ }
+ } catch {
+ case e: SparkRuntimeException if e.getCondition == "ARTIFACT_ALREADY_EXISTS" =>
+ failedArtifactExceptions += e
}
}
+
+ if (failedArtifactExceptions.nonEmpty) {
+ throw ArtifactUtils.mergeExceptionsWithSuppressed(failedArtifactExceptions.toSeq)
+ }
}
def classloader: ClassLoader = synchronized {
@@ -340,7 +352,7 @@ class ArtifactManager(session: SparkSession) extends AutoCloseable with Logging
val sparkContext = session.sparkContext
val newArtifactManager = new ArtifactManager(newSession)
if (artifactPath.toFile.exists()) {
- FileUtils.copyDirectory(artifactPath.toFile, newArtifactManager.artifactPath.toFile)
+ Utils.copyDirectory(artifactPath.toFile, newArtifactManager.artifactPath.toFile)
}
val blockManager = sparkContext.env.blockManager
val newBlockIds = cachedBlockIdList.asScala.map { blockId =>
@@ -524,7 +536,7 @@ object ArtifactManager extends Logging {
// Clean up artifacts folder
try {
- FileUtils.deleteDirectory(artifactPath.toFile)
+ Utils.deleteRecursively(artifactPath.toFile)
} catch {
case e: IOException =>
logWarning(log"Failed to delete directory ${MDC(LogKeys.PATH, artifactPath.toFile)}: " +
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala
index 65fafb5a34c6e..f66b5bd988c2b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala
@@ -58,7 +58,7 @@ private[sql] class AvroDeserializer(
def this(
rootAvroType: Schema,
rootCatalystType: DataType,
- datetimeRebaseMode: LegacyBehaviorPolicy.Value,
+ datetimeRebaseMode: String,
useStableIdForUnionType: Boolean,
stableIdPrefixForUnionType: String,
recursiveFieldMaxDepth: Int) = {
@@ -66,7 +66,7 @@ private[sql] class AvroDeserializer(
rootAvroType,
rootCatalystType,
positionalFieldMatch = false,
- RebaseSpec(datetimeRebaseMode),
+ RebaseSpec(LegacyBehaviorPolicy.withName(datetimeRebaseMode)),
new NoopFilters,
useStableIdForUnionType,
stableIdPrefixForUnionType,
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala
index bf35a78488e44..90781d4ad7077 100755
--- a/sql/core/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala
@@ -34,13 +34,16 @@ import org.apache.spark.internal.Logging
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.{InternalRow, NoopFilters, OrderedFilters}
import org.apache.spark.sql.execution.datasources.{DataSourceUtils, FileFormat, OutputWriterFactory, PartitionedFile}
-import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.{SessionStateHelper, SQLConf}
import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
import org.apache.spark.sql.types._
import org.apache.spark.util.SerializableConfiguration
private[sql] class AvroFileFormat extends FileFormat
- with DataSourceRegister with Logging with Serializable {
+ with DataSourceRegister
+ with SessionStateHelper
+ with Logging
+ with Serializable {
AvroFileFormat.registerCustomAvroTypes()
@@ -73,7 +76,7 @@ private[sql] class AvroFileFormat extends FileFormat
job: Job,
options: Map[String, String],
dataSchema: StructType): OutputWriterFactory = {
- AvroUtils.prepareWrite(spark.sessionState.conf, job, options, dataSchema)
+ AvroUtils.prepareWrite(getSqlConf(spark), job, options, dataSchema)
}
override def buildReader(
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala
index ab3607d1bd7a7..da42333fad0fb 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala
@@ -29,7 +29,7 @@ import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.{DataSourceOptions, FileSourceOptions}
import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, FailFastMode, ParseMode}
import org.apache.spark.sql.errors.QueryCompilationErrors
-import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf}
+import org.apache.spark.sql.internal.SQLConf
/**
* Options for Avro Reader and Writer stored in case insensitive manner.
@@ -129,9 +129,9 @@ private[sql] class AvroOptions(
/**
* The rebasing mode for the DATE and TIMESTAMP_MICROS, TIMESTAMP_MILLIS values in reads.
*/
- val datetimeRebaseModeInRead: LegacyBehaviorPolicy.Value = parameters
- .get(DATETIME_REBASE_MODE).map(LegacyBehaviorPolicy.withName)
- .getOrElse(SQLConf.get.getConf(SQLConf.AVRO_REBASE_MODE_IN_READ))
+ val datetimeRebaseModeInRead: String = parameters
+ .get(DATETIME_REBASE_MODE)
+ .getOrElse(SQLConf.get.getConf(SQLConf.AVRO_REBASE_MODE_IN_READ).toString)
val useStableIdForUnionType: Boolean =
parameters.get(STABLE_ID_FOR_UNION_TYPE).map(_.toBoolean).getOrElse(false)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala
index b388c98ffcb1d..c338bd428bbed 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala
@@ -31,7 +31,7 @@ import org.apache.hadoop.fs.FileStatus
import org.apache.hadoop.mapreduce.Job
import org.apache.spark.{SparkException, SparkIllegalArgumentException}
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{CODEC_LEVEL, CODEC_NAME, CONFIG, PATH}
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.avro.AvroCompressionCodec._
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala b/sql/core/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala
index c936099ef3764..b425f63d6a7ed 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala
@@ -28,7 +28,7 @@ import org.apache.avro.Schema.Type._
import org.apache.avro.SchemaBuilder.FieldAssembler
import org.apache.spark.annotation.DeveloperApi
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{FIELD_NAME, FIELD_TYPE, RECURSIVE_DEPTH}
import org.apache.spark.sql.avro.AvroOptions.RECURSIVE_FIELD_MAX_DEPTH_LIMIT
import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDataSource.scala
index 07208ca7760db..a7b3385a27820 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDataSource.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDataSource.scala
@@ -34,7 +34,7 @@ import org.apache.spark.sql.execution.command.DDLUtils
import org.apache.spark.sql.execution.datasources.{DataSource, LogicalRelation}
import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Utils, FileDataSourceV2}
import org.apache.spark.sql.execution.datasources.v2.python.PythonDataSourceV2
-import org.apache.spark.sql.execution.streaming.StreamingRelation
+import org.apache.spark.sql.execution.streaming.runtime.StreamingRelation
import org.apache.spark.sql.sources.StreamSourceProvider
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.util.CaseInsensitiveStringMap
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala
index d7d8fc07b872f..b664f52c8c1c5 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala
@@ -17,11 +17,8 @@
package org.apache.spark.sql.catalyst.analysis
-import org.apache.commons.lang3.StringUtils
-
import org.apache.spark.SparkException
import org.apache.spark.internal.LogKeys.CONFIG
-import org.apache.spark.internal.MDC
import org.apache.spark.sql.SaveMode
import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier}
import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, CatalogUtils, ClusterBySpec}
@@ -40,6 +37,7 @@ import org.apache.spark.sql.internal.{HiveSerDe, SQLConf}
import org.apache.spark.sql.internal.connector.V1Function
import org.apache.spark.sql.types.{MetadataBuilder, StringType, StructField, StructType}
import org.apache.spark.util.ArrayImplicits._
+import org.apache.spark.util.SparkStringUtils
/**
* Converts resolved v2 commands to v1 if the catalog is the session catalog. Since the v2 commands
@@ -149,7 +147,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager)
AlterDatabasePropertiesCommand(db, properties)
case SetNamespaceLocation(ResolvedV1Database(db), location) if conf.useV1Command =>
- if (StringUtils.isEmpty(location)) {
+ if (SparkStringUtils.isEmpty(location)) {
throw QueryExecutionErrors.invalidEmptyLocationError(location)
}
AlterDatabaseSetLocationCommand(db, location)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/Catalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/Catalog.scala
index 1e028d2046eb5..45f494f65c300 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/classic/Catalog.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/Catalog.scala
@@ -36,6 +36,7 @@ import org.apache.spark.sql.catalyst.types.DataTypeUtils
import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
import org.apache.spark.sql.connector.catalog.{CatalogManager, SupportsNamespaces, TableCatalog}
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.{CatalogHelper, MultipartIdentifierHelper, NamespaceHelper, TransformHelper}
+import org.apache.spark.sql.connector.catalog.CatalogV2Util.v2ColumnsToStructType
import org.apache.spark.sql.errors.QueryCompilationErrors
import org.apache.spark.sql.execution.command.{ShowNamespacesCommand, ShowTablesCommand}
import org.apache.spark.sql.execution.datasources.{DataSource, LogicalRelation}
@@ -408,7 +409,8 @@ class Catalog(sparkSession: SparkSession) extends catalog.Catalog {
val clusteringColumnNames = clusterBySpecOpt.map { clusterBySpec =>
clusterBySpec.columnNames.map(_.toString)
}.getOrElse(Nil).toSet
- schemaToColumns(table.schema(), partitionColumnNames.contains, bucketColumnNames.contains,
+ val schema = v2ColumnsToStructType(table.columns())
+ schemaToColumns(schema, partitionColumnNames.contains, bucketColumnNames.contains,
clusteringColumnNames.contains)
case ResolvedPersistentView(_, _, metadata) =>
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/DataFrameWriter.scala
index 501b4985128dd..2d3e4b84d9ae5 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/classic/DataFrameWriter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/DataFrameWriter.scala
@@ -25,7 +25,7 @@ import org.apache.spark.annotation.Stable
import org.apache.spark.sql
import org.apache.spark.sql.SaveMode
import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, NoSuchTableException, UnresolvedIdentifier, UnresolvedRelation}
+import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, UnresolvedIdentifier, UnresolvedRelation}
import org.apache.spark.sql.catalyst.catalog._
import org.apache.spark.sql.catalyst.expressions.Literal
import org.apache.spark.sql.catalyst.plans.logical._
@@ -36,8 +36,8 @@ import org.apache.spark.sql.connector.catalog.TableWritePrivilege._
import org.apache.spark.sql.connector.expressions.{ClusterByTransform, FieldReference, IdentityTransform, Transform}
import org.apache.spark.sql.errors.QueryCompilationErrors
import org.apache.spark.sql.execution.QueryExecution
-import org.apache.spark.sql.execution.command.DDLUtils
-import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, DataSourceUtils, LogicalRelation}
+import org.apache.spark.sql.execution.command.{DDLUtils, SaveAsV1TableCommand}
+import org.apache.spark.sql.execution.datasources.{DataSource, DataSourceUtils}
import org.apache.spark.sql.execution.datasources.v2._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode
@@ -111,11 +111,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) extends sql.DataFram
* @since 1.4.0
*/
def save(path: String): Unit = {
- if (!df.sparkSession.sessionState.conf.legacyPathOptionBehavior &&
- extraOptions.contains("path")) {
- throw QueryCompilationErrors.pathOptionNotSetCorrectlyWhenWritingError()
+ runCommand(df.sparkSession) {
+ saveCommand(Some(path))
}
- saveInternal(Some(path))
}
/**
@@ -123,9 +121,17 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) extends sql.DataFram
*
* @since 1.4.0
*/
- def save(): Unit = saveInternal(None)
+ def save(): Unit = {
+ runCommand(df.sparkSession) {
+ saveCommand(None)
+ }
+ }
- private def saveInternal(path: Option[String]): Unit = {
+ private[sql] def saveCommand(path: Option[String]): LogicalPlan = {
+ if (path.isDefined && !df.sparkSession.sessionState.conf.legacyPathOptionBehavior &&
+ extraOptions.contains("path")) {
+ throw QueryCompilationErrors.pathOptionNotSetCorrectlyWhenWritingError()
+ }
if (source.toLowerCase(Locale.ROOT) == DDLUtils.HIVE_PROVIDER) {
throw QueryCompilationErrors.cannotOperateOnHiveDataSourceFilesError("write")
}
@@ -179,23 +185,19 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) extends sql.DataFram
// Streaming also uses the data source V2 API. So it may be that the data source
// implements v2, but has no v2 implementation for batch writes. In that case, we
// fall back to saving as though it's a V1 source.
- return saveToV1Source(path)
+ return saveToV1SourceCommand(path)
}
}
val relation = DataSourceV2Relation.create(table, catalog, ident, dsOptions)
checkPartitioningMatchesV2Table(table)
if (curmode == SaveMode.Append) {
- runCommand(df.sparkSession) {
- AppendData.byName(relation, df.logicalPlan, finalOptions)
- }
+ AppendData.byName(relation, df.logicalPlan, finalOptions)
} else {
// Truncate the table. TableCapabilityCheck will throw a nice exception if this
// isn't supported
- runCommand(df.sparkSession) {
- OverwriteByExpression.byName(
- relation, df.logicalPlan, Literal(true), finalOptions)
- }
+ OverwriteByExpression.byName(
+ relation, df.logicalPlan, Literal(true), finalOptions)
}
case createMode =>
@@ -215,16 +217,14 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) extends sql.DataFram
serde = None,
external = false,
constraints = Seq.empty)
- runCommand(df.sparkSession) {
- CreateTableAsSelect(
- UnresolvedIdentifier(
- catalog.name +: ident.namespace.toImmutableArraySeq :+ ident.name),
- partitioningAsV2,
- df.queryExecution.analyzed,
- tableSpec,
- finalOptions,
- ignoreIfExists = createMode == SaveMode.Ignore)
- }
+ CreateTableAsSelect(
+ UnresolvedIdentifier(
+ catalog.name +: ident.namespace.toImmutableArraySeq :+ ident.name),
+ partitioningAsV2,
+ df.queryExecution.analyzed,
+ tableSpec,
+ finalOptions,
+ ignoreIfExists = createMode == SaveMode.Ignore)
case _: TableProvider =>
if (getTable.supports(BATCH_WRITE)) {
throw QueryCompilationErrors.writeWithSaveModeUnsupportedBySourceError(
@@ -233,13 +233,13 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) extends sql.DataFram
// Streaming also uses the data source V2 API. So it may be that the data source
// implements v2, but has no v2 implementation for batch writes. In that case, we
// fallback to saving as though it's a V1 source.
- saveToV1Source(path)
+ saveToV1SourceCommand(path)
}
}
}
} else {
- saveToV1Source(path)
+ saveToV1SourceCommand(path)
}
}
@@ -251,7 +251,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) extends sql.DataFram
}
}
- private def saveToV1Source(path: Option[String]): Unit = {
+ private def saveToV1SourceCommand(path: Option[String]): LogicalPlan = {
partitioningColumns.foreach { columns =>
extraOptions = extraOptions + (
DataSourceUtils.PARTITIONING_COLUMNS_KEY ->
@@ -266,13 +266,11 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) extends sql.DataFram
val optionsWithPath = getOptionsWithPath(path)
// Code path for data source v1.
- runCommand(df.sparkSession) {
- DataSource(
- sparkSession = df.sparkSession,
- className = source,
- partitionColumns = partitioningColumns.getOrElse(Nil),
- options = optionsWithPath.originalMap).planForWriting(curmode, df.logicalPlan)
- }
+ DataSource(
+ sparkSession = df.sparkSession,
+ className = source,
+ partitionColumns = partitioningColumns.getOrElse(Nil),
+ options = optionsWithPath.originalMap).planForWriting(curmode, df.logicalPlan)
}
/**
@@ -304,6 +302,12 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) extends sql.DataFram
* @since 1.4.0
*/
def insertInto(tableName: String): Unit = {
+ runCommand(df.sparkSession) {
+ insertIntoCommand(tableName)
+ }
+ }
+
+ private[sql] def insertIntoCommand(tableName: String): LogicalPlan = {
import df.sparkSession.sessionState.analyzer.{AsTableIdentifier, NonSessionCatalogAndIdentifier, SessionCatalogAndIdentifier}
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
@@ -318,30 +322,30 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) extends sql.DataFram
session.sessionState.sqlParser.parseMultipartIdentifier(tableName) match {
case NonSessionCatalogAndIdentifier(catalog, ident) =>
- insertInto(catalog, ident)
+ insertIntoCommand(catalog, ident)
case SessionCatalogAndIdentifier(catalog, ident)
if canUseV2 && ident.namespace().length <= 1 =>
- insertInto(catalog, ident)
+ insertIntoCommand(catalog, ident)
case AsTableIdentifier(tableIdentifier) =>
- insertInto(tableIdentifier)
+ insertIntoCommand(tableIdentifier)
case other =>
throw QueryCompilationErrors.cannotFindCatalogToHandleIdentifierError(other.quoted)
}
}
- private def insertInto(catalog: CatalogPlugin, ident: Identifier): Unit = {
+ private def insertIntoCommand(catalog: CatalogPlugin, ident: Identifier): LogicalPlan = {
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
val table = catalog.asTableCatalog.loadTable(ident, getWritePrivileges.toSet.asJava) match {
case _: V1Table =>
- return insertInto(TableIdentifier(ident.name(), ident.namespace().headOption))
+ return insertIntoCommand(TableIdentifier(ident.name(), ident.namespace().headOption))
case t =>
DataSourceV2Relation.create(t, Some(catalog), Some(ident))
}
- val command = curmode match {
+ curmode match {
case SaveMode.Append | SaveMode.ErrorIfExists | SaveMode.Ignore =>
AppendData.byPosition(table, df.logicalPlan, extraOptions.toMap)
@@ -356,22 +360,16 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) extends sql.DataFram
OverwriteByExpression.byPosition(table, df.logicalPlan, Literal(true), extraOptions.toMap)
}
}
-
- runCommand(df.sparkSession) {
- command
- }
}
- private def insertInto(tableIdent: TableIdentifier): Unit = {
- runCommand(df.sparkSession) {
- InsertIntoStatement(
- table = UnresolvedRelation(tableIdent).requireWritePrivileges(getWritePrivileges),
- partitionSpec = Map.empty[String, Option[String]],
- Nil,
- query = df.logicalPlan,
- overwrite = curmode == SaveMode.Overwrite,
- ifPartitionNotExists = false)
- }
+ private def insertIntoCommand(tableIdent: TableIdentifier): LogicalPlan = {
+ InsertIntoStatement(
+ table = UnresolvedRelation(tableIdent).requireWritePrivileges(getWritePrivileges),
+ partitionSpec = Map.empty[String, Option[String]],
+ Nil,
+ query = df.logicalPlan,
+ overwrite = curmode == SaveMode.Overwrite,
+ ifPartitionNotExists = false)
}
private def getWritePrivileges: Seq[TableWritePrivilege] = curmode match {
@@ -430,6 +428,12 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) extends sql.DataFram
* @since 1.4.0
*/
def saveAsTable(tableName: String): Unit = {
+ runCommand(df.sparkSession) {
+ saveAsTableCommand(tableName)
+ }
+ }
+
+ private[sql] def saveAsTableCommand(tableName: String): LogicalPlan = {
import df.sparkSession.sessionState.analyzer.{AsTableIdentifier, NonSessionCatalogAndIdentifier, SessionCatalogAndIdentifier}
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
@@ -440,30 +444,29 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) extends sql.DataFram
session.sessionState.sqlParser.parseMultipartIdentifier(tableName) match {
case nameParts @ NonSessionCatalogAndIdentifier(catalog, ident) =>
- saveAsTable(catalog.asTableCatalog, ident, nameParts)
+ saveAsTableCommand(catalog.asTableCatalog, ident, nameParts)
case nameParts @ SessionCatalogAndIdentifier(catalog, ident)
if canUseV2 && ident.namespace().length <= 1 =>
- saveAsTable(catalog.asTableCatalog, ident, nameParts)
+ saveAsTableCommand(catalog.asTableCatalog, ident, nameParts)
case AsTableIdentifier(tableIdentifier) =>
- saveAsTable(tableIdentifier)
+ saveAsV1TableCommand(tableIdentifier)
case other =>
throw QueryCompilationErrors.cannotFindCatalogToHandleIdentifierError(other.quoted)
}
}
-
- private def saveAsTable(
- catalog: TableCatalog, ident: Identifier, nameParts: Seq[String]): Unit = {
+ private def saveAsTableCommand(
+ catalog: TableCatalog, ident: Identifier, nameParts: Seq[String]): LogicalPlan = {
val tableOpt = try Option(catalog.loadTable(ident, getWritePrivileges.toSet.asJava)) catch {
case _: NoSuchTableException => None
}
- val command = (curmode, tableOpt) match {
+ (curmode, tableOpt) match {
case (_, Some(_: V1Table)) =>
- return saveAsTable(TableIdentifier(ident.name(), ident.namespace().headOption))
+ saveAsV1TableCommand(TableIdentifier(ident.name(), ident.namespace().headOption))
case (SaveMode.Append, Some(table)) =>
checkPartitioningMatchesV2Table(table)
@@ -512,56 +515,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) extends sql.DataFram
writeOptions = extraOptions.toMap,
other == SaveMode.Ignore)
}
-
- runCommand(df.sparkSession) {
- command
- }
- }
-
- private def saveAsTable(tableIdent: TableIdentifier): Unit = {
- val catalog = df.sparkSession.sessionState.catalog
- val qualifiedIdent = catalog.qualifyIdentifier(tableIdent)
- val tableExists = catalog.tableExists(qualifiedIdent)
-
- (tableExists, curmode) match {
- case (true, SaveMode.Ignore) =>
- // Do nothing
-
- case (true, SaveMode.ErrorIfExists) =>
- throw QueryCompilationErrors.tableAlreadyExistsError(qualifiedIdent)
-
- case (true, SaveMode.Overwrite) =>
- // Get all input data source or hive relations of the query.
- val srcRelations = df.logicalPlan.collect {
- case l: LogicalRelation => l.relation
- case relation: HiveTableRelation => relation.tableMeta.identifier
- }
-
- val tableRelation = df.sparkSession.table(qualifiedIdent).queryExecution.analyzed
- EliminateSubqueryAliases(tableRelation) match {
- // check if the table is a data source table (the relation is a BaseRelation).
- case l: LogicalRelation if srcRelations.contains(l.relation) =>
- throw QueryCompilationErrors.cannotOverwriteTableThatIsBeingReadFromError(
- qualifiedIdent)
- // check hive table relation when overwrite mode
- case relation: HiveTableRelation
- if srcRelations.contains(relation.tableMeta.identifier) =>
- throw QueryCompilationErrors.cannotOverwriteTableThatIsBeingReadFromError(
- qualifiedIdent)
- case _ => // OK
- }
-
- // Drop the existing table
- catalog.dropTable(qualifiedIdent, ignoreIfNotExists = true, purge = false)
- createTable(qualifiedIdent)
- // Refresh the cache of the table in the catalog.
- catalog.refreshTable(qualifiedIdent)
-
- case _ => createTable(qualifiedIdent)
- }
}
- private def createTable(tableIdent: TableIdentifier): Unit = {
+ private def saveAsV1TableCommand(tableIdent: TableIdentifier): SaveAsV1TableCommand = {
val storage = DataSource.buildStorageFormatFromOptions(extraOptions.toMap)
val tableType = if (storage.locationUri.isDefined) {
CatalogTableType.EXTERNAL
@@ -586,8 +542,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) extends sql.DataFram
bucketSpec = getBucketSpec,
properties = properties)
- runCommand(df.sparkSession)(
- CreateTable(tableDesc, curmode, Some(df.logicalPlan)))
+ SaveAsV1TableCommand(tableDesc, curmode, df.logicalPlan)
}
/** Converts the provided partitioning and bucketing information to DataSourceV2 Transforms. */
@@ -621,7 +576,8 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) extends sql.DataFram
* user-registered callback functions.
*/
private def runCommand(session: SparkSession)(command: LogicalPlan): Unit = {
- val qe = new QueryExecution(session, command, df.queryExecution.tracker)
+ val qe = new QueryExecution(session, command, df.queryExecution.tracker,
+ shuffleCleanupMode = QueryExecution.determineShuffleCleanupMode(session.sessionState.conf))
qe.assertCommandExecuted()
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/DataFrameWriterV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/DataFrameWriterV2.scala
index c6eacfe8f1ed9..7d481b86eb777 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/classic/DataFrameWriterV2.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/DataFrameWriterV2.scala
@@ -148,14 +148,17 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T])
/** @inheritdoc */
override def create(): Unit = {
- runCommand(
- CreateTableAsSelect(
- UnresolvedIdentifier(tableName),
- partitioning.getOrElse(Seq.empty) ++ clustering,
- logicalPlan,
- buildTableSpec(),
- options.toMap,
- false))
+ runCommand(createCommand())
+ }
+
+ private[sql] def createCommand(): LogicalPlan = {
+ CreateTableAsSelect(
+ UnresolvedIdentifier(tableName),
+ partitioning.getOrElse(Seq.empty) ++ clustering,
+ logicalPlan,
+ buildTableSpec(),
+ options.toMap,
+ false)
}
private def buildTableSpec(): UnresolvedTableSpec = {
@@ -186,28 +189,37 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T])
/** @inheritdoc */
@throws(classOf[NoSuchTableException])
def append(): Unit = {
- val append = AppendData.byName(
+ runCommand(appendCommand())
+ }
+
+ private[sql] def appendCommand(): LogicalPlan = {
+ AppendData.byName(
UnresolvedRelation(tableName).requireWritePrivileges(Seq(INSERT)),
logicalPlan, options.toMap)
- runCommand(append)
}
/** @inheritdoc */
@throws(classOf[NoSuchTableException])
def overwrite(condition: Column): Unit = {
- val overwrite = OverwriteByExpression.byName(
+ runCommand(overwriteCommand(condition))
+ }
+
+ private[sql] def overwriteCommand(condition: Column): LogicalPlan = {
+ OverwriteByExpression.byName(
UnresolvedRelation(tableName).requireWritePrivileges(Seq(INSERT, DELETE)),
logicalPlan, expression(condition), options.toMap)
- runCommand(overwrite)
}
/** @inheritdoc */
@throws(classOf[NoSuchTableException])
def overwritePartitions(): Unit = {
- val dynamicOverwrite = OverwritePartitionsDynamic.byName(
+ runCommand(overwritePartitionsCommand())
+ }
+
+ private[sql] def overwritePartitionsCommand(): LogicalPlan = {
+ OverwritePartitionsDynamic.byName(
UnresolvedRelation(tableName).requireWritePrivileges(Seq(INSERT, DELETE)),
logicalPlan, options.toMap)
- runCommand(dynamicOverwrite)
}
/**
@@ -215,18 +227,24 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T])
* callback functions.
*/
private def runCommand(command: LogicalPlan): Unit = {
- val qe = new QueryExecution(sparkSession, command, df.queryExecution.tracker)
+ val qe = new QueryExecution(sparkSession, command, df.queryExecution.tracker,
+ shuffleCleanupMode =
+ QueryExecution.determineShuffleCleanupMode(sparkSession.sessionState.conf))
qe.assertCommandExecuted()
}
private def internalReplace(orCreate: Boolean): Unit = {
- runCommand(ReplaceTableAsSelect(
+ runCommand(replaceCommand(orCreate))
+ }
+
+ private[sql] def replaceCommand(orCreate: Boolean): LogicalPlan = {
+ ReplaceTableAsSelect(
UnresolvedIdentifier(tableName),
partitioning.getOrElse(Seq.empty) ++ clustering,
logicalPlan,
buildTableSpec(),
writeOptions = options.toMap,
- orCreate = orCreate))
+ orCreate = orCreate)
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/Dataset.scala
index 8327d81816191..e2688c7ddab12 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/classic/Dataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/Dataset.scala
@@ -26,10 +26,9 @@ import scala.reflect.ClassTag
import scala.reflect.runtime.universe.TypeTag
import scala.util.control.NonFatal
-import org.apache.commons.lang3.StringUtils
import org.apache.commons.text.StringEscapeUtils
-import org.apache.spark.{sql, TaskContext}
+import org.apache.spark.{sql, SparkException, TaskContext}
import org.apache.spark.annotation.{DeveloperApi, Stable, Unstable}
import org.apache.spark.api.java.JavaRDD
import org.apache.spark.api.java.function._
@@ -396,15 +395,15 @@ class Dataset[T] private[sql](
val paddedRows = rows.map { row =>
row.zipWithIndex.map { case (cell, i) =>
if (truncate > 0) {
- StringUtils.leftPad(cell, colWidths(i) - Utils.stringHalfWidth(cell) + cell.length)
+ Utils.leftPad(cell, colWidths(i) - Utils.stringHalfWidth(cell) + cell.length)
} else {
- StringUtils.rightPad(cell, colWidths(i) - Utils.stringHalfWidth(cell) + cell.length)
+ Utils.rightPad(cell, colWidths(i) - Utils.stringHalfWidth(cell) + cell.length)
}
}
}
// Create SeparateLine
- val sep: String = colWidths.map("-" * _).addString(sb, "+", "+", "+\n").toString()
+ val sep: String = colWidths.map("-".repeat(_)).addString(sb, "+", "+", "+\n").toString()
// column names
paddedRows.head.addString(sb, "|", "|", "|\n")
@@ -428,13 +427,13 @@ class Dataset[T] private[sql](
dataRows.zipWithIndex.foreach { case (row, i) =>
// "+ 5" in size means a character length except for padded names and data
- val rowHeader = StringUtils.rightPad(
+ val rowHeader = Utils.rightPad(
s"-RECORD $i", fieldNameColWidth + dataColWidth + 5, "-")
sb.append(rowHeader).append("\n")
row.zipWithIndex.map { case (cell, j) =>
- val fieldName = StringUtils.rightPad(fieldNames(j),
+ val fieldName = Utils.rightPad(fieldNames(j),
fieldNameColWidth - Utils.stringHalfWidth(fieldNames(j)) + fieldNames(j).length)
- val data = StringUtils.rightPad(cell,
+ val data = Utils.rightPad(cell,
dataColWidth - Utils.stringHalfWidth(cell) + cell.length)
s" $fieldName | $data "
}.addString(sb, "", "\n", "\n")
@@ -650,7 +649,7 @@ class Dataset[T] private[sql](
private def resolveSelfJoinCondition(
right: Dataset[_],
joinExprs: Option[Column],
- joinType: String): Join = {
+ joinType: String): LogicalPlan = {
// Note that in this function, we introduce a hack in the case of self-join to automatically
// resolve ambiguous join conditions into ones that might make sense [SPARK-6231].
// Consider this case: df.join(df, df("key") === df("key"))
@@ -661,28 +660,40 @@ class Dataset[T] private[sql](
// Trigger analysis so in the case of self-join, the analyzer will clone the plan.
// After the cloning, left and right side will have distinct expression ids.
- val plan = withPlan(
- Join(logicalPlan, right.logicalPlan,
- JoinType(joinType), joinExprs.map(_.expr), JoinHint.NONE))
- .queryExecution.analyzed.asInstanceOf[Join]
+ val planToAnalyze = Join(
+ logicalPlan, right.logicalPlan, JoinType(joinType), joinExprs.map(_.expr), JoinHint.NONE)
+ val analyzedJoinPlan = withPlan(planToAnalyze).queryExecution.analyzed
// If auto self join alias is disabled, return the plan.
if (!sparkSession.sessionState.conf.dataFrameSelfJoinAutoResolveAmbiguity) {
- return plan
+ return analyzedJoinPlan
}
// If left/right have no output set intersection, return the plan.
val lanalyzed = this.queryExecution.analyzed
val ranalyzed = right.queryExecution.analyzed
if (lanalyzed.outputSet.intersect(ranalyzed.outputSet).isEmpty) {
- return plan
+ return analyzedJoinPlan
}
// Otherwise, find the trivially true predicates and automatically resolves them to both sides.
// By the time we get here, since we have already run analysis, all attributes should've been
// resolved and become AttributeReference.
-
- JoinWith.resolveSelfJoinCondition(sparkSession.sessionState.analyzer.resolver, plan)
+ analyzedJoinPlan match {
+ case project @ Project(_, join: Join) =>
+ // SPARK-53143: Handling edge-cases when `AddMetadataColumns` analyzer rule adds `Project`
+ // node on top of `Join` node.
+ // Check "SPARK-53143: self join edge-case when Join is not returned by the analyzer" in
+ // `DataframeSelfJoinSuite` for more details.
+ val newProject = project.copy(child = JoinWith.resolveSelfJoinCondition(
+ sparkSession.sessionState.analyzer.resolver, join))
+ newProject.copyTagsFrom(project)
+ newProject
+ case join: Join =>
+ JoinWith.resolveSelfJoinCondition(sparkSession.sessionState.analyzer.resolver, join)
+ case _ => throw SparkException.internalError(
+ s"Unexpected plan type: ${analyzedJoinPlan.getClass.getName} for self join resolution.")
+ }
}
/** @inheritdoc */
@@ -782,28 +793,38 @@ class Dataset[T] private[sql](
tolerance: Column,
allowExactMatches: Boolean,
direction: String): DataFrame = {
- val joined = resolveSelfJoinCondition(other, Option(joinExprs), joinType)
- val leftAsOfExpr = leftAsOf.expr.transformUp {
- case a: AttributeReference if logicalPlan.outputSet.contains(a) =>
- val index = logicalPlan.output.indexWhere(_.exprId == a.exprId)
- joined.left.output(index)
- }
- val rightAsOfExpr = rightAsOf.expr.transformUp {
- case a: AttributeReference if other.logicalPlan.outputSet.contains(a) =>
- val index = other.logicalPlan.output.indexWhere(_.exprId == a.exprId)
- joined.right.output(index)
- }
- withPlan {
+
+ def createAsOfJoinPlan(joinPlan: Join): AsOfJoin = {
+ val leftAsOfExpr = leftAsOf.expr.transformUp {
+ case a: AttributeReference if logicalPlan.outputSet.contains(a) =>
+ val index = logicalPlan.output.indexWhere(_.exprId == a.exprId)
+ joinPlan.left.output(index)
+ }
+ val rightAsOfExpr = rightAsOf.expr.transformUp {
+ case a: AttributeReference if other.logicalPlan.outputSet.contains(a) =>
+ val index = other.logicalPlan.output.indexWhere(_.exprId == a.exprId)
+ joinPlan.right.output(index)
+ }
AsOfJoin(
- joined.left, joined.right,
+ joinPlan.left, joinPlan.right,
leftAsOfExpr, rightAsOfExpr,
- joined.condition,
- joined.joinType,
+ joinPlan.condition,
+ joinPlan.joinType,
Option(tolerance).map(_.expr),
allowExactMatches,
AsOfJoinDirection(direction)
)
}
+
+ resolveSelfJoinCondition(other, Option(joinExprs), joinType) match {
+ case project @ Project(_, join: Join) =>
+ val newProjectPlan = project.copy(child = createAsOfJoinPlan(join))
+ newProjectPlan.copyTagsFrom(project)
+ withPlan { newProjectPlan }
+ case join: Join => withPlan { createAsOfJoinPlan(join) }
+ case plan => throw SparkException.internalError(
+ s"Unexpected plan type: ${plan.getClass.getName} returned from self join resolution.")
+ }
}
/** @inheritdoc */
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/MergeIntoWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/MergeIntoWriter.scala
index 0269b15061c97..e3c872658c86a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/classic/MergeIntoWriter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/MergeIntoWriter.scala
@@ -57,13 +57,18 @@ class MergeIntoWriter[T] private[sql](table: String, ds: Dataset[T], on: Column)
/** @inheritdoc */
def merge(): Unit = {
+ val qe = sparkSession.sessionState.executePlan(mergeCommand())
+ qe.assertCommandExecuted()
+ }
+
+ private[sql] def mergeCommand(): LogicalPlan = {
if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
throw new SparkRuntimeException(
errorClass = "NO_MERGE_ACTION_SPECIFIED",
messageParameters = Map.empty)
}
- val merge = MergeIntoTable(
+ MergeIntoTable(
UnresolvedRelation(tableName).requireWritePrivileges(MergeIntoTable.getWritePrivileges(
matchedActions, notMatchedActions, notMatchedBySourceActions)),
logicalPlan,
@@ -72,8 +77,6 @@ class MergeIntoWriter[T] private[sql](table: String, ds: Dataset[T], on: Column)
notMatchedActions.toSeq,
notMatchedBySourceActions.toSeq,
schemaEvolutionEnabled)
- val qe = sparkSession.sessionState.executePlan(merge)
- qe.assertCommandExecuted()
}
override protected[sql] def insertAll(condition: Option[Column]): this.type = {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/RelationalGroupedDataset.scala
index 0fa6e91e21459..0b1da71be9ede 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/classic/RelationalGroupedDataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/RelationalGroupedDataset.scala
@@ -444,8 +444,8 @@ class RelationalGroupedDataset protected[sql](
stateStructType: StructType,
outputModeStr: String,
timeoutConfStr: String): DataFrame = {
- val timeoutConf = org.apache.spark.sql.execution.streaming
- .GroupStateImpl.groupStateTimeoutFromString(timeoutConfStr)
+ val timeoutConf = org.apache.spark.sql.execution.streaming.operators.stateful.
+ flatmapgroupswithstate.GroupStateImpl.groupStateTimeoutFromString(timeoutConfStr)
val outputMode = InternalOutputModes(outputModeStr)
if (outputMode != OutputMode.Append && outputMode != OutputMode.Update) {
throw new IllegalArgumentException("The output mode of function should be append or update")
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala
index 0015d7ff99e2b..681e1b16af590 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala
@@ -31,7 +31,7 @@ import scala.util.control.NonFatal
import org.apache.spark.{SPARK_VERSION, SparkConf, SparkContext, SparkException, TaskContext}
import org.apache.spark.annotation.{DeveloperApi, Experimental, Stable, Unstable}
import org.apache.spark.api.java.JavaRDD
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{CALL_SITE_LONG_FORM, CLASS_NAME, CONFIG}
import org.apache.spark.internal.config.{ConfigEntry, EXECUTOR_ALLOW_SPARK_CONTEXT}
import org.apache.spark.rdd.RDD
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/StreamingQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/StreamingQueryManager.scala
index 6d4a3ecd36037..0470f3b20ecc8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/classic/StreamingQueryManager.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/StreamingQueryManager.scala
@@ -25,7 +25,7 @@ import scala.collection.mutable
import scala.jdk.CollectionConverters._
import org.apache.spark.annotation.Evolving
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{CLASS_NAME, QUERY_ID, RUN_ID}
import org.apache.spark.sql.catalyst.catalog.CatalogTable
import org.apache.spark.sql.catalyst.streaming.{WriteToStream, WriteToStreamStatement}
@@ -33,6 +33,7 @@ import org.apache.spark.sql.connector.catalog.{Identifier, SupportsWrite, Table,
import org.apache.spark.sql.errors.QueryExecutionErrors
import org.apache.spark.sql.execution.streaming._
import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution
+import org.apache.spark.sql.execution.streaming.runtime.{AsyncProgressTrackingMicroBatchExecution, MicroBatchExecution, StreamingQueryListenerBus, StreamingQueryWrapper}
import org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorRef
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.StaticSQLConf.STREAMING_QUERY_LISTENERS
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
index 885ddf4110cbb..2f1b79ec90f99 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
@@ -18,12 +18,14 @@
package org.apache.spark.sql.columnar
import org.apache.spark.annotation.{DeveloperApi, Since}
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{FILTER, PREDICATE}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeReference, BindReferences, EqualNullSafe, EqualTo, Expression, GreaterThan, GreaterThanOrEqual, In, IsNotNull, IsNull, Length, LessThan, LessThanOrEqual, Literal, Or, Predicate, StartsWith}
+import org.apache.spark.sql.execution.{ColumnarToRowTransition, InputAdapter, SparkPlan, WholeStageCodegenExec}
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec
import org.apache.spark.sql.execution.columnar.{ColumnStatisticsSchema, PartitionStatistics}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{AtomicType, BinaryType, StructType}
@@ -58,6 +60,40 @@ trait CachedBatchSerializer extends Serializable {
*/
def supportsColumnarInput(schema: Seq[Attribute]): Boolean
+ /**
+ * Attempt to convert a query plan to its columnar equivalence for columnar caching.
+ * Called on the query plan that is about to be cached once [[supportsColumnarInput]] returns
+ * true on its output schema.
+ *
+ * The default implementation works by stripping the topmost columnar-to-row transition to
+ * expose the columnar-based plan to the serializer.
+ *
+ * @param plan The plan to convert.
+ * @return The output plan. Could either be a columnar plan if the input plan is convertible, or
+ * the input plan unchanged if no viable conversion can be done.
+ */
+ @DeveloperApi
+ @Since("4.1.0")
+ def convertToColumnarPlanIfPossible(plan: SparkPlan): SparkPlan = plan match {
+ case gen: WholeStageCodegenExec =>
+ gen.child match {
+ case c2r: ColumnarToRowTransition =>
+ c2r.child match {
+ case ia: InputAdapter => ia.child
+ case _ => plan
+ }
+ case _ => plan
+ }
+ case c2r: ColumnarToRowTransition => // This matches when whole stage code gen is disabled.
+ c2r.child
+ case adaptive: AdaptiveSparkPlanExec =>
+ // If AQE is enabled for cached plan and table cache supports columnar in, we should mark
+ // `AdaptiveSparkPlanExec.supportsColumnar` as true to avoid inserting `ColumnarToRow`, so
+ // that `CachedBatchSerializer` can use `convertColumnarBatchToCachedBatch` to cache data.
+ adaptive.copy(supportsColumnar = true)
+ case _ => plan
+ }
+
/**
* Convert an `RDD[InternalRow]` into an `RDD[CachedBatch]` in preparation for caching the data.
* @param input the input `RDD` to be converted.
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala
index 60156bff1fb71..bfd813ad5ef11 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala
@@ -27,7 +27,7 @@ import scala.util.control.NonFatal
import org.apache.hadoop.conf.Configuration
import org.apache.spark.{SparkFiles, TaskContext}
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys._
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala
index 5b68be2c4ce55..a8292a8dbaa3b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.execution
import org.apache.hadoop.fs.{FileSystem, Path}
-import org.apache.spark.internal.{Logging, MDC, MessageWithContext}
+import org.apache.spark.internal.{Logging, MessageWithContext}
import org.apache.spark.internal.LogKeys._
import org.apache.spark.sql.catalyst.catalog.HiveTableRelation
import org.apache.spark.sql.catalyst.expressions.{Attribute, SubqueryExpression}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala
index 4c9ae155ec17a..f9193cd0495f6 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala
@@ -32,7 +32,6 @@ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
import org.apache.spark.sql.execution.vectorized.WritableColumnVector
import org.apache.spark.sql.types._
import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
-import org.apache.spark.util.Utils
/**
* Holds a user defined rule that can be used to inject columnar implementations of various
@@ -66,9 +65,6 @@ trait ColumnarToRowTransition extends UnaryExecNode
* [[MapPartitionsInRWithArrowExec]]. Eventually this should replace those implementations.
*/
case class ColumnarToRowExec(child: SparkPlan) extends ColumnarToRowTransition with CodegenSupport {
- // supportsColumnar requires to be only called on driver side, see also SPARK-37779.
- assert(Utils.isInRunningSparkTask || child.supportsColumnar)
-
override def output: Seq[Attribute] = child.output
override def outputPartitioning: Partitioning = child.outputPartitioning
@@ -500,33 +496,31 @@ case class ApplyColumnarRulesAndInsertTransitions(
extends Rule[SparkPlan] {
/**
- * Inserts an transition to columnar formatted data.
+ * Ensures columnar output on the input query plan. Transitions will be inserted
+ * on demand.
*/
- private def insertRowToColumnar(plan: SparkPlan): SparkPlan = {
+ private def ensureOutputsColumnar(plan: SparkPlan): SparkPlan = {
if (!plan.supportsColumnar) {
// The tree feels kind of backwards
// Columnar Processing will start here, so transition from row to columnar
- RowToColumnarExec(insertTransitions(plan, outputsColumnar = false))
+ RowToColumnarExec(ensureOutputsRowBased(plan))
} else if (!plan.isInstanceOf[RowToColumnarTransition]) {
- plan.withNewChildren(plan.children.map(insertRowToColumnar))
+ plan.withNewChildren(plan.children.map(ensureOutputsColumnar))
} else {
plan
}
}
/**
- * Inserts RowToColumnarExecs and ColumnarToRowExecs where needed.
+ * Ensures row-based output on the input query plan. Transitions will be inserted
+ * on demand.
*/
- private def insertTransitions(plan: SparkPlan, outputsColumnar: Boolean): SparkPlan = {
- if (outputsColumnar) {
- insertRowToColumnar(plan)
- } else if (plan.supportsColumnar && !plan.supportsRowBased) {
+ private def ensureOutputsRowBased(plan: SparkPlan): SparkPlan = {
+ if (plan.supportsColumnar && !plan.supportsRowBased) {
// `outputsColumnar` is false but the plan only outputs columnar format, so add a
// to-row transition here.
- ColumnarToRowExec(insertRowToColumnar(plan))
- } else if (plan.isInstanceOf[ColumnarToRowTransition]) {
- plan
- } else {
+ ColumnarToRowExec(ensureOutputsColumnar(plan))
+ } else if (!plan.isInstanceOf[ColumnarToRowTransition]) {
val outputsColumnar = plan match {
// With planned write, the write command invokes child plan's `executeWrite` which is
// neither columnar nor row-based.
@@ -541,6 +535,19 @@ case class ApplyColumnarRulesAndInsertTransitions(
false
}
plan.withNewChildren(plan.children.map(insertTransitions(_, outputsColumnar)))
+ } else {
+ plan
+ }
+ }
+
+ /**
+ * Inserts RowToColumnarExecs and ColumnarToRowExecs where needed.
+ */
+ private def insertTransitions(plan: SparkPlan, outputsColumnar: Boolean): SparkPlan = {
+ if (outputsColumnar) {
+ ensureOutputsColumnar(plan)
+ } else {
+ ensureOutputsRowBased(plan)
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala
index 31ab367c2d003..66e07aa4f7d44 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala
@@ -19,11 +19,9 @@ package org.apache.spark.sql.execution
import java.util.concurrent.TimeUnit._
-import org.apache.commons.lang3.StringUtils
import org.apache.hadoop.fs.Path
import org.apache.spark.internal.LogKeys.{COUNT, MAX_SPLIT_BYTES, OPEN_COST_IN_BYTES}
-import org.apache.spark.internal.MDC
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.{FileSourceOptions, InternalRow, TableIdentifier}
import org.apache.spark.sql.catalyst.catalog.BucketSpec
@@ -36,7 +34,7 @@ import org.apache.spark.sql.errors.QueryExecutionErrors
import org.apache.spark.sql.execution
import org.apache.spark.sql.execution.datasources._
import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat => ParquetSource}
-import org.apache.spark.sql.execution.datasources.v2.PushedDownOperators
+import org.apache.spark.sql.execution.datasources.v2.{PushedDownOperators, TableSampleInfo}
import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
import org.apache.spark.sql.execution.vectorized.ConstantColumnVector
import org.apache.spark.sql.internal.SQLConf
@@ -65,7 +63,7 @@ trait DataSourceScanExec extends LeafExecNode with StreamSourceAwareSparkPlan {
override def simpleString(maxFields: Int): String = {
val metadataEntries = metadata.toSeq.sorted.map {
case (key, value) =>
- key + ": " + StringUtils.abbreviate(redact(value), maxMetadataValueLength)
+ key + ": " + Utils.abbreviate(redact(value), maxMetadataValueLength)
}
val metadataStr = truncatedString(metadataEntries, " ", ", ", "", maxFields)
redact(
@@ -159,10 +157,12 @@ case class RowDataSourceScanExec(
override def inputRDD: RDD[InternalRow] = rdd
- override val metadata: Map[String, String] = {
+ private def seqToString(seq: Seq[Any]): String = seq.mkString("[", ", ", "]")
- def seqToString(seq: Seq[Any]): String = seq.mkString("[", ", ", "]")
+ private def pushedSampleMetadataString(s: TableSampleInfo): String =
+ s"SAMPLE (${(s.upperBound - s.lowerBound) * 100}) ${s.withReplacement} SEED(${s.seed})"
+ override val metadata: Map[String, String] = {
val markedFilters = if (filters.nonEmpty) {
for (filter <- filters) yield {
if (handledFilters.contains(filter)) s"*$filter" else s"$filter"
@@ -189,6 +189,15 @@ case class RowDataSourceScanExec(
seqToString(markedFilters.toSeq)
}
+ val pushedJoins = if (pushedDownOperators.joinedRelationPushedDownOperators.nonEmpty) {
+ Map("PushedJoins" ->
+ s"\n${getPushedJoinString(
+ pushedDownOperators.joinedRelationPushedDownOperators(0),
+ pushedDownOperators.joinedRelationPushedDownOperators(1))}\n")
+ } else {
+ Map()
+ }
+
Map("ReadSchema" -> requiredSchema.catalogString,
"PushedFilters" -> pushedFilters) ++
pushedDownOperators.aggregation.fold(Map[String, String]()) { v =>
@@ -198,9 +207,78 @@ case class RowDataSourceScanExec(
seqToString(v.groupByExpressions.map(_.describe()).toImmutableArraySeq))} ++
topNOrLimitInfo ++
offsetInfo ++
- pushedDownOperators.sample.map(v => "PushedSample" ->
- s"SAMPLE (${(v.upperBound - v.lowerBound) * 100}) ${v.withReplacement} SEED(${v.seed})"
- )
+ pushedDownOperators.sample.map(v => "PushedSample" -> pushedSampleMetadataString(v)) ++
+ pushedJoins
+ }
+
+ /**
+ * Build string for all the pushed down join operators. The method is recursive, so if there is
+ * join on top of 2 already joined relations, all of these will be present in string.
+ *
+ * The exmaple of resulting string is the following:
+ *
+ * PushedFilters: [id_3 = (id_4 + 1)], PushedJoins:
+ * [L]: PushedFilters: [ID_1 = (id_3 + 1)]
+ * PushedJoins:
+ * [L]: PushedFilters: [ID = (ID_1 + 1)]
+ * PushedJoins:
+ * [L]: Relation: join_pushdown_catalog.JOIN_SCHEMA.JOIN_TABLE_1
+ * PushedFilters: [ID IS NOT NULL]
+ * [R]: Relation: join_pushdown_catalog.JOIN_SCHEMA.JOIN_TABLE_2
+ * PushedFilters: [ID IS NOT NULL]
+ * [R]: Relation: join_pushdown_catalog.JOIN_SCHEMA.JOIN_TABLE_3
+ * PushedFilters: [id IS NOT NULL]
+ * [R]: Relation: join_pushdown_catalog.JOIN_SCHEMA.JOIN_TABLE_4
+ * PushedFilters: [id IS NOT NULL]
+ */
+ private def getPushedJoinString(
+ leftSidePushedDownOperators: PushedDownOperators,
+ rightSidePushedDownOperators: PushedDownOperators,
+ indent: Int = 0): String = {
+ val indentStr = " ".repeat(indent)
+
+ val leftSideOperators = buildOperatorParts(leftSidePushedDownOperators, indent)
+ val leftSideMetadataStr = formatMetadata(leftSideOperators, indentStr + " ".repeat(5))
+
+ val rightSideOperators = buildOperatorParts(rightSidePushedDownOperators, indent)
+ val rightSideMetadataStr = formatMetadata(rightSideOperators, indentStr + " ".repeat(5))
+
+ val leftSideString = s"$indentStr[L]: $leftSideMetadataStr"
+ val rightSideString = s"$indentStr[R]: $rightSideMetadataStr"
+ Seq(leftSideString, rightSideString).mkString("\n")
+ }
+
+ private def buildOperatorParts(operators: PushedDownOperators, indent: Int): List[String] = {
+ val parts = List.newBuilder[String]
+
+ // Add relation name for leaf nodes (nodes without further joins)
+ if (operators.joinedRelationPushedDownOperators.isEmpty) {
+ operators.relationName.foreach(name => parts += s"Relation: $name")
+ }
+
+ if (operators.pushedPredicates.nonEmpty) {
+ parts += s"PushedFilters: ${seqToString(operators.pushedPredicates.map(_.describe()))}"
+ }
+
+ operators.sample.foreach { sample =>
+ parts += s"PushedSample: ${pushedSampleMetadataString(sample)}"
+ }
+
+ // Recursively get the pushed join string for child with correct indentation.
+ if (operators.joinedRelationPushedDownOperators.nonEmpty) {
+ val nestedJoins = getPushedJoinString(
+ operators.joinedRelationPushedDownOperators(0),
+ operators.joinedRelationPushedDownOperators(1),
+ indent + 5)
+ parts += s"PushedJoins:\n$nestedJoins"
+ }
+
+ parts.result()
+ }
+
+ private def formatMetadata(parts: List[String], indentStr: String): String = {
+ val (basicParts, nestedJoinsParts) = parts.partition(!_.startsWith("PushedJoins:"))
+ (basicParts ++ nestedJoinsParts).mkString("\n" + indentStr)
}
// Don't care about `rdd` and `tableIdentifier`, and `stream` when canonicalizing.
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/EmptyRelationExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/EmptyRelationExec.scala
index a0c3d7b51c2c3..70c9e5359e2ab 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/EmptyRelationExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/EmptyRelationExec.scala
@@ -61,6 +61,7 @@ case class EmptyRelationExec(@transient logical: LogicalPlan) extends LeafExecNo
addSuffix: Boolean = false,
maxFields: Int,
printNodeId: Boolean,
+ printOutputColumns: Boolean,
indent: Int = 0): Unit = {
super.generateTreeString(depth,
lastChildren,
@@ -70,11 +71,13 @@ case class EmptyRelationExec(@transient logical: LogicalPlan) extends LeafExecNo
addSuffix,
maxFields,
printNodeId,
+ printOutputColumns,
indent)
Option(logical).foreach { _ =>
lastChildren.add(true)
logical.generateTreeString(
- depth + 1, lastChildren, append, verbose, "", false, maxFields, printNodeId, indent)
+ depth + 1, lastChildren, append, verbose, "", false, maxFields, printNodeId,
+ printOutputColumns, indent)
lastChildren.remove(lastChildren.size() - 1)
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
index 6000b3c20251e..6148fb30783e8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
@@ -17,7 +17,7 @@
package org.apache.spark.sql.execution
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{LOGICAL_PLAN_COLUMNS, OPTIMIZED_PLAN_COLUMNS}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.Encoder
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArray.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArray.scala
index 59810adc4b22e..3e98c28b29fbc 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArray.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArray.scala
@@ -22,7 +22,7 @@ import java.io.Closeable
import scala.collection.mutable.ArrayBuffer
import org.apache.spark.{SparkEnv, TaskContext}
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{CLASS_NAME, MAX_NUM_ROWS_IN_MEMORY_BUFFER}
import org.apache.spark.memory.TaskMemoryManager
import org.apache.spark.serializer.SerializerManager
@@ -52,9 +52,12 @@ private[sql] class ExternalAppendOnlyUnsafeRowArray(
initialSize: Int,
pageSizeBytes: Long,
numRowsInMemoryBufferThreshold: Int,
- numRowsSpillThreshold: Int) extends Logging {
+ numRowsSpillThreshold: Int,
+ maxSizeSpillThreshold: Long) extends Logging {
- def this(numRowsInMemoryBufferThreshold: Int, numRowsSpillThreshold: Int) = {
+ def this(numRowsInMemoryBufferThreshold: Int,
+ numRowsSpillThreshold: Int,
+ maxSizeSpillThreshold: Long) = {
this(
TaskContext.get().taskMemoryManager(),
SparkEnv.get.blockManager,
@@ -63,7 +66,8 @@ private[sql] class ExternalAppendOnlyUnsafeRowArray(
1024,
SparkEnv.get.memoryManager.pageSizeBytes,
numRowsInMemoryBufferThreshold,
- numRowsSpillThreshold)
+ numRowsSpillThreshold,
+ maxSizeSpillThreshold)
}
private val initialSizeOfInMemoryBuffer =
@@ -138,6 +142,7 @@ private[sql] class ExternalAppendOnlyUnsafeRowArray(
initialSize,
pageSizeBytes,
numRowsSpillThreshold,
+ maxSizeSpillThreshold,
false)
// populate with existing in-memory buffered rows
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala
index 21cf70dab59f7..3a7b75a555af1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala
@@ -149,6 +149,6 @@ object HiveResult extends SQLConfHelper {
startField,
endField)
case (v: VariantVal, VariantType) => v.toString
- case (other, _: UserDefinedType[_]) => other.toString
+ case (other, u: UserDefinedType[_]) => u.stringifyValue(other)
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/KeyGroupedPartitionedScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/KeyGroupedPartitionedScan.scala
new file mode 100644
index 0000000000000..5a789179219ad
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/KeyGroupedPartitionedScan.scala
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.InternalRow
+import org.apache.spark.sql.catalyst.expressions.RowOrdering
+import org.apache.spark.sql.catalyst.plans.physical.{KeyGroupedPartitioning, KeyGroupedShuffleSpec}
+import org.apache.spark.sql.catalyst.util.InternalRowComparableWrapper
+import org.apache.spark.sql.execution.joins.StoragePartitionJoinParams
+
+/** Base trait for a data source scan capable of producing a key-grouped output. */
+trait KeyGroupedPartitionedScan[T] {
+ /**
+ * The output partitioning of this scan after applying any pushed-down SPJ parameters.
+ *
+ * @param basePartitioning The original key-grouped partitioning of the scan.
+ * @param spjParams SPJ parameters for the scan.
+ */
+ def getOutputKeyGroupedPartitioning(
+ basePartitioning: KeyGroupedPartitioning,
+ spjParams: StoragePartitionJoinParams): KeyGroupedPartitioning = {
+ val expressions = spjParams.joinKeyPositions match {
+ case Some(projectionPositions) =>
+ projectionPositions.map(i => basePartitioning.expressions(i))
+ case _ => basePartitioning.expressions
+ }
+
+ val newPartValues = spjParams.commonPartitionValues match {
+ case Some(commonPartValues) =>
+ // We allow duplicated partition values if
+ // `spark.sql.sources.v2.bucketing.partiallyClusteredDistribution.enabled` is true
+ commonPartValues.flatMap {
+ case (partValue, numSplits) => Seq.fill(numSplits)(partValue)
+ }
+ case None =>
+ spjParams.joinKeyPositions match {
+ case Some(projectionPositions) => basePartitioning.partitionValues.map { r =>
+ val projectedRow = KeyGroupedPartitioning.project(expressions,
+ projectionPositions, r)
+ InternalRowComparableWrapper(projectedRow, expressions)
+ }.distinct.map(_.row)
+ case _ => basePartitioning.partitionValues
+ }
+ }
+ basePartitioning.copy(expressions = expressions, numPartitions = newPartValues.length,
+ partitionValues = newPartValues)
+ }
+
+ /**
+ * Re-groups the input partitions for this scan based on the provided SPJ params, returning a list
+ * of partitions to be scanned by each scan task.
+ *
+ * @param p The output KeyGroupedPartitioning of this scan.
+ * @param spjParams SPJ parameters for the scan.
+ * @param filteredPartitions The input partitions (after applying filtering) to be
+ * re-grouped for this scan, initially grouped by partition value.
+ * @param partitionValueAccessor Accessor for the partition values (as an [[InternalRow]])
+ */
+ def getInputPartitionGrouping(
+ p: KeyGroupedPartitioning,
+ spjParams: StoragePartitionJoinParams,
+ filteredPartitions: Seq[Seq[T]],
+ partitionValueAccessor: T => InternalRow): Seq[Seq[T]] = {
+ assert(spjParams.keyGroupedPartitioning.isDefined)
+ val expressions = spjParams.keyGroupedPartitioning.get
+
+ // Re-group the input partitions if we are projecting on a subset of join keys
+ val (groupedPartitions, partExpressions) = spjParams.joinKeyPositions match {
+ case Some(projectPositions) =>
+ val projectedExpressions = projectPositions.map(i => expressions(i))
+ val parts = filteredPartitions.flatten.groupBy(part => {
+ val row = partitionValueAccessor(part)
+ val projectedRow = KeyGroupedPartitioning.project(
+ expressions, projectPositions, row)
+ InternalRowComparableWrapper(projectedRow, projectedExpressions)
+ }).map { case (wrapper, splits) => (wrapper.row, splits) }.toSeq
+ (parts, projectedExpressions)
+ case _ =>
+ val groupedParts = filteredPartitions.map(splits => {
+ assert(splits.nonEmpty)
+ (partitionValueAccessor(splits.head), splits)
+ })
+ (groupedParts, expressions)
+ }
+
+ // Also re-group the partitions if we are reducing compatible partition expressions
+ val finalGroupedPartitions = spjParams.reducers match {
+ case Some(reducers) =>
+ val result = groupedPartitions.groupBy { case (row, _) =>
+ KeyGroupedShuffleSpec.reducePartitionValue(row, partExpressions, reducers)
+ }.map { case (wrapper, splits) => (wrapper.row, splits.flatMap(_._2)) }.toSeq
+ val rowOrdering = RowOrdering.createNaturalAscendingOrdering(
+ partExpressions.map(_.dataType))
+ result.sorted(rowOrdering.on((t: (InternalRow, _)) => t._1))
+ case _ => groupedPartitions
+ }
+
+ // When partially clustered, the input partitions are not grouped by partition
+ // values. Here we'll need to check `commonPartitionValues` and decide how to group
+ // and replicate splits within a partition.
+ if (spjParams.commonPartitionValues.isDefined && spjParams.applyPartialClustering) {
+ // A mapping from the common partition values to how many splits the partition
+ // should contain.
+ val commonPartValuesMap = spjParams.commonPartitionValues
+ .get
+ .map(t => (InternalRowComparableWrapper(t._1, partExpressions), t._2))
+ .toMap
+ val filteredGroupedPartitions = finalGroupedPartitions.filter {
+ case (partValues, _) =>
+ commonPartValuesMap.keySet.contains(
+ InternalRowComparableWrapper(partValues, partExpressions))
+ }
+ val nestGroupedPartitions = filteredGroupedPartitions.map { case (partValue, splits) =>
+ // `commonPartValuesMap` should contain the part value since it's the super set.
+ val numSplits = commonPartValuesMap
+ .get(InternalRowComparableWrapper(partValue, partExpressions))
+ assert(numSplits.isDefined, s"Partition value $partValue does not exist in " +
+ "common partition values from Spark plan")
+
+ val newSplits = if (spjParams.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, partExpressions), newSplits)
+ }
+
+ // Now fill missing partition keys with empty partitions
+ val partitionMapping = nestGroupedPartitions.toMap
+ spjParams.commonPartitionValues.get.flatMap {
+ case (partValue, numSplits) =>
+ // Use empty partition for those partition values that are not present.
+ partitionMapping.getOrElse(
+ InternalRowComparableWrapper(partValue, partExpressions),
+ Seq.fill(numSplits)(Seq.empty))
+ }
+ } else {
+ // either `commonPartitionValues` is not defined, or it is defined but
+ // `applyPartialClustering` is false.
+ val partitionMapping = finalGroupedPartitions.map { case (partValue, splits) =>
+ InternalRowComparableWrapper(partValue, partExpressions) -> splits
+ }.toMap
+
+ // In case `commonPartitionValues` is not defined (e.g., SPJ is not used), there
+ // could exist duplicated partition values, as partition grouping is not done
+ // at the beginning and postponed to this method. It is important to use unique
+ // partition values here so that grouped partitions won't get duplicated.
+ p.uniquePartitionValues.map { partValue =>
+ // Use empty partition for those partition values that are not present
+ partitionMapping.getOrElse(
+ InternalRowComparableWrapper(partValue, partExpressions), Seq.empty)
+ }
+ }
+ }
+}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
index 9e5264d8d4f31..9f59bded94fe1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
@@ -26,7 +26,7 @@ import scala.util.control.NonFatal
import org.apache.hadoop.fs.Path
import org.apache.spark.SparkException
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.EXTENDED_EXPLAIN_GENERATOR
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{AnalysisException, ExtendedExplainGenerator, Row}
@@ -44,7 +44,8 @@ import org.apache.spark.sql.execution.bucketing.{CoalesceBucketsInJoin, DisableU
import org.apache.spark.sql.execution.dynamicpruning.PlanDynamicPruningFilters
import org.apache.spark.sql.execution.exchange.EnsureRequirements
import org.apache.spark.sql.execution.reuse.ReuseExchangeAndSubquery
-import org.apache.spark.sql.execution.streaming.{IncrementalExecution, OffsetSeqMetadata, WatermarkPropagator}
+import org.apache.spark.sql.execution.streaming.checkpointing.OffsetSeqMetadata
+import org.apache.spark.sql.execution.streaming.runtime.{IncrementalExecution, WatermarkPropagator}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.scripting.SqlScriptingExecution
import org.apache.spark.sql.streaming.OutputMode
@@ -683,4 +684,12 @@ object QueryExecution {
normalized
}
}
+
+ def determineShuffleCleanupMode(conf: SQLConf): ShuffleCleanupMode = {
+ if (conf.getConf(SQLConf.CLASSIC_SHUFFLE_DEPENDENCY_FILE_CLEANUP_ENABLED)) {
+ RemoveShuffleFiles
+ } else {
+ DoNotCleanup
+ }
+ }
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala
index 9dcb38f8ff10e..c5c2f9bb6a6f6 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala
@@ -30,6 +30,7 @@ import org.apache.spark.internal.config.{SPARK_DRIVER_PREFIX, SPARK_EXECUTOR_PRE
import org.apache.spark.internal.config.Tests.IS_TESTING
import org.apache.spark.sql.classic.SparkSession
import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec
+import org.apache.spark.sql.execution.exchange.ShuffleExchangeLike
import org.apache.spark.sql.execution.ui.{SparkListenerSQLExecutionEnd, SparkListenerSQLExecutionStart}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.StaticSQLConf.SQL_EVENT_TRUNCATE_LENGTH
@@ -178,8 +179,11 @@ object SQLExecution extends Logging {
val shuffleIds = queryExecution.executedPlan match {
case ae: AdaptiveSparkPlanExec =>
ae.context.shuffleIds.asScala.keys
- case _ =>
- Iterable.empty
+ case nonAdaptivePlan =>
+ nonAdaptivePlan.collect {
+ case exec: ShuffleExchangeLike =>
+ exec.shuffleId
+ }
}
shuffleIds.foreach { shuffleId =>
queryExecution.shuffleCleanupMode match {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala
index 5abc6f3ed5769..11fde41aae9e4 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala
@@ -62,7 +62,9 @@ case class SortExec(
"peakMemory" -> SQLMetrics.createSizeMetric(sparkContext, "peak memory"),
"spillSize" -> SQLMetrics.createSizeMetric(sparkContext, "spill size"))
- private[sql] var rowSorter: UnsafeExternalRowSorter = _
+ // Each task has its own instance of UnsafeExternalRowSorter. It is created in the
+ // createSorter method and stored in a ThreadLocal variable.
+ private[sql] var rowSorter: ThreadLocal[UnsafeExternalRowSorter] = _
/**
* This method gets invoked only once for each SortExec instance to initialize an
@@ -71,6 +73,8 @@ case class SortExec(
* should make it public.
*/
def createSorter(): UnsafeExternalRowSorter = {
+ rowSorter = new ThreadLocal[UnsafeExternalRowSorter]()
+
val ordering = RowOrdering.create(sortOrder, output)
// The comparator for comparing prefix
@@ -95,13 +99,14 @@ case class SortExec(
}
val pageSize = SparkEnv.get.memoryManager.pageSizeBytes
- rowSorter = UnsafeExternalRowSorter.create(
+ val newRowSorter = UnsafeExternalRowSorter.create(
schema, ordering, prefixComparator, prefixComputer, pageSize, canUseRadixSort)
if (testSpillFrequency > 0) {
- rowSorter.setTestSpillFrequency(testSpillFrequency)
+ newRowSorter.setTestSpillFrequency(testSpillFrequency)
}
- rowSorter
+ rowSorter.set(newRowSorter)
+ rowSorter.get()
}
protected override def doExecute(): RDD[InternalRow] = {
@@ -194,11 +199,11 @@ case class SortExec(
* In SortExec, we overwrites cleanupResources to close UnsafeExternalRowSorter.
*/
override protected[sql] def cleanupResources(): Unit = {
- if (rowSorter != null) {
+ if (rowSorter != null && rowSorter.get() != null) {
// There's possible for rowSorter is null here, for example, in the scenario of empty
// iterator in the current task, the downstream physical node(like SortMergeJoinExec) will
// trigger cleanupResources before rowSorter initialized in createSorter.
- rowSorter.cleanupResources()
+ rowSorter.get().cleanupResources()
}
super.cleanupResources()
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
index d9bb057282dff..e7844a88bf141 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
@@ -172,7 +172,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ
* guarantees that the outputs of these children will have same number of partitions, so that the
* operator can safely zip partitions of these children's result RDDs. Some operators can leverage
* this guarantee to satisfy some interesting requirement, e.g., non-broadcast joins can specify
- * HashClusteredDistribution(a,b) for its left child, and specify HashClusteredDistribution(c,d)
+ * ClusteredDistribution(a,b) for its left child, and specify ClusteredDistribution(c,d)
* for its right child, then it's guaranteed that left and right child are co-partitioned by
* a,b/c,d, which means tuples of same value are in the partitions of same index, e.g.,
* (a=1,b=2) and (c=1,d=2) are both in the second partition of left and right child.
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
index d19ec2defc022..f76bc911bef8f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
@@ -40,7 +40,11 @@ import org.apache.spark.sql.execution.datasources.{WriteFiles, WriteFilesExec}
import org.apache.spark.sql.execution.exchange.{REBALANCE_PARTITIONS_BY_COL, REBALANCE_PARTITIONS_BY_NONE, REPARTITION_BY_COL, REPARTITION_BY_NUM, ShuffleExchangeExec}
import org.apache.spark.sql.execution.python._
import org.apache.spark.sql.execution.python.streaming.{FlatMapGroupsInPandasWithStateExec, TransformWithStateInPySparkExec}
-import org.apache.spark.sql.execution.streaming._
+import org.apache.spark.sql.execution.streaming.operators.stateful.{EventTimeWatermarkExec, StreamingDeduplicateExec, StreamingDeduplicateWithinWatermarkExec, StreamingGlobalLimitExec, StreamingLocalLimitExec, UpdateEventTimeColumnExec}
+import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.FlatMapGroupsWithStateExec
+import org.apache.spark.sql.execution.streaming.operators.stateful.join.StreamingSymmetricHashJoinExec
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateExec
+import org.apache.spark.sql.execution.streaming.runtime.{StreamingExecutionRelation, StreamingRelation, StreamingRelationExec}
import org.apache.spark.sql.execution.streaming.sources.MemoryPlan
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.streaming.OutputMode
@@ -644,7 +648,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
case PhysicalAggregation(groupingExpressions, aggExpressions, resultExpressions, child)
if aggExpressions.forall(_.aggregateFunction.isInstanceOf[PythonUDAF]) =>
- Seq(execution.python.AggregateInPandasExec(
+ Seq(execution.python.ArrowAggregatePythonExec(
groupingExpressions,
aggExpressions,
resultExpressions,
@@ -671,7 +675,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
case PhysicalWindow(
WindowFunctionType.Python, windowExprs, partitionSpec, orderSpec, child) =>
- execution.python.WindowInPandasExec(
+ execution.python.ArrowWindowPythonExec(
windowExprs, partitionSpec, orderSpec, planLater(child)) :: Nil
case _ => Nil
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/UnionLoopExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/UnionLoopExec.scala
index e14f6f378b02a..47b5ba365cc20 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/UnionLoopExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/UnionLoopExec.scala
@@ -135,7 +135,8 @@ case class UnionLoopExec(
// reference any external tables, we are able to calculate everything in the optimizer,
// using the ConvertToLocalRelation rule, which significantly improves runtime.
if (count <= localRelationLimit) {
- val local = LocalRelation.fromExternalRows(anchor.output, df.collect().toIndexedSeq)
+ val local = LocalRelation.fromExternalRows(df.logicalPlan.output,
+ df.collect().toIndexedSeq)
(Dataset.ofRows(session, local), count)
} else {
(materializedDF, count)
@@ -182,7 +183,10 @@ case class UnionLoopExec(
// Main loop for obtaining the result of the recursive query.
while (prevCount > 0 && !limitReached) {
- var prevPlan: LogicalPlan = null
+ // The optimizer might have removed the UnionLoopRef in the recursion node (for example as a
+ // result of an empty join). In this case, prevPlan cannot be defined according to the cases
+ // below, so we set a default value of the previous result here.
+ var prevPlan: LogicalPlan = prevDF.logicalPlan
// If the recursive part contains non-deterministic expressions that depends on a seed, we
// need to create a new seed since the seed for this expression is set in the analysis, and
@@ -210,7 +214,7 @@ case class UnionLoopExec(
// SQLConf.CTE_RECURSION_ANCHOR_ROWS_LIMIT_TO_CONVERT_TO_LOCAL_RELATION is set to be
// anything larger than 0. However, we still handle this case in a special way to
// optimize the case when the flag is set to 0.
- case p @ Project(projectList, _: OneRowRelation) =>
+ case p @ Project(projectList, _: OneRowRelation) if p.subqueries.isEmpty =>
prevPlan = p
val prevPlanToRefMapping = projectList.zip(r.output).map {
case (fa: Alias, ta) => fa.withExprId(ta.exprId).withName(ta.name)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala
index 42fcfa8d60fa7..9728d664998e1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala
@@ -22,13 +22,12 @@ import java.nio.ByteBuffer
import scala.reflect.ClassTag
-import com.google.common.io.ByteStreams
-
import org.apache.spark.SparkUnsupportedOperationException
import org.apache.spark.serializer.{DeserializationStream, SerializationStream, Serializer, SerializerInstance}
import org.apache.spark.sql.catalyst.expressions.UnsafeRow
import org.apache.spark.sql.execution.metric.SQLMetric
import org.apache.spark.unsafe.Platform
+import org.apache.spark.util.Utils
/**
* Serializer for serializing [[UnsafeRow]]s during shuffle. Since UnsafeRows are already stored as
@@ -125,7 +124,7 @@ private class UnsafeRowSerializerInstance(
if (rowBuffer.length < rowSize) {
rowBuffer = new Array[Byte](rowSize)
}
- ByteStreams.readFully(dIn, rowBuffer, 0, rowSize)
+ Utils.readFully(dIn, rowBuffer, 0, rowSize)
row.pointTo(rowBuffer, Platform.BYTE_ARRAY_OFFSET, rowSize)
rowSize = readSize()
if (rowSize == EOF) { // We are returning the last row in this stream
@@ -160,7 +159,7 @@ private class UnsafeRowSerializerInstance(
if (rowBuffer.length < rowSize) {
rowBuffer = new Array[Byte](rowSize)
}
- ByteStreams.readFully(dIn, rowBuffer, 0, rowSize)
+ Utils.readFully(dIn, rowBuffer, 0, rowSize)
row.pointTo(rowBuffer, Platform.BYTE_ARRAY_OFFSET, rowSize)
row.asInstanceOf[T]
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala
index 21b5177fe2208..246508965d3d6 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala
@@ -25,7 +25,6 @@ import scala.util.control.NonFatal
import org.apache.spark.{broadcast, SparkException, SparkUnsupportedOperationException}
import org.apache.spark.internal.LogKeys.{CODEGEN_STAGE_ID, CONFIG, ERROR, HUGE_METHOD_LIMIT, MAX_METHOD_CODE_SIZE, TREE_NODE}
-import org.apache.spark.internal.MDC
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
@@ -548,6 +547,7 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with InputRDDCod
addSuffix: Boolean = false,
maxFields: Int,
printNodeId: Boolean,
+ printOutputColumns: Boolean,
indent: Int = 0): Unit = {
child.generateTreeString(
depth,
@@ -558,6 +558,7 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with InputRDDCod
addSuffix = false,
maxFields,
printNodeId,
+ printOutputColumns,
indent)
}
@@ -819,6 +820,7 @@ case class WholeStageCodegenExec(child: SparkPlan)(val codegenStageId: Int)
addSuffix: Boolean = false,
maxFields: Int,
printNodeId: Boolean,
+ printOutputColumns: Boolean,
indent: Int = 0): Unit = {
child.generateTreeString(
depth,
@@ -829,6 +831,7 @@ case class WholeStageCodegenExec(child: SparkPlan)(val codegenStageId: Int)
false,
maxFields,
printNodeId,
+ printOutputColumns,
indent)
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEOptimizer.scala
index 0f1743eeaacfb..f16c6d9cfe6dd 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEOptimizer.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEOptimizer.scala
@@ -18,7 +18,6 @@
package org.apache.spark.sql.execution.adaptive
import org.apache.spark.internal.LogKeys.{BATCH_NAME, RULE_NAME}
-import org.apache.spark.internal.MDC
import org.apache.spark.sql.catalyst.analysis.UpdateAttributeNullability
import org.apache.spark.sql.catalyst.optimizer.{ConvertToLocalRelation, EliminateLimits, OptimizeOneRowPlan}
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LogicalPlanIntegrity}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEShuffleReadExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEShuffleReadExec.scala
index e8b70f94a7692..2a600b31cc297 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEShuffleReadExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEShuffleReadExec.scala
@@ -178,6 +178,15 @@ case class AQEShuffleReadExec private(
numPartitionsMetric.set(partitionSpecs.length)
driverAccumUpdates += (numPartitionsMetric.id -> partitionSpecs.length.toLong)
+ val numEmptyPartitionsMetric = metrics("numEmptyPartitions")
+ val numEmptyPartitions = child match {
+ case s: ShuffleQueryStageExec =>
+ s.mapStats.map(stats => stats.bytesByPartitionId.count(_ == 0)).getOrElse(0)
+ case _ => 0
+ }
+ numEmptyPartitionsMetric.set(numEmptyPartitions)
+ driverAccumUpdates += (numEmptyPartitionsMetric.id -> numEmptyPartitions.toLong)
+
if (hasSkewedPartition) {
val skewedSpecs = partitionSpecs.collect {
case p: PartialReducerPartitionSpec => p
@@ -200,15 +209,7 @@ case class AQEShuffleReadExec private(
val numCoalescedPartitionsMetric = metrics("numCoalescedPartitions")
val x = partitionSpecs.count(isCoalescedSpec)
numCoalescedPartitionsMetric.set(x)
- val numEmptyPartitionsMetric = metrics("numEmptyPartitions")
- val y = child match {
- case s: ShuffleQueryStageExec =>
- s.mapStats.map(stats => stats.bytesByPartitionId.count(_ == 0)).getOrElse(0)
- case _ => 0
- }
- numEmptyPartitionsMetric.set(y)
- driverAccumUpdates ++= Seq(numCoalescedPartitionsMetric.id -> x,
- numEmptyPartitionsMetric.id -> y)
+ driverAccumUpdates ++= Seq(numCoalescedPartitionsMetric.id -> x)
}
partitionDataSizes.foreach { dataSizes =>
@@ -223,7 +224,9 @@ case class AQEShuffleReadExec private(
@transient override lazy val metrics: Map[String, SQLMetric] = {
if (shuffleStage.isDefined) {
- Map("numPartitions" -> SQLMetrics.createMetric(sparkContext, "number of partitions")) ++ {
+ Map("numPartitions" -> SQLMetrics.createMetric(sparkContext, "number of partitions"),
+ "numEmptyPartitions" ->
+ SQLMetrics.createMetric(sparkContext, "number of empty partitions")) ++ {
if (isLocalRead) {
// We split the mapper partition evenly when creating local shuffle read, so no
// data size info is available.
@@ -244,9 +247,7 @@ case class AQEShuffleReadExec private(
} ++ {
if (hasCoalescedPartition) {
Map("numCoalescedPartitions" ->
- SQLMetrics.createMetric(sparkContext, "number of coalesced partitions"),
- "numEmptyPartitions" ->
- SQLMetrics.createMetric(sparkContext, "number of empty partitions"))
+ SQLMetrics.createMetric(sparkContext, "number of coalesced partitions"))
} else {
Map.empty
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala
index 996e01a0ea936..0e50c03b6cc9f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala
@@ -28,8 +28,8 @@ import scala.util.control.NonFatal
import org.apache.spark.SparkException
import org.apache.spark.broadcast
-import org.apache.spark.internal.{MDC, MessageWithContext}
import org.apache.spark.internal.LogKeys._
+import org.apache.spark.internal.MessageWithContext
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.Attribute
@@ -430,6 +430,7 @@ case class AdaptiveSparkPlanExec(
addSuffix: Boolean = false,
maxFields: Int,
printNodeId: Boolean,
+ printOutputColumns: Boolean,
indent: Int = 0): Unit = {
super.generateTreeString(
depth,
@@ -440,6 +441,7 @@ case class AdaptiveSparkPlanExec(
addSuffix,
maxFields,
printNodeId,
+ printOutputColumns,
indent)
if (currentPhysicalPlan.fastEquals(initialPlan)) {
lastChildren.add(true)
@@ -452,6 +454,7 @@ case class AdaptiveSparkPlanExec(
addSuffix = false,
maxFields,
printNodeId,
+ printOutputColumns,
indent)
lastChildren.remove(lastChildren.size() - 1)
} else {
@@ -462,7 +465,8 @@ case class AdaptiveSparkPlanExec(
append,
verbose,
maxFields,
- printNodeId)
+ printNodeId,
+ printOutputColumns)
generateTreeStringWithHeader(
"Initial Plan",
initialPlan,
@@ -470,11 +474,11 @@ case class AdaptiveSparkPlanExec(
append,
verbose,
maxFields,
- printNodeId)
+ printNodeId,
+ printOutputColumns)
}
}
-
private def generateTreeStringWithHeader(
header: String,
plan: SparkPlan,
@@ -482,8 +486,9 @@ case class AdaptiveSparkPlanExec(
append: String => Unit,
verbose: Boolean,
maxFields: Int,
- printNodeId: Boolean): Unit = {
- append(" " * depth)
+ printNodeId: Boolean,
+ printOutputColumns: Boolean): Unit = {
+ append(" ".repeat(depth))
append(s"+- == $header ==\n")
plan.generateTreeString(
0,
@@ -494,6 +499,7 @@ case class AdaptiveSparkPlanExec(
addSuffix = false,
maxFields,
printNodeId,
+ printOutputColumns,
indent = depth + 1)
}
@@ -891,7 +897,7 @@ case class AdaptiveSparkPlanExec(
val e = if (originalErrors.size == 1) {
originalErrors.head
} else {
- val se = QueryExecutionErrors.multiFailuresInStageMaterializationError(originalErrors.head)
+ val se = QueryExecutionErrors.multiFailuresInStageMaterializationError(originalErrors)
originalErrors.tail.foreach(se.addSuppressed)
se
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala
index 2855f902a8509..aa748d8de6dc0 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala
@@ -20,7 +20,6 @@ package org.apache.spark.sql.execution.adaptive
import scala.collection.mutable
import org.apache.spark.internal.LogKeys.{CONFIG, SUB_QUERY}
-import org.apache.spark.internal.MDC
import org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.expressions.{DynamicPruningSubquery, ListQuery, SubqueryExpression}
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LogicalQueryStage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LogicalQueryStage.scala
index 506f52fd9072e..62e00d1ea6eda 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LogicalQueryStage.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LogicalQueryStage.scala
@@ -82,4 +82,32 @@ case class LogicalQueryStage(
case _: QueryStageExec => true
case _ => false
}
+
+ override def generateTreeString(
+ depth: Int,
+ lastChildren: java.util.ArrayList[Boolean],
+ append: String => Unit,
+ verbose: Boolean,
+ prefix: String = "",
+ addSuffix: Boolean = false,
+ maxFields: Int,
+ printNodeId: Boolean,
+ printOutputColumns: Boolean,
+ indent: Int = 0): Unit = {
+ super.generateTreeString(depth,
+ lastChildren,
+ append,
+ verbose,
+ prefix,
+ addSuffix,
+ maxFields,
+ printNodeId,
+ printOutputColumns,
+ indent)
+ lastChildren.add(true)
+ logicalPlan.generateTreeString(
+ depth + 1, lastChildren, append, verbose, "", false, maxFields, printNodeId,
+ printOutputColumns, indent)
+ lastChildren.remove(lastChildren.size() - 1)
+ }
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala
index 0a5bdefea7bc5..be58bccd1489a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala
@@ -134,6 +134,7 @@ abstract class QueryStageExec extends LeafExecNode {
addSuffix: Boolean = false,
maxFields: Int,
printNodeId: Boolean,
+ printOutputColumns: Boolean,
indent: Int = 0): Unit = {
super.generateTreeString(depth,
lastChildren,
@@ -143,10 +144,12 @@ abstract class QueryStageExec extends LeafExecNode {
addSuffix,
maxFields,
printNodeId,
+ printOutputColumns,
indent)
lastChildren.add(true)
plan.generateTreeString(
- depth + 1, lastChildren, append, verbose, "", false, maxFields, printNodeId, indent)
+ depth + 1, lastChildren, append, verbose, "", false, maxFields, printNodeId,
+ printOutputColumns, indent)
lastChildren.remove(lastChildren.size() - 1)
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala
index 1ea4df0254673..b7cf0ce3150ba 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.adaptive
import scala.collection.mutable.ArrayBuffer
import org.apache.spark.{MapOutputStatistics, MapOutputTrackerMaster, SparkEnv}
-import org.apache.spark.internal.{Logging, LogKeys, MDC}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.sql.execution.{CoalescedPartitionSpec, PartialReducerPartitionSpec, ShufflePartitionSpec}
object ShufflePartitionsUtil extends Logging {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala
index de1b83c16ac97..58055fa6129a5 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala
@@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate._
import org.apache.spark.sql.catalyst.plans.logical.Aggregate
import org.apache.spark.sql.execution.SparkPlan
-import org.apache.spark.sql.execution.streaming._
+import org.apache.spark.sql.execution.streaming.operators.stateful._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.util.Utils
import org.apache.spark.util.collection.{Utils => CUtils}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregateCodegenSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregateCodegenSupport.scala
index 40112979c6d46..595cecabe9185 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregateCodegenSupport.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregateCodegenSupport.scala
@@ -19,7 +19,6 @@ package org.apache.spark.sql.execution.aggregate
import org.apache.spark.SparkException
import org.apache.spark.internal.LogKeys.MAX_JVM_METHOD_PARAMS_LENGTH
-import org.apache.spark.internal.MDC
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression, ExpressionEquals, UnsafeRow}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/BaseAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/BaseAggregateExec.scala
index 5391d5807597c..7ae00467a0114 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/BaseAggregateExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/BaseAggregateExec.scala
@@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference,
import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Final, PartialMerge}
import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, UnspecifiedDistribution}
import org.apache.spark.sql.execution.{ExplainUtils, PartitioningPreservingUnaryExecNode, UnaryExecNode}
-import org.apache.spark.sql.execution.streaming.StatefulOperatorPartitioning
+import org.apache.spark.sql.execution.streaming.operators.stateful.StatefulOperatorPartitioning
/**
* Holds common logic for aggregate operators
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala
index 24528b6f4da15..4a0db1c613d60 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala
@@ -23,7 +23,6 @@ import scala.collection.mutable
import org.apache.spark.TaskContext
import org.apache.spark.internal.LogKeys.CONFIG
-import org.apache.spark.internal.MDC
import org.apache.spark.memory.SparkOutOfMemoryError
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashMapGenerator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashMapGenerator.scala
index 19a36483abe6d..af9dcb44cf1f6 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashMapGenerator.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashMapGenerator.scala
@@ -160,7 +160,8 @@ abstract class HashMapGenerator(
case BooleanType => hashInt(s"$input ? 1 : 0")
case ByteType | ShortType | IntegerType | DateType | _: YearMonthIntervalType =>
hashInt(input)
- case LongType | TimestampType | TimestampNTZType | _: DayTimeIntervalType => hashLong(input)
+ case LongType | TimestampType | TimestampNTZType | _: DayTimeIntervalType | _: TimeType =>
+ hashLong(input)
case FloatType => hashInt(s"Float.floatToIntBits($input)")
case DoubleType => hashLong(s"Double.doubleToLongBits($input)")
case d: DecimalType =>
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationIterator.scala
index a4a6dc8e4ab01..fbd3ce040e487 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationIterator.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationIterator.scala
@@ -18,7 +18,7 @@
package org.apache.spark.sql.execution.aggregate
import org.apache.spark.{SparkEnv, SparkException, TaskContext}
-import org.apache.spark.internal.{config, Logging, MDC}
+import org.apache.spark.internal.{config, Logging}
import org.apache.spark.internal.LogKeys.{CONFIG, HASH_MAP_SIZE, OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
@@ -332,6 +332,7 @@ class SortBasedAggregator(
SparkEnv.get.serializerManager,
TaskContext.get().taskMemoryManager().pageSizeBytes,
SparkEnv.get.conf.get(config.SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD),
+ SparkEnv.get.conf.get(config.SHUFFLE_SPILL_MAX_SIZE_FORCE_SPILL_THRESHOLD),
null
)
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationMap.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationMap.scala
index 9b68e6f02a859..5384f939c31ab 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationMap.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationMap.scala
@@ -79,6 +79,7 @@ class ObjectAggregationMap() {
SparkEnv.get.serializerManager,
TaskContext.get().taskMemoryManager().pageSizeBytes,
SparkEnv.get.conf.get(config.SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD),
+ SparkEnv.get.conf.get(config.SHUFFLE_SPILL_MAX_SIZE_FORCE_SPILL_THRESHOLD),
null
)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/UpdatingSessionsExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/UpdatingSessionsExec.scala
index b5dfd4639d8f2..b0b6d45620ac9 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/UpdatingSessionsExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/UpdatingSessionsExec.scala
@@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, SortOrder}
import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, Partitioning}
import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
-import org.apache.spark.sql.execution.streaming.StatefulOperatorPartitioning
+import org.apache.spark.sql.execution.streaming.operators.stateful.StatefulOperatorPartitioning
/**
* This node updates the session window spec of each input rows via analyzing neighbor rows and
@@ -52,10 +52,11 @@ case class UpdatingSessionsExec(
override protected def doExecute(): RDD[InternalRow] = {
val inMemoryThreshold = conf.sessionWindowBufferInMemoryThreshold
val spillThreshold = conf.sessionWindowBufferSpillThreshold
+ val spillSizeThreshold = conf.sessionWindowBufferSpillSizeThreshold
child.execute().mapPartitions { iter =>
new UpdatingSessionsIterator(iter, groupingExpression, sessionExpression,
- child.output, inMemoryThreshold, spillThreshold)
+ child.output, inMemoryThreshold, spillThreshold, spillSizeThreshold)
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/UpdatingSessionsIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/UpdatingSessionsIterator.scala
index 39b835f1f45fb..64bb3717f52bc 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/UpdatingSessionsIterator.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/UpdatingSessionsIterator.scala
@@ -43,7 +43,8 @@ class UpdatingSessionsIterator(
sessionExpression: NamedExpression,
inputSchema: Seq[Attribute],
inMemoryThreshold: Int,
- spillThreshold: Int) extends Iterator[InternalRow] {
+ spillThreshold: Int,
+ spillSizeThreshold: Long) extends Iterator[InternalRow] {
private val groupingWithoutSession: Seq[NamedExpression] =
groupingExpressions.diff(Seq(sessionExpression))
@@ -150,7 +151,8 @@ class UpdatingSessionsIterator(
currentKeys = groupingKey.copy()
currentSession = sessionStruct.copy()
- rowsForCurrentSession = new ExternalAppendOnlyUnsafeRowArray(inMemoryThreshold, spillThreshold)
+ rowsForCurrentSession = new ExternalAppendOnlyUnsafeRowArray(inMemoryThreshold, spillThreshold,
+ spillSizeThreshold)
rowsForCurrentSession.add(currentRow.asInstanceOf[UnsafeRow])
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala
index 1197a16a35e9b..492f11607ce6d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala
@@ -84,7 +84,7 @@ sealed trait BufferSetterGetterUtils {
(row: InternalRow, ordinal: Int) =>
if (row.isNullAt(ordinal)) null else row.getInt(ordinal)
- case TimestampType | TimestampNTZType =>
+ case TimestampType | TimestampNTZType | _: TimeType =>
(row: InternalRow, ordinal: Int) =>
if (row.isNullAt(ordinal)) null else row.getLong(ordinal)
@@ -188,7 +188,7 @@ sealed trait BufferSetterGetterUtils {
row.setNullAt(ordinal)
}
- case TimestampType | TimestampNTZType =>
+ case TimestampType | TimestampNTZType | _: TimeType =>
(row: InternalRow, ordinal: Int, value: Any) =>
if (value != null) {
row.setLong(ordinal, value.asInstanceOf[Long])
@@ -530,7 +530,7 @@ case class ScalaAggregator[IN, BUF, OUT](
def eval(buffer: BUF): Any = {
val row = outputSerializer(agg.finish(buffer))
- if (outputEncoder.isSerializedAsStruct) row else row.get(0, dataType)
+ if (outputEncoder.isSerializedAsStructForTopLevel) row else row.get(0, dataType)
}
@transient private[this] lazy val bufferRow = new UnsafeRow(bufferEncoder.namedExpressions.length)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
index ed490347ae821..3072a12e3d587 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
@@ -264,6 +264,109 @@ private[sql] object ArrowConverters extends Logging {
}
}
+ /**
+ * This is a class that converts input data in the form of a Byte array to InternalRow instances
+ * implementing the Iterator interface.
+ *
+ * The input data must be a valid Arrow IPC stream, this means that the first message is always
+ * the schema followed by N record batches.
+ *
+ * @param input Input Data
+ * @param context Task Context for Spark
+ */
+ private[sql] class InternalRowIteratorFromIPCStream(
+ input: Array[Byte],
+ context: TaskContext) extends Iterator[InternalRow] {
+
+ // Keep all the resources we have opened in order, should be closed
+ // in reverse order finally.
+ private val resources = new ArrayBuffer[AutoCloseable]()
+
+ // Create an allocator used for all Arrow related memory.
+ protected val allocator: BufferAllocator = ArrowUtils.rootAllocator.newChildAllocator(
+ s"to${this.getClass.getSimpleName}",
+ 0,
+ Long.MaxValue)
+ resources.append(allocator)
+
+ private val reader = try {
+ new ArrowStreamReader(new ByteArrayInputStream(input), allocator)
+ } catch {
+ case e: Exception =>
+ closeAll(resources.toSeq.reverse: _*)
+ throw new IllegalArgumentException(
+ s"Failed to create ArrowStreamReader: ${e.getMessage}", e)
+ }
+ resources.append(reader)
+
+ private val root: VectorSchemaRoot = try {
+ reader.getVectorSchemaRoot
+ } catch {
+ case e: Exception =>
+ closeAll(resources.toSeq.reverse: _*)
+ throw new IllegalArgumentException(
+ s"Failed to read schema from IPC stream: ${e.getMessage}", e)
+ }
+ resources.append(root)
+
+ val schema: StructType = try {
+ ArrowUtils.fromArrowSchema(root.getSchema)
+ } catch {
+ case e: Exception =>
+ closeAll(resources.toSeq.reverse: _*)
+ throw new IllegalArgumentException(s"Failed to convert Arrow schema: ${e.getMessage}", e)
+ }
+
+ // TODO: wrap in exception
+ private var rowIterator: Iterator[InternalRow] = vectorSchemaRootToIter(root)
+
+ // Metrics to track batch processing
+ private var _batchesLoaded: Int = 0
+ private var _totalRowsProcessed: Long = 0L
+
+ if (context != null) {
+ context.addTaskCompletionListener[Unit] { _ =>
+ closeAll(resources.toSeq.reverse: _*)
+ }
+ }
+
+ // Public accessors for metrics
+ def batchesLoaded: Int = _batchesLoaded
+ def totalRowsProcessed: Long = _totalRowsProcessed
+
+ // Loads the next batch from the Arrow reader and returns true or
+ // false if the next batch could be loaded.
+ private def loadNextBatch(): Boolean = {
+ if (reader.loadNextBatch()) {
+ rowIterator = vectorSchemaRootToIter(root)
+ _batchesLoaded += 1
+ true
+ } else {
+ false
+ }
+ }
+
+ override def hasNext: Boolean = {
+ if (rowIterator.hasNext) {
+ true
+ } else {
+ if (!loadNextBatch()) {
+ false
+ } else {
+ hasNext
+ }
+ }
+ }
+
+ override def next(): InternalRow = {
+ if (!hasNext) {
+ throw new NoSuchElementException("No more elements in iterator")
+ }
+ _totalRowsProcessed += 1
+ rowIterator.next()
+ }
+ }
+
/**
* An InternalRow iterator which parse data from serialized ArrowRecordBatches, subclass should
* implement [[nextBatch]] to parse data from binary records.
@@ -382,6 +485,23 @@ private[sql] object ArrowConverters extends Logging {
(iterator, iterator.schema)
}
+ /**
+ * Creates an iterator from a Byte array to deserialize an Arrow IPC stream with exactly
+ * one schema and a varying number of record batches. Returns an iterator over the
+ * created InternalRow.
+ */
+ private[sql] def fromIPCStream(input: Array[Byte], context: TaskContext):
+ (Iterator[InternalRow], StructType) = {
+ fromIPCStreamWithIterator(input, context)
+ }
+
+ // Overloaded method for tests to access the iterator with metrics
+ private[sql] def fromIPCStreamWithIterator(input: Array[Byte], context: TaskContext):
+ (InternalRowIteratorFromIPCStream, StructType) = {
+ val iterator = new InternalRowIteratorFromIPCStream(input, context)
+ (iterator, iterator.schema)
+ }
+
/**
* Convert an arrow batch container into an iterator of InternalRow.
*/
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala
index 995f857bbf635..78d7eb4ec1f51 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala
@@ -25,7 +25,7 @@ import scala.concurrent.ExecutionContext
import scala.concurrent.duration.Duration
import org.apache.spark.{InterruptibleIterator, Partition, SparkContext, SparkException, TaskContext}
-import org.apache.spark.rdd.{EmptyRDD, PartitionwiseSampledRDD, RDD}
+import org.apache.spark.rdd.{EmptyRDD, PartitionwiseSampledRDD, RDD, SQLPartitioningAwareUnionRDD}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences
@@ -699,8 +699,80 @@ case class UnionExec(children: Seq[SparkPlan]) extends SparkPlan {
}
}
- protected override def doExecute(): RDD[InternalRow] =
- sparkContext.union(children.map(_.execute()))
+ /**
+ * Returns the output partitionings of the children, with the attributes converted to
+ * the first child's attributes at the same position.
+ */
+ private def prepareOutputPartitioning(): Seq[Partitioning] = {
+ // Create a map of attributes from the other children to the first child.
+ val firstAttrs = children.head.output
+ val attributesMap = children.tail.map(_.output).map { otherAttrs =>
+ otherAttrs.zip(firstAttrs).map { case (attr, firstAttr) =>
+ attr -> firstAttr
+ }.toMap
+ }
+
+ val partitionings = children.map(_.outputPartitioning)
+ val firstPartitioning = partitionings.head
+ val otherPartitionings = partitionings.tail
+
+ val convertedOtherPartitionings = otherPartitionings.zipWithIndex.map { case (p, idx) =>
+ val attributeMap = attributesMap(idx)
+ p match {
+ case e: Expression =>
+ e.transform {
+ case a: Attribute if attributeMap.contains(a) => attributeMap(a)
+ }.asInstanceOf[Partitioning]
+ case _ => p
+ }
+ }
+ Seq(firstPartitioning) ++ convertedOtherPartitionings
+ }
+
+ private def comparePartitioning(left: Partitioning, right: Partitioning): Boolean = {
+ (left, right) match {
+ case (SinglePartition, SinglePartition) => true
+ case (l: HashPartitioningLike, r: HashPartitioningLike) => l == r
+ // Note: two `RangePartitioning`s with even same ordering and number of partitions
+ // are not equal, because they might have different partition bounds.
+ case _ => false
+ }
+ }
+
+ override def outputPartitioning: Partitioning = {
+ if (conf.getConf(SQLConf.UNION_OUTPUT_PARTITIONING)) {
+ val partitionings = prepareOutputPartitioning()
+ if (partitionings.forall(comparePartitioning(_, partitionings.head))) {
+ val partitioner = partitionings.head
+
+ // Take the output attributes of this union and map the partitioner to them.
+ val attributeMap = children.head.output.zip(output).toMap
+ partitioner match {
+ case e: Expression =>
+ e.transform {
+ case a: Attribute if attributeMap.contains(a) => attributeMap(a)
+ }.asInstanceOf[Partitioning]
+ case _ => partitioner
+ }
+ } else {
+ super.outputPartitioning
+ }
+ } else {
+ super.outputPartitioning
+ }
+ }
+
+ protected override def doExecute(): RDD[InternalRow] = {
+ if (outputPartitioning.isInstanceOf[UnknownPartitioning]) {
+ sparkContext.union(children.map(_.execute()))
+ } else {
+ // This union has a known partitioning, i.e., its children have the same partitioning
+ // in semantics so this union can choose not to change the partitioning by using a
+ // custom partitioning aware union RDD.
+ val nonEmptyRdds = children.map(_.execute()).filter(!_.partitions.isEmpty)
+ new SQLPartitioningAwareUnionRDD(sparkContext, nonEmptyRdds, outputPartitioning.numPartitions)
+ }
+ }
override def supportsColumnar: Boolean = children.forall(_.supportsColumnar)
@@ -790,6 +862,7 @@ abstract class BaseSubqueryExec extends SparkPlan {
addSuffix: Boolean = false,
maxFields: Int,
printNodeId: Boolean,
+ printOutputColumns: Boolean,
indent: Int = 0): Unit = {
/**
* In the new explain mode `EXPLAIN FORMATTED`, the subqueries are not shown in the
@@ -807,6 +880,7 @@ abstract class BaseSubqueryExec extends SparkPlan {
false,
maxFields,
printNodeId,
+ printOutputColumns,
indent)
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala
index 2074649cc9863..6fb1f5263b518 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala
@@ -147,7 +147,7 @@ private[sql] object ColumnAccessor {
case ByteType => new ByteColumnAccessor(buf)
case ShortType => new ShortColumnAccessor(buf)
case IntegerType | DateType | _: YearMonthIntervalType => new IntColumnAccessor(buf)
- case LongType | TimestampType | TimestampNTZType | _: DayTimeIntervalType =>
+ case LongType | TimestampType | TimestampNTZType | _: DayTimeIntervalType | _: TimeType =>
new LongColumnAccessor(buf)
case FloatType => new FloatColumnAccessor(buf)
case DoubleType => new DoubleColumnAccessor(buf)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBuilder.scala
index b65ef12f12d56..a63569b19a018 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBuilder.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBuilder.scala
@@ -185,7 +185,7 @@ private[columnar] object ColumnBuilder {
case ByteType => new ByteColumnBuilder
case ShortType => new ShortColumnBuilder
case IntegerType | DateType | _: YearMonthIntervalType => new IntColumnBuilder
- case LongType | TimestampType | TimestampNTZType | _: DayTimeIntervalType =>
+ case LongType | TimestampType | TimestampNTZType | _: DayTimeIntervalType | _: TimeType =>
new LongColumnBuilder
case FloatType => new FloatColumnBuilder
case DoubleType => new DoubleColumnBuilder
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala
index 60695a6c5d49c..df250e529e2ce 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala
@@ -869,7 +869,8 @@ private[columnar] object ColumnType {
case ByteType => BYTE
case ShortType => SHORT
case IntegerType | DateType | _: YearMonthIntervalType => INT
- case LongType | TimestampType | TimestampNTZType | _: DayTimeIntervalType => LONG
+ case LongType | TimestampType | TimestampNTZType | _: DayTimeIntervalType | _: TimeType =>
+ LONG
case FloatType => FLOAT
case DoubleType => DOUBLE
case s: StringType => STRING(s)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala
index d07ebeb843bba..dd64d92bed71e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala
@@ -82,7 +82,7 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera
case ByteType => classOf[ByteColumnAccessor].getName
case ShortType => classOf[ShortColumnAccessor].getName
case IntegerType | DateType | _: YearMonthIntervalType => classOf[IntColumnAccessor].getName
- case LongType | TimestampType | TimestampNTZType | _: DayTimeIntervalType =>
+ case LongType | TimestampType | TimestampNTZType | _: DayTimeIntervalType | _: TimeType =>
classOf[LongColumnAccessor].getName
case FloatType => classOf[FloatColumnAccessor].getName
case DoubleType => classOf[DoubleColumnAccessor].getName
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
index 8f704cec7e892..eabbc7fc74f50 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
@@ -19,7 +19,6 @@ package org.apache.spark.sql.execution.columnar
import com.esotericsoftware.kryo.{DefaultSerializer, Kryo, Serializer => KryoSerializer}
import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput}
-import org.apache.commons.lang3.StringUtils
import org.apache.spark.{SparkException, TaskContext}
import org.apache.spark.network.util.JavaUtils
@@ -267,42 +266,29 @@ case class CachedRDDBuilder(
private val materializedPartitions = cachedPlan.session.sparkContext.longAccumulator
val cachedName = tableName.map(n => s"In-memory table $n")
- .getOrElse(StringUtils.abbreviate(cachedPlan.toString, 1024))
+ .getOrElse(Utils.abbreviate(cachedPlan.toString, 1024))
val supportsColumnarInput: Boolean = {
cachedPlan.supportsColumnar &&
serializer.supportsColumnarInput(cachedPlan.output)
}
- def cachedColumnBuffers: RDD[CachedBatch] = {
+ def cachedColumnBuffers: RDD[CachedBatch] = synchronized {
if (_cachedColumnBuffers == null) {
- synchronized {
- if (_cachedColumnBuffers == null) {
- _cachedColumnBuffers = buildBuffers()
- }
- }
+ _cachedColumnBuffers = buildBuffers()
}
_cachedColumnBuffers
}
- def clearCache(blocking: Boolean = false): Unit = {
+ def clearCache(blocking: Boolean = false): Unit = synchronized {
if (_cachedColumnBuffers != null) {
- synchronized {
- if (_cachedColumnBuffers != null) {
- _cachedColumnBuffers.unpersist(blocking)
- _cachedColumnBuffers = null
- }
- }
+ _cachedColumnBuffers.unpersist(blocking)
+ _cachedColumnBuffers = null
}
}
- def isCachedColumnBuffersLoaded: Boolean = {
- if (_cachedColumnBuffers != null) {
- synchronized {
- return _cachedColumnBuffers != null && isCachedRDDLoaded
- }
- }
- false
+ def isCachedColumnBuffersLoaded: Boolean = synchronized {
+ _cachedColumnBuffers != null && isCachedRDDLoaded
}
private def isCachedRDDLoaded: Boolean = {
@@ -381,22 +367,8 @@ object InMemoryRelation {
/* Visible for testing */
private[columnar] def clearSerializer(): Unit = synchronized { ser = None }
- def convertToColumnarIfPossible(plan: SparkPlan): SparkPlan = plan match {
- case gen: WholeStageCodegenExec => gen.child match {
- case c2r: ColumnarToRowTransition => c2r.child match {
- case ia: InputAdapter => ia.child
- case _ => plan
- }
- case _ => plan
- }
- case c2r: ColumnarToRowTransition => // This matches when whole stage code gen is disabled.
- c2r.child
- case adaptive: AdaptiveSparkPlanExec =>
- // If AQE is enabled for cached plan and table cache supports columnar in, we should mark
- // `AdaptiveSparkPlanExec.supportsColumnar` as true to avoid inserting `ColumnarToRow`, so
- // that `CachedBatchSerializer` can use `convertColumnarBatchToCachedBatch` to cache data.
- adaptive.copy(supportsColumnar = true)
- case _ => plan
+ def convertToColumnarIfPossible(plan: SparkPlan): SparkPlan = {
+ getSerializer(plan.conf).convertToColumnarPlanIfPossible(plan)
}
def apply(
@@ -406,7 +378,7 @@ object InMemoryRelation {
val optimizedPlan = qe.optimizedPlan
val serializer = getSerializer(optimizedPlan.conf)
val child = if (serializer.supportsColumnarInput(optimizedPlan.output)) {
- convertToColumnarIfPossible(qe.executedPlan)
+ serializer.convertToColumnarPlanIfPossible(qe.executedPlan)
} else {
qe.executedPlan
}
@@ -433,8 +405,9 @@ object InMemoryRelation {
def apply(cacheBuilder: CachedRDDBuilder, qe: QueryExecution): InMemoryRelation = {
val optimizedPlan = qe.optimizedPlan
- val newBuilder = if (cacheBuilder.serializer.supportsColumnarInput(optimizedPlan.output)) {
- cacheBuilder.copy(cachedPlan = convertToColumnarIfPossible(qe.executedPlan))
+ val serializer = cacheBuilder.serializer
+ val newBuilder = if (serializer.supportsColumnarInput(optimizedPlan.output)) {
+ cacheBuilder.copy(cachedPlan = serializer.convertToColumnarPlanIfPossible(qe.executedPlan))
} else {
cacheBuilder.copy(cachedPlan = qe.executedPlan)
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTablesCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTablesCommand.scala
index 26192551632e3..f3a0da2437ae8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTablesCommand.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTablesCommand.scala
@@ -20,7 +20,6 @@ package org.apache.spark.sql.execution.command
import scala.util.control.NonFatal
import org.apache.spark.internal.LogKeys.{DATABASE_NAME, ERROR, TABLE_NAME}
-import org.apache.spark.internal.MDC
import org.apache.spark.sql.{Row, SparkSession}
import org.apache.spark.sql.classic.ClassicConversions.castToImpl
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala
index 7cbba170cd1e6..9a86357ca0b76 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala
@@ -24,7 +24,7 @@ import scala.util.control.NonFatal
import org.apache.hadoop.fs.{FileStatus, FileSystem, Path, PathFilter}
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{COUNT, DATABASE_NAME, ERROR, TABLE_NAME, TIME}
import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
import org.apache.spark.sql.catalyst.analysis.ResolvedIdentifier
@@ -36,7 +36,7 @@ import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData}
import org.apache.spark.sql.classic.SparkSession
import org.apache.spark.sql.errors.QueryCompilationErrors
-import org.apache.spark.sql.execution.QueryExecution
+import org.apache.spark.sql.execution.{QueryExecution, RemoveShuffleFiles}
import org.apache.spark.sql.execution.datasources.{DataSourceUtils, InMemoryFileIndex}
import org.apache.spark.sql.functions.{col, lit}
import org.apache.spark.sql.internal.{SessionState, SQLConf}
@@ -304,8 +304,8 @@ object CommandUtils extends Logging {
columns.map(statExprs(_, conf, attributePercentiles))
val namedExpressions = expressions.map(e => Alias(e, e.toString)())
- val statsRow = new QueryExecution(sparkSession, Aggregate(Nil, namedExpressions, relation))
- .executedPlan.executeTake(1).head
+ val statsRow = new QueryExecution(sparkSession, Aggregate(Nil, namedExpressions, relation),
+ shuffleCleanupMode = RemoveShuffleFiles).executedPlan.executeTake(1).head
val rowCount = statsRow.getLong(0)
val columnStats = columns.zipWithIndex.map { case (attr, i) =>
@@ -341,8 +341,8 @@ object CommandUtils extends Logging {
Alias(expr, expr.toString)()
}
- val percentilesRow = new QueryExecution(sparkSession, Aggregate(Nil, namedExprs, relation))
- .executedPlan.executeTake(1).head
+ val percentilesRow = new QueryExecution(sparkSession, Aggregate(Nil, namedExprs, relation),
+ shuffleCleanupMode = RemoveShuffleFiles).executedPlan.executeTake(1).head
attrsToGenHistogram.zipWithIndex.foreach { case (attr, i) =>
val percentiles = percentilesRow.getArray(i)
// When there is no non-null value, `percentiles` is null. In such case, there is no
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateSQLFunctionCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateSQLFunctionCommand.scala
index 09b2c86970754..eb860089b0c88 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateSQLFunctionCommand.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateSQLFunctionCommand.scala
@@ -20,13 +20,15 @@ package org.apache.spark.sql.execution.command
import org.apache.spark.SparkException
import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
import org.apache.spark.sql.catalyst.FunctionIdentifier
-import org.apache.spark.sql.catalyst.analysis.{Analyzer, SQLFunctionNode, UnresolvedAlias, UnresolvedAttribute, UnresolvedFunction, UnresolvedRelation}
+import org.apache.spark.sql.catalyst.analysis.{withPosition, Analyzer, SQLFunctionExpression, SQLFunctionNode, SQLScalarFunction, SQLTableFunction, UnresolvedAlias, UnresolvedAttribute, UnresolvedFunction, UnresolvedRelation, UnresolvedTableValuedFunction}
import org.apache.spark.sql.catalyst.catalog.{SessionCatalog, SQLFunction, UserDefinedFunction, UserDefinedFunctionErrors}
-import org.apache.spark.sql.catalyst.expressions.{Alias, Cast, Generator, LateralSubquery, Literal, ScalarSubquery, SubqueryExpression, WindowExpression}
+import org.apache.spark.sql.catalyst.catalog.UserDefinedFunction._
+import org.apache.spark.sql.catalyst.expressions.{Alias, Cast, Expression, Generator, LateralSubquery, Literal, ScalarSubquery, SubqueryExpression, WindowExpression}
import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
import org.apache.spark.sql.catalyst.plans.Inner
-import org.apache.spark.sql.catalyst.plans.logical.{LateralJoin, LogicalPlan, OneRowRelation, Project, UnresolvedWith}
+import org.apache.spark.sql.catalyst.plans.logical.{LateralJoin, LocalRelation, LogicalPlan, OneRowRelation, Project, Range, UnresolvedWith, View}
import org.apache.spark.sql.catalyst.trees.TreePattern.UNRESOLVED_ATTRIBUTE
+import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper
import org.apache.spark.sql.errors.QueryCompilationErrors
import org.apache.spark.sql.execution.command.CreateUserDefinedFunctionCommand._
import org.apache.spark.sql.types.{DataType, StructField, StructType}
@@ -150,6 +152,9 @@ case class CreateSQLFunctionCommand(
Project(outputAlias, inputPlan)
}
+ // Check cyclic function reference before running the analyzer.
+ checkCyclicFunctionReference(catalog, name, plan)
+
// Check the function body can be analyzed correctly.
val analyzed = analyzer.execute(plan)
val (resolved, resolvedReturnType) = analyzed match {
@@ -172,6 +177,8 @@ case class CreateSQLFunctionCommand(
if (query.isEmpty) {
throw UserDefinedFunctionErrors.bodyIsNotAQueryForSqlTableUdf(name.funcName)
}
+ // Check cyclic function reference before running the analyzer.
+ checkCyclicFunctionReference(catalog, name, query.get)
// Construct a lateral join to analyze the function body.
val plan = LateralJoin(inputPlan, LateralSubquery(query.get), Inner, None)
@@ -241,10 +248,14 @@ case class CreateSQLFunctionCommand(
// Derive determinism of the SQL function.
val deterministic = analyzedPlan.deterministic
+ // Derive and check a SQL function with CONTAINS SQL data access should not reads SQL data.
+ val readsSQLData = deriveSQLDataAccess(analyzedPlan)
+
function.copy(
// Assign the return type, inferring from the function body if needed.
returnType = inferredReturnType,
deterministic = Some(function.deterministic.getOrElse(deterministic)),
+ containsSQL = Some(function.containsSQL.getOrElse(!readsSQLData)),
properties = properties
)
}
@@ -367,6 +378,61 @@ case class CreateSQLFunctionCommand(
}
}
+ /**
+ * Check if the given plan contains cyclic function references.
+ */
+ private def checkCyclicFunctionReference(
+ catalog: SessionCatalog,
+ identifier: FunctionIdentifier,
+ plan: LogicalPlan): Unit = {
+
+ def checkPlan(plan: LogicalPlan, path: Seq[FunctionIdentifier]): Unit = {
+ plan.foreach {
+ case u @ UnresolvedTableValuedFunction(nameParts, arguments, _) =>
+ val funcId = nameParts.asFunctionIdentifier
+ val info = catalog.lookupFunctionInfo(funcId)
+ if (isSQLFunction(info.getClassName)) {
+ val f = withPosition(u) {
+ catalog.lookupTableFunction(funcId, arguments).asInstanceOf[SQLTableFunction]
+ }
+ // Check cyclic reference using qualified function names.
+ val newPath = path :+ f.function.name
+ if (f.function.name == name) {
+ throw UserDefinedFunctionErrors.cyclicFunctionReference(newPath.mkString(" -> "))
+ }
+ val plan = catalog.makeSQLTableFunctionPlan(f.name, f.function, f.inputs, f.output)
+ checkPlan(plan, newPath)
+ }
+ case p: LogicalPlan =>
+ p.expressions.foreach(checkExpression(_, path))
+ }
+ }
+
+ def checkExpression(expression: Expression, path: Seq[FunctionIdentifier]): Unit = {
+ expression.foreach {
+ case s: SubqueryExpression => checkPlan(s.plan, path)
+ case u @ UnresolvedFunction(nameParts, arguments, _, _, _, _, _) =>
+ val funcId = nameParts.asFunctionIdentifier
+ val info = catalog.lookupFunctionInfo(funcId)
+ if (isSQLFunction(info.getClassName)) {
+ val f = withPosition(u) {
+ catalog.lookupFunction(funcId, arguments).asInstanceOf[SQLFunctionExpression]
+ }
+ // Check cyclic reference using qualified function names.
+ val newPath = path :+ f.function.name
+ if (f.function.name == name) {
+ throw UserDefinedFunctionErrors.cyclicFunctionReference(newPath.mkString(" -> "))
+ }
+ val plan = catalog.makeSQLFunctionPlan(f.name, f.function, f.inputs)
+ checkPlan(plan, newPath)
+ }
+ case _ =>
+ }
+ }
+
+ checkPlan(plan, Seq(identifier))
+ }
+
/**
* Check if the SQL function body contains aggregate/window/generate functions.
* Note subqueries inside the SQL function body can contain aggregate/window/generate functions.
@@ -384,6 +450,43 @@ case class CreateSQLFunctionCommand(
}
}
+ /**
+ * Derive the SQL data access routine of the function and check if the SQL function matches
+ * its data access routine. If the data access is CONTAINS SQL, the expression should not
+ * access operators and expressions that read SQL data.
+ *
+ * Returns true is SQL data access routine is READS SQL DATA, otherwise returns false.
+ */
+ private def deriveSQLDataAccess(plan: LogicalPlan): Boolean = {
+ // Find logical plan nodes that read SQL data.
+ val readsSQLData = plan.find {
+ case _: View => true
+ case p if p.children.isEmpty => p match {
+ case _: OneRowRelation | _: LocalRelation | _: Range => false
+ case _ => true
+ }
+ case f: SQLFunctionNode => f.function.containsSQL.contains(false)
+ case p: LogicalPlan =>
+ lazy val sub = p.subqueries.exists(deriveSQLDataAccess)
+ // If the SQL function contains another SQL function that has SQL data access routine
+ // to be READS SQL DATA, then this SQL function will also be READS SQL DATA.
+ p.expressions.exists(expr => expr.find {
+ case f: SQLScalarFunction => f.function.containsSQL.contains(false)
+ case sub: SubqueryExpression => deriveSQLDataAccess(sub.plan)
+ case _ => false
+ }.isDefined)
+ }.isDefined
+
+ if (containsSQL.contains(true) && readsSQLData) {
+ throw new AnalysisException(
+ errorClass = "INVALID_SQL_FUNCTION_DATA_ACCESS",
+ messageParameters = Map.empty
+ )
+ }
+
+ readsSQLData
+ }
+
/**
* Generate the function properties, including:
* 1. the SQL configs when creating the function.
@@ -407,7 +510,7 @@ case class CreateSQLFunctionCommand(
}
val tempVars = ViewHelper.collectTemporaryVariables(analyzed)
- sqlConfigsToProps(conf) ++
+ sqlConfigsToProps(conf, SQL_CONFIG_PREFIX) ++
catalogAndNamespaceToProps(
manager.currentCatalog.name,
manager.currentNamespace.toIndexedSeq) ++
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateUserDefinedFunctionCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateUserDefinedFunctionCommand.scala
index 1ee3c8a4c388f..78ff514bf9e51 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateUserDefinedFunctionCommand.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateUserDefinedFunctionCommand.scala
@@ -20,9 +20,8 @@ package org.apache.spark.sql.execution.command
import java.util.Locale
import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.catalyst.FunctionIdentifier
+import org.apache.spark.sql.catalyst.{CapturesConfig, FunctionIdentifier}
import org.apache.spark.sql.catalyst.catalog.{LanguageSQL, RoutineLanguage, UserDefinedFunctionErrors}
-import org.apache.spark.sql.catalyst.catalog.UserDefinedFunction._
import org.apache.spark.sql.catalyst.plans.logical.IgnoreCachedData
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.StructType
@@ -31,7 +30,7 @@ import org.apache.spark.sql.types.StructType
* The base class for CreateUserDefinedFunctionCommand
*/
abstract class CreateUserDefinedFunctionCommand
- extends LeafRunnableCommand with IgnoreCachedData
+ extends LeafRunnableCommand with IgnoreCachedData with CapturesConfig
object CreateUserDefinedFunctionCommand {
@@ -81,18 +80,6 @@ object CreateUserDefinedFunctionCommand {
}
}
- /**
- * Convert SQL configs to properties by prefixing all configs with a key.
- * When converting a function to [[org.apache.spark.sql.catalyst.catalog.CatalogFunction]] or
- * [[org.apache.spark.sql.catalyst.expressions.ExpressionInfo]], all SQL configs and other
- * function properties (such as the function parameters and the function return type)
- * are saved together in a property map.
- */
- def sqlConfigsToProps(conf: SQLConf): Map[String, String] = {
- val modifiedConfs = ViewHelper.getModifiedConf(conf)
- modifiedConfs.map { case (key, value) => s"$SQL_CONFIG_PREFIX$key" -> value }
- }
-
/**
* Check whether the function parameters contain duplicated column names.
* It takes the function input parameter struct as input and verifies that there is no duplicates
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala
index d3a72f915c47b..667623fb95b03 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala
@@ -18,7 +18,6 @@
package org.apache.spark.sql.execution.command
import org.apache.spark.internal.LogKeys._
-import org.apache.spark.internal.MDC
import org.apache.spark.sql.{AnalysisException, Row, SaveMode, SparkSession}
import org.apache.spark.sql.catalyst.catalog._
import org.apache.spark.sql.catalyst.plans.logical.{CTEInChildren, CTERelationDef, LogicalPlan, WithCTE}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala
index 87cd9376b77b1..a3591ff89e5cf 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala
@@ -17,9 +17,10 @@
package org.apache.spark.sql.execution.command
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{CONFIG, CONFIG2, KEY, VALUE}
import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
+import org.apache.spark.sql.catalyst.analysis.{AnalysisContext, VariableResolution}
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.parser.ParseException
import org.apache.spark.sql.catalyst.plans.logical.IgnoreCachedData
@@ -107,7 +108,14 @@ case class SetCommand(kv: Option[(String, Option[String])])
Seq()
}
if (varName.nonEmpty && varName.length <= 3) {
- if (sparkSession.sessionState.analyzer.lookupVariable(varName).isDefined) {
+ val variableResolution = new VariableResolution(
+ sparkSession.sessionState.analyzer.catalogManager.tempVariableManager
+ )
+ val variable = variableResolution.lookupVariable(
+ nameParts = varName,
+ resolvingExecuteImmediate = AnalysisContext.get.isExecuteImmediate
+ )
+ if (variable.isDefined) {
throw new AnalysisException(
errorClass = "UNSUPPORTED_FEATURE.SET_VARIABLE_USING_SET",
messageParameters = Map("variableName" -> toSQLId(varName)))
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala
index ee21d7e970dfd..0ea52f3dccb84 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala
@@ -32,7 +32,7 @@ import org.apache.spark.sql.errors.QueryCompilationErrors
import org.apache.spark.sql.execution.{CommandExecutionMode, ExplainMode, LeafExecNode, SparkPlan, UnaryExecNode}
import org.apache.spark.sql.execution.datasources.DataSource
import org.apache.spark.sql.execution.metric.SQLMetric
-import org.apache.spark.sql.execution.streaming.IncrementalExecution
+import org.apache.spark.sql.execution.streaming.runtime.IncrementalExecution
import org.apache.spark.sql.types._
import org.apache.spark.sql.util.CaseInsensitiveStringMap
import org.apache.spark.util.ArrayImplicits._
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala
index f29d2267f75fd..5ef19b832f5b3 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala
@@ -20,7 +20,6 @@ package org.apache.spark.sql.execution.command
import java.net.URI
import org.apache.spark.internal.LogKeys._
-import org.apache.spark.internal.MDC
import org.apache.spark.sql.{AnalysisException, Row, SaveMode, SparkSession}
import org.apache.spark.sql.catalyst.catalog._
import org.apache.spark.sql.catalyst.plans.logical.{CTEInChildren, CTERelationDef, LogicalPlan, WithCTE}
@@ -119,9 +118,7 @@ case class CreateDataSourceTableCommand(table: CatalogTable, ignoreIfExists: Boo
}
- // We will return Nil or throw exception at the beginning if the table already exists, so when
- // we reach here, the table should not exist and we should set `ignoreIfExists` to false.
- sessionState.catalog.createTable(newTable, ignoreIfExists = false)
+ sessionState.catalog.createTable(newTable, ignoreIfExists)
Seq.empty[Row]
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala
index 13994c58f1206..8a4f586edfe05 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala
@@ -28,11 +28,11 @@ import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs._
import org.apache.hadoop.mapred.{FileInputFormat, JobConf}
-import org.apache.spark.internal.{Logging, LogKeys, MDC}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.internal.config.RDD_PARALLEL_LISTING_THRESHOLD
-import org.apache.spark.sql.{Row, SparkSession}
+import org.apache.spark.sql.{Row, SaveMode, SparkSession}
import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.analysis.Resolver
+import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, Resolver}
import org.apache.spark.sql.catalyst.catalog._
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
import org.apache.spark.sql.catalyst.expressions.Attribute
@@ -46,7 +46,7 @@ import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAM
import org.apache.spark.sql.connector.catalog.SupportsNamespaces._
import org.apache.spark.sql.errors.QueryCompilationErrors
import org.apache.spark.sql.errors.QueryExecutionErrors.hiveTableWithAnsiIntervalsError
-import org.apache.spark.sql.execution.datasources.{DataSource, DataSourceUtils, FileFormat, HadoopFsRelation, LogicalRelationWithTable}
+import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, DataSourceUtils, FileFormat, HadoopFsRelation, LogicalRelation, LogicalRelationWithTable}
import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2
import org.apache.spark.sql.internal.{HiveSerDe, SQLConf}
import org.apache.spark.sql.types._
@@ -189,8 +189,9 @@ case class DescribeDatabaseCommand(
Row("Catalog Name", SESSION_CATALOG_NAME) ::
Row("Database Name", dbMetadata.name) ::
Row("Comment", dbMetadata.description) ::
- Row("Location", CatalogUtils.URIToString(dbMetadata.locationUri))::
- Row("Owner", allDbProperties.getOrElse(PROP_OWNER, "")) :: Nil
+ Row("Location", CatalogUtils.URIToString(dbMetadata.locationUri)) ::
+ Row("Owner", allDbProperties.getOrElse(PROP_OWNER, "")) ::
+ allDbProperties.get(PROP_COLLATION).map(Row("Collation", _)).toList
if (extended) {
val properties = allDbProperties -- CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES
@@ -464,7 +465,7 @@ case class AlterTableChangeColumnCommand(
// when altering column. Only changes in collation of data type or its nested types (recursively)
// are allowed.
private def canEvolveType(from: StructField, to: StructField): Boolean = {
- DataType.equalsIgnoreCompatibleCollation(from.dataType, to.dataType)
+ DataType.equalsIgnoreCompatibleCollation(from.dataType, to.dataType, checkComplexTypes = false)
}
}
@@ -955,6 +956,64 @@ case class AlterTableSetLocationCommand(
}
}
+/**
+ * A command that saves a query as a V1 table.
+ */
+private[sql] case class SaveAsV1TableCommand(
+ tableDesc: CatalogTable,
+ mode: SaveMode,
+ query: LogicalPlan) extends LeafRunnableCommand {
+ override def run(sparkSession: SparkSession): Seq[Row] = {
+ val catalog = sparkSession.sessionState.catalog
+ val qualifiedIdent = catalog.qualifyIdentifier(tableDesc.identifier)
+ val tableDescWithQualifiedIdent = tableDesc.copy(identifier = qualifiedIdent)
+ val tableExists = catalog.tableExists(qualifiedIdent)
+
+ (tableExists, mode) match {
+ case (true, SaveMode.Ignore) =>
+ // Do nothing
+
+ case (true, SaveMode.ErrorIfExists) =>
+ throw QueryCompilationErrors.tableAlreadyExistsError(qualifiedIdent)
+
+ case (true, SaveMode.Overwrite) =>
+ // Get all input data source or hive relations of the query.
+ val srcRelations = query.collect {
+ case l: LogicalRelation => l.relation
+ case relation: HiveTableRelation => relation.tableMeta.identifier
+ }
+
+ val tableRelation = sparkSession.table(qualifiedIdent).queryExecution.analyzed
+ EliminateSubqueryAliases(tableRelation) match {
+ // check if the table is a data source table (the relation is a BaseRelation).
+ case l: LogicalRelation if srcRelations.contains(l.relation) =>
+ throw QueryCompilationErrors.cannotOverwriteTableThatIsBeingReadFromError(
+ qualifiedIdent)
+ // check hive table relation when overwrite mode
+ case relation: HiveTableRelation
+ if srcRelations.contains(relation.tableMeta.identifier) =>
+ throw QueryCompilationErrors.cannotOverwriteTableThatIsBeingReadFromError(
+ qualifiedIdent)
+ case _ => // OK
+ }
+
+ // Drop the existing table
+ catalog.dropTable(qualifiedIdent, ignoreIfNotExists = true, purge = false)
+ runCommand(sparkSession, CreateTable(tableDescWithQualifiedIdent, mode, Some(query)))
+ // Refresh the cache of the table in the catalog.
+ catalog.refreshTable(qualifiedIdent)
+
+ case _ =>
+ runCommand(sparkSession, CreateTable(tableDescWithQualifiedIdent, mode, Some(query)))
+ }
+ Seq.empty[Row]
+ }
+
+ private def runCommand(session: SparkSession, command: LogicalPlan): Unit = {
+ val qe = session.sessionState.executePlan(command)
+ qe.assertCommandExecuted()
+ }
+}
object DDLUtils extends Logging {
val HIVE_PROVIDER = "hive"
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
index 092e6669338ee..d5dd934af2be1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
@@ -247,8 +247,9 @@ case class AlterTableAddColumnsCommand(
}
DDLUtils.checkTableColumns(catalogTable, StructType(colsWithProcessedDefaults))
- val existingSchema = CharVarcharUtils.getRawSchema(catalogTable.dataSchema)
- catalog.alterTableDataSchema(table, StructType(existingSchema ++ colsWithProcessedDefaults))
+ val existingDataSchema = CharVarcharUtils.getRawSchema(catalogTable.dataSchema)
+ catalog.alterTableSchema(table,
+ StructType(existingDataSchema ++ colsWithProcessedDefaults ++ catalogTable.partitionSchema))
Seq.empty[Row]
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/v2/CreateVariableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/v2/CreateVariableExec.scala
index 1b9c1711853c3..e625f02b8bbe5 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/v2/CreateVariableExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/v2/CreateVariableExec.scala
@@ -19,6 +19,9 @@ package org.apache.spark.sql.execution.command.v2
import java.util.Locale
+import scala.collection.mutable
+
+import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.{InternalRow, SqlScriptingContextManager}
import org.apache.spark.sql.catalyst.analysis.{FakeLocalCatalog, ResolvedIdentifier}
import org.apache.spark.sql.catalyst.catalog.VariableDefinition
@@ -28,10 +31,10 @@ import org.apache.spark.sql.connector.catalog.Identifier
import org.apache.spark.sql.execution.datasources.v2.LeafV2CommandExec
/**
- * Physical plan node for creating a variable.
+ * Physical plan node for creating variables.
*/
case class CreateVariableExec(
- resolvedIdentifier: ResolvedIdentifier,
+ resolvedIdentifiers: Seq[ResolvedIdentifier],
defaultExpr: DefaultValueExpression,
replace: Boolean) extends LeafV2CommandExec with ExpressionsEvaluator {
@@ -43,25 +46,58 @@ case class CreateVariableExec(
initializeExprs(exprs, 0)
val initValue = Literal(exprs.head.eval(), defaultExpr.dataType)
- val normalizedIdentifier = if (session.sessionState.conf.caseSensitiveAnalysis) {
- resolvedIdentifier.identifier
- } else {
- Identifier.of(
- resolvedIdentifier.identifier.namespace().map(_.toLowerCase(Locale.ROOT)),
- resolvedIdentifier.identifier.name().toLowerCase(Locale.ROOT))
- }
- val varDef = VariableDefinition(normalizedIdentifier, defaultExpr.originalSQL, initValue)
-
- // create local variable if we are in a script, otherwise create session variable
- scriptingVariableManager
- .filter(_ => resolvedIdentifier.catalog == FakeLocalCatalog)
+ val variableTuples = resolvedIdentifiers.map(resolvedIdentifier => {
+ val normalizedIdentifier = if (session.sessionState.conf.caseSensitiveAnalysis) {
+ resolvedIdentifier.identifier
+ } else {
+ Identifier.of(
+ resolvedIdentifier.identifier.namespace().map(_.toLowerCase(Locale.ROOT)),
+ resolvedIdentifier.identifier.name().toLowerCase(Locale.ROOT))
+ }
+ val varDef = VariableDefinition(normalizedIdentifier, defaultExpr.originalSQL, initValue)
+
+ (normalizedIdentifier.namespace().toSeq :+ normalizedIdentifier.name(), varDef)
+ })
+
+ // create local variables if we are in a script, otherwise create session variable
+ val variableManager =
+ scriptingVariableManager
+ .filter(_ => resolvedIdentifiers.head.catalog == FakeLocalCatalog)
// If resolvedIdentifier.catalog is FakeLocalCatalog, scriptingVariableManager
// will always be present.
.getOrElse(tempVariableManager)
- .create(
- normalizedIdentifier.namespace().toSeq :+ normalizedIdentifier.name(),
- varDef,
- replace)
+
+ val uniqueNames = mutable.Set[String]()
+
+ variableTuples.foreach(variable => {
+ val nameParts: Seq[String] = variable._1
+ val name = nameParts.last
+
+ // Check if the variable name was already declared inside the same DECLARE statement
+ if (uniqueNames.contains(name)) {
+ throw new AnalysisException(
+ errorClass = "DUPLICATE_VARIABLE_NAME_INSIDE_DECLARE",
+ messageParameters = Map(
+ "variableName" -> variableManager.getVariableNameForError(name)))
+ }
+
+ // If DECLARE statement does not have OR REPLACE part, check if any of the variable names
+ // declared in the DECLARE statement already exists as a name of another variable
+ if (!replace && variableManager.get(nameParts).isDefined) {
+ throw new AnalysisException(
+ errorClass = "VARIABLE_ALREADY_EXISTS",
+ messageParameters = Map(
+ "variableName" -> variableManager.getVariableNameForError(name)))
+ }
+
+ uniqueNames.add(name)
+ })
+
+ variableTuples.foreach(variable => {
+ val nameParts: Seq[String] = variable._1
+ val varDef: VariableDefinition = variable._2
+ variableManager.create(nameParts, varDef, replace)
+ })
Nil
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/v2/V2CommandStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/v2/V2CommandStrategy.scala
index 3e073202d4c7f..63c4cd9db158a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/v2/V2CommandStrategy.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/v2/V2CommandStrategy.scala
@@ -27,8 +27,12 @@ object V2CommandStrategy extends Strategy {
// TODO: move v2 commands to here which are not data source v2 related.
override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
- case CreateVariable(ident: ResolvedIdentifier, defaultExpr, replace) =>
- CreateVariableExec(ident, defaultExpr, replace) :: Nil
+ case CreateVariable(idents: Seq[LogicalPlan], defaultExpr, replace)
+ if idents.forall(_.isInstanceOf[ResolvedIdentifier]) =>
+ CreateVariableExec(
+ idents.map(_.asInstanceOf[ResolvedIdentifier]),
+ defaultExpr,
+ replace) :: Nil
case DropVariable(ident: ResolvedIdentifier, ifExists) =>
DropVariableExec(ident.identifier.name, ifExists) :: Nil
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala
index e3beffaf920dc..514b64f6abed2 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala
@@ -25,7 +25,7 @@ import org.json4s.jackson.JsonMethods._
import org.apache.spark.SparkException
import org.apache.spark.internal.Logging
import org.apache.spark.sql.{Row, SparkSession}
-import org.apache.spark.sql.catalyst.{SQLConfHelper, TableIdentifier}
+import org.apache.spark.sql.catalyst.{CapturesConfig, SQLConfHelper, TableIdentifier}
import org.apache.spark.sql.catalyst.analysis.{AnalysisContext, GlobalTempView, LocalTempView, SchemaEvolution, SchemaUnsupported, ViewSchemaMode, ViewType}
import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, TemporaryViewRelation}
import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, SubqueryExpression, VariableReference}
@@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.util.CharVarcharUtils
import org.apache.spark.sql.classic.ClassicConversions.castToImpl
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper
import org.apache.spark.sql.errors.QueryCompilationErrors
-import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.internal.StaticSQLConf
import org.apache.spark.sql.types.{MetadataBuilder, StructType}
import org.apache.spark.sql.util.SchemaUtils
import org.apache.spark.util.ArrayImplicits._
@@ -413,37 +413,7 @@ case class ShowViewsCommand(
}
}
-object ViewHelper extends SQLConfHelper with Logging {
-
- private val configPrefixDenyList = Seq(
- SQLConf.MAX_NESTED_VIEW_DEPTH.key,
- "spark.sql.optimizer.",
- "spark.sql.codegen.",
- "spark.sql.execution.",
- "spark.sql.shuffle.",
- "spark.sql.adaptive.",
- // ignore optimization configs used in `RelationConversions`
- "spark.sql.hive.convertMetastoreParquet",
- "spark.sql.hive.convertMetastoreOrc",
- "spark.sql.hive.convertInsertingPartitionedTable",
- "spark.sql.hive.convertInsertingUnpartitionedTable",
- "spark.sql.hive.convertMetastoreCtas",
- SQLConf.ADDITIONAL_REMOTE_REPOSITORIES.key)
-
- private val configAllowList = Seq(
- SQLConf.DISABLE_HINTS.key
- )
-
- /**
- * Capture view config either of:
- * 1. exists in allowList
- * 2. do not exists in denyList
- */
- private def shouldCaptureConfig(key: String): Boolean = {
- configAllowList.exists(prefix => key.equals(prefix)) ||
- !configPrefixDenyList.exists(prefix => key.startsWith(prefix))
- }
-
+object ViewHelper extends SQLConfHelper with Logging with CapturesConfig {
import CatalogTable._
/**
@@ -471,34 +441,6 @@ object ViewHelper extends SQLConfHelper with Logging {
}
}
- /**
- * Get all configurations that are modifiable and should be captured.
- */
- def getModifiedConf(conf: SQLConf): Map[String, String] = {
- conf.getAllConfs.filter { case (k, _) =>
- conf.isModifiable(k) && shouldCaptureConfig(k)
- }
- }
-
- /**
- * Convert the view SQL configs to `properties`.
- */
- private def sqlConfigsToProps(conf: SQLConf): Map[String, String] = {
- val modifiedConfs = getModifiedConf(conf)
- // Some configs have dynamic default values, such as SESSION_LOCAL_TIMEZONE whose
- // default value relies on the JVM system timezone. We need to always capture them to
- // to make sure we apply the same configs when reading the view.
- val alwaysCaptured = Seq(SQLConf.SESSION_LOCAL_TIMEZONE)
- .filter(c => !modifiedConfs.contains(c.key))
- .map(c => (c.key, conf.getConf(c)))
-
- val props = new mutable.HashMap[String, String]
- for ((key, value) <- modifiedConfs ++ alwaysCaptured) {
- props.put(s"$VIEW_SQL_CONFIG_PREFIX$key", value)
- }
- props.toMap
- }
-
/**
* Remove the view SQL configs in `properties`.
*/
@@ -586,13 +528,21 @@ object ViewHelper extends SQLConfHelper with Logging {
// names.
SchemaUtils.checkColumnNameDuplication(fieldNames.toImmutableArraySeq, conf.resolver)
+ val queryColumnNameProps = if (viewSchemaMode == SchemaEvolution) {
+ // If the view schema mode is SCHEMA EVOLUTION, we can avoid generating the query output
+ // column names as table properties, and always use view schema as they are always same
+ Seq()
+ } else {
+ generateQueryColumnNames(queryOutput.toImmutableArraySeq)
+ }
+
// Generate the view default catalog and namespace, as well as captured SQL configs.
val manager = session.sessionState.catalogManager
removeReferredTempNames(removeSQLConfigs(removeQueryColumnNames(properties))) ++
catalogAndNamespaceToProps(
manager.currentCatalog.name, manager.currentNamespace.toImmutableArraySeq) ++
- sqlConfigsToProps(conf) ++
- generateQueryColumnNames(queryOutput.toImmutableArraySeq) ++
+ sqlConfigsToProps(conf, VIEW_SQL_CONFIG_PREFIX) ++
+ queryColumnNameProps ++
referredTempNamesToProps(tempViewNames, tempFunctionNames, tempVariableNames) ++
viewSchemaModeToProps(viewSchemaMode)
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/BasicWriteStatsTracker.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/BasicWriteStatsTracker.scala
index 1858a84213598..9a38cc35b33f8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/BasicWriteStatsTracker.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/BasicWriteStatsTracker.scala
@@ -26,7 +26,7 @@ import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.spark.{SparkContext, TaskContext}
-import org.apache.spark.internal.{Logging, LogKeys, MDC}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.internal.LogKeys.{ACTUAL_NUM_FILES, EXPECTED_NUM_FILES}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.execution.SQLExecution
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/CodecStreams.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/CodecStreams.scala
index 9fddfad249e5b..0ee3a9d90cfe2 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/CodecStreams.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/CodecStreams.scala
@@ -28,20 +28,11 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat
import org.apache.hadoop.util.ReflectionUtils
import org.apache.spark.TaskContext
+import org.apache.spark.io.HadoopCodecStreams
object CodecStreams {
- private def getDecompressionCodec(config: Configuration, file: Path): Option[CompressionCodec] = {
- val compressionCodecs = new CompressionCodecFactory(config)
- Option(compressionCodecs.getCodec(file))
- }
-
def createInputStream(config: Configuration, file: Path): InputStream = {
- val fs = file.getFileSystem(config)
- val inputStream: InputStream = fs.open(file)
-
- getDecompressionCodec(config, file)
- .map(codec => codec.createInputStream(inputStream))
- .getOrElse(inputStream)
+ HadoopCodecStreams.createInputStream(config, file)
}
/**
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
index 489f62bd97769..3816a446a57c7 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
@@ -27,7 +27,7 @@ import org.apache.hadoop.fs.Path
import org.apache.spark.SparkException
import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{CLASS_NAME, DATA_SOURCE, DATA_SOURCES, PATHS}
import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession}
import org.apache.spark.sql.catalyst.DataSourceOptions
@@ -49,7 +49,9 @@ import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2
import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2
import org.apache.spark.sql.execution.datasources.v2.python.PythonDataSourceV2
import org.apache.spark.sql.execution.datasources.xml.XmlFileFormat
-import org.apache.spark.sql.execution.streaming._
+import org.apache.spark.sql.execution.streaming.{Sink, Source}
+import org.apache.spark.sql.execution.streaming.runtime._
+import org.apache.spark.sql.execution.streaming.sinks.FileStreamSink
import org.apache.spark.sql.execution.streaming.sources.{RateStreamProvider, TextSocketSourceProvider}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.sources._
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceManager.scala
index 711e096ebd1f8..790797e1f3183 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceManager.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceManager.scala
@@ -21,7 +21,7 @@ import java.util.Locale
import java.util.concurrent.ConcurrentHashMap
import org.apache.spark.api.python.PythonUtils
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.DATA_SOURCE
import org.apache.spark.sql.errors.QueryCompilationErrors
import org.apache.spark.sql.execution.datasources.v2.python.UserDefinedPythonDataSource
@@ -48,14 +48,13 @@ class DataSourceManager extends Logging {
*/
def registerDataSource(name: String, source: UserDefinedPythonDataSource): Unit = {
val normalizedName = normalize(name)
- if (staticDataSourceBuilders.contains(normalizedName)) {
- // Cannot overwrite static Python Data Sources.
- throw QueryCompilationErrors.dataSourceAlreadyExists(name)
- }
val previousValue = runtimeDataSourceBuilders.put(normalizedName, source)
if (previousValue != null) {
logWarning(log"The data source ${MDC(DATA_SOURCE, name)} replaced a previously " +
log"registered data source.")
+ } else if (staticDataSourceBuilders.contains(normalizedName)) {
+ logWarning(log"The data source ${MDC(DATA_SOURCE, name)} replaced a statically " +
+ log"registered data source.")
}
}
@@ -64,11 +63,7 @@ class DataSourceManager extends Logging {
* it does not exist.
*/
def lookupDataSource(name: String): UserDefinedPythonDataSource = {
- if (dataSourceExists(name)) {
- val normalizedName = normalize(name)
- staticDataSourceBuilders.getOrElse(
- normalizedName, runtimeDataSourceBuilders.get(normalizedName))
- } else {
+ getDataSource(name).getOrElse {
throw QueryCompilationErrors.dataSourceDoesNotExist(name)
}
}
@@ -77,9 +72,14 @@ class DataSourceManager extends Logging {
* Checks if a data source with the specified name exists (case-insensitive).
*/
def dataSourceExists(name: String): Boolean = {
+ getDataSource(name).isDefined
+ }
+
+ private def getDataSource(name: String): Option[UserDefinedPythonDataSource] = {
val normalizedName = normalize(name)
- staticDataSourceBuilders.contains(normalizedName) ||
- runtimeDataSourceBuilders.containsKey(normalizedName)
+ // Runtime registration takes precedence over static.
+ Option(runtimeDataSourceBuilders.get(normalizedName))
+ .orElse(staticDataSourceBuilders.get(normalizedName))
}
override def clone(): DataSourceManager = {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolver.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolver.scala
index ec8182304f791..3e6dc6255d7f2 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolver.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolver.scala
@@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.catalog.UnresolvedCatalogRelation
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.streaming.StreamingRelationV2
import org.apache.spark.sql.classic.SparkSession
-import org.apache.spark.sql.execution.streaming.StreamingRelation
+import org.apache.spark.sql.execution.streaming.runtime.StreamingRelation
/**
* The [[DataSourceResolver]] is a [[Resolver]] extension that resolves nodes defined in the
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
index 3b55a294b21b5..2e47f08ac115a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
@@ -25,7 +25,7 @@ import scala.jdk.CollectionConverters._
import org.apache.hadoop.fs.Path
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.PREDICATES
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{Row, SaveMode}
@@ -37,6 +37,7 @@ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
import org.apache.spark.sql.catalyst.planning.PhysicalOperation
+import org.apache.spark.sql.catalyst.plans.{Inner, JoinType, LeftOuter, RightOuter}
import org.apache.spark.sql.catalyst.plans.logical.{AppendData, InsertIntoDir, InsertIntoStatement, LogicalPlan, Project}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.catalyst.streaming.StreamingRelationV2
@@ -47,12 +48,13 @@ import org.apache.spark.sql.connector.catalog.{SupportsRead, V1Table}
import org.apache.spark.sql.connector.catalog.TableCapability._
import org.apache.spark.sql.connector.expressions.{Expression => V2Expression, NullOrdering, SortDirection, SortOrder => V2SortOrder, SortValue}
import org.apache.spark.sql.connector.expressions.aggregate.{AggregateFunc, Aggregation}
+import org.apache.spark.sql.connector.join.{JoinType => V2JoinType}
import org.apache.spark.sql.errors.QueryCompilationErrors
import org.apache.spark.sql.execution
import org.apache.spark.sql.execution.{RowDataSourceScanExec, SparkPlan}
import org.apache.spark.sql.execution.command._
import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, PushedDownOperators}
-import org.apache.spark.sql.execution.streaming.StreamingRelation
+import org.apache.spark.sql.execution.streaming.runtime.StreamingRelation
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.sources
import org.apache.spark.sql.sources._
@@ -399,7 +401,7 @@ object DataSourceStrategy
l.output.toStructType,
Set.empty,
Set.empty,
- PushedDownOperators(None, None, None, None, Seq.empty, Seq.empty),
+ PushedDownOperators(None, None, None, None, Seq.empty, Seq.empty, Seq.empty, None),
toCatalystRDD(l, baseRelation.buildScan()),
baseRelation,
l.stream,
@@ -474,7 +476,7 @@ object DataSourceStrategy
requestedColumns.toStructType,
pushedFilters.toSet,
handledFilters,
- PushedDownOperators(None, None, None, None, Seq.empty, Seq.empty),
+ PushedDownOperators(None, None, None, None, Seq.empty, Seq.empty, Seq.empty, None),
scanBuilder(requestedColumns, candidatePredicates, pushedFilters),
relation.relation,
relation.stream,
@@ -498,7 +500,7 @@ object DataSourceStrategy
requestedColumns.toStructType,
pushedFilters.toSet,
handledFilters,
- PushedDownOperators(None, None, None, None, Seq.empty, Seq.empty),
+ PushedDownOperators(None, None, None, None, Seq.empty, Seq.empty, Seq.empty, None),
scanBuilder(requestedColumns, candidatePredicates, pushedFilters),
relation.relation,
relation.stream,
@@ -508,6 +510,15 @@ object DataSourceStrategy
}
}
+ def translateJoinType(joinType: JoinType): Option[V2JoinType] = {
+ joinType match {
+ case Inner => Some(V2JoinType.INNER_JOIN)
+ case LeftOuter => Some(V2JoinType.LEFT_OUTER_JOIN)
+ case RightOuter => Some(V2JoinType.RIGHT_OUTER_JOIN)
+ case _ => None
+ }
+ }
+
/**
* Convert RDD of Row into RDD of InternalRow with objects in catalyst types
*/
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala
index 3e66b97f61a63..d43c9eab0a5ba 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala
@@ -17,6 +17,7 @@
package org.apache.spark.sql.execution.datasources
+import java.io.IOException
import java.util.Locale
import scala.jdk.CollectionConverters._
@@ -127,7 +128,7 @@ object DataSourceUtils extends PredicateHelper {
private def getRebaseSpec(
lookupFileMeta: String => String,
- modeByConfig: LegacyBehaviorPolicy.Value,
+ modeByConfig: String,
minVersion: String,
metadataKey: String): RebaseSpec = {
val policy = if (Utils.isTesting &&
@@ -145,7 +146,7 @@ object DataSourceUtils extends PredicateHelper {
} else {
LegacyBehaviorPolicy.CORRECTED
}
- }.getOrElse(modeByConfig)
+ }.getOrElse(LegacyBehaviorPolicy.withName(modeByConfig))
}
policy match {
case LegacyBehaviorPolicy.LEGACY =>
@@ -156,7 +157,7 @@ object DataSourceUtils extends PredicateHelper {
def datetimeRebaseSpec(
lookupFileMeta: String => String,
- modeByConfig: LegacyBehaviorPolicy.Value): RebaseSpec = {
+ modeByConfig: String): RebaseSpec = {
getRebaseSpec(
lookupFileMeta,
modeByConfig,
@@ -166,7 +167,7 @@ object DataSourceUtils extends PredicateHelper {
def int96RebaseSpec(
lookupFileMeta: String => String,
- modeByConfig: LegacyBehaviorPolicy.Value): RebaseSpec = {
+ modeByConfig: String): RebaseSpec = {
getRebaseSpec(
lookupFileMeta,
modeByConfig,
@@ -197,6 +198,11 @@ object DataSourceUtils extends PredicateHelper {
QueryExecutionErrors.sparkUpgradeInWritingDatesError(format, config)
}
+ def shouldIgnoreCorruptFileException(e: Throwable): Boolean = e match {
+ case _: RuntimeException | _: IOException | _: InternalError => true
+ case _ => false
+ }
+
def createDateRebaseFuncInRead(
rebaseMode: LegacyBehaviorPolicy.Value,
format: String): Int => Int = rebaseMode match {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala
index d3078740b819c..8a254b464da71 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala
@@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes
import org.apache.spark.sql.errors.QueryExecutionErrors
-import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.{SessionStateHelper, SQLConf}
import org.apache.spark.sql.sources.Filter
import org.apache.spark.sql.types._
@@ -356,7 +356,7 @@ object FileFormat {
/**
* The base class file format that is based on text file.
*/
-abstract class TextBasedFileFormat extends FileFormat {
+abstract class TextBasedFileFormat extends FileFormat with SessionStateHelper {
private var codecFactory: CompressionCodecFactory = _
override def isSplitable(
@@ -364,8 +364,7 @@ abstract class TextBasedFileFormat extends FileFormat {
options: Map[String, String],
path: Path): Boolean = {
if (codecFactory == null) {
- codecFactory = new CompressionCodecFactory(
- sparkSession.sessionState.newHadoopConfWithOptions(options))
+ codecFactory = new CompressionCodecFactory(getHadoopConf(sparkSession, options))
}
val codec = codecFactory.getCodec(path)
codec == null || codec.isInstanceOf[SplittableCompressionCodec]
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
index 7d071124b0b30..e11c2b15e0541 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
@@ -22,7 +22,7 @@ import org.apache.hadoop.fs.{FileAlreadyExistsException, Path}
import org.apache.hadoop.mapreduce.TaskAttemptContext
import org.apache.spark.TaskOutputFileAlreadyExistException
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{CONFIG, NUM_CONCURRENT_WRITER}
import org.apache.spark.internal.io.{FileCommitProtocol, FileNameSpec}
import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage
@@ -172,7 +172,7 @@ class SingleDirectoryDataWriter(
val currentPath = committer.newTaskTempFile(
taskAttemptContext,
None,
- f"-c$fileCounter%03d" + ext)
+ FileNameSpec("", f"-c$fileCounter%03d" + ext))
currentWriter = description.outputWriterFactory.newInstance(
path = currentPath,
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala
index b4cffa59c98d5..55e2271dc058b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala
@@ -26,7 +26,7 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat
import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
import org.apache.spark._
-import org.apache.spark.internal.{Logging, LogKeys, MDC}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.internal.LogKeys._
import org.apache.spark.internal.io.{FileCommitProtocol, SparkHadoopWriterUtils}
import org.apache.spark.sql.catalyst.InternalRow
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala
index 50af845c37cb1..68a9bbfc7ab1d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala
@@ -21,7 +21,7 @@ import scala.collection.mutable.ArrayBuffer
import scala.math.BigDecimal.RoundingMode
import org.apache.spark.Partition
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{CONFIG, DESIRED_NUM_PARTITIONS, MAX_NUM_PARTITIONS, NUM_PARTITIONS}
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.classic.ClassicConversions._
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala
index e468807f4ffd1..5dc13ccee9ce0 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala
@@ -17,7 +17,7 @@
package org.apache.spark.sql.execution.datasources
-import java.io.{Closeable, FileNotFoundException, IOException}
+import java.io.{Closeable, FileNotFoundException}
import java.net.URI
import org.apache.hadoop.fs.Path
@@ -27,7 +27,6 @@ import org.apache.hadoop.security.AccessControlException
import org.apache.spark.{Partition => RDDPartition, TaskContext}
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.internal.LogKeys.{CURRENT_FILE, PATH}
-import org.apache.spark.internal.MDC
import org.apache.spark.paths.SparkPath
import org.apache.spark.rdd.{InputFileBlockHolder, RDD}
import org.apache.spark.sql.SparkSession
@@ -269,7 +268,8 @@ class FileScanRDD(
// Throw FileNotFoundException even if `ignoreCorruptFiles` is true
case e: FileNotFoundException if !ignoreMissingFiles => throw e
case e @ (_ : AccessControlException | _ : BlockMissingException) => throw e
- case e @ (_: RuntimeException | _: IOException) if ignoreCorruptFiles =>
+ case e if ignoreCorruptFiles &&
+ DataSourceUtils.shouldIgnoreCorruptFileException(e) =>
logWarning(log"Skipped the rest of the content in the corrupted file: " +
log"${MDC(PATH, currentFile)}", e)
finished = true
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
index 7291da248294a..396375890c249 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
@@ -21,7 +21,7 @@ import java.util.Locale
import scala.collection.mutable
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{NUM_PRUNED, POST_SCAN_FILTERS, PUSHED_FILTERS, TOTAL}
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.catalog.BucketSpec
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileStatusCache.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileStatusCache.scala
index 2bb8476a9f0e8..cf182ec18f1c9 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileStatusCache.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileStatusCache.scala
@@ -25,7 +25,7 @@ import scala.jdk.CollectionConverters._
import com.google.common.cache._
import org.apache.hadoop.fs.{FileStatus, Path}
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{CACHED_TABLE_PARTITION_METADATA_SIZE, MAX_TABLE_PARTITION_METADATA_SIZE}
import org.apache.spark.sql.SparkSession
import org.apache.spark.util.SizeEstimator
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala
index bf6da8765e516..be13dc340f48d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala
@@ -25,6 +25,7 @@ import org.apache.hadoop.mapreduce._
import org.apache.hadoop.mapreduce.lib.input.{FileSplit, LineRecordReader}
import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.util.Utils
/**
@@ -57,10 +58,13 @@ class HadoopFileLinesReader(
val hadoopAttemptContext = new TaskAttemptContextImpl(conf, attemptId)
Utils.tryInitializeResource(
- lineSeparator match {
- case Some(sep) => new LineRecordReader(sep)
- // If the line separator is `None`, it covers `\r`, `\r\n` and `\n`.
- case _ => new LineRecordReader()
+ // HadoopLineRecordReader is LineRecordReader with some changes like support for specifying
+ // compression codec as opposed to always infer from file extension.
+ // If the line separator is `None`, it covers `\r`, `\r\n` and `\n`.
+ if (SQLConf.get.hadoopLineRecordReaderEnabled) {
+ new HadoopLineRecordReader(lineSeparator.orNull)
+ } else {
+ new LineRecordReader(lineSeparator.orNull)
}
) { reader =>
reader.initialize(fileSplit, hadoopAttemptContext)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala
index 3b8a20c7cf741..2d68faa3ff526 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala
@@ -24,13 +24,13 @@ import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs._
import org.apache.hadoop.mapred.{FileInputFormat, JobConf}
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{COUNT, ELAPSED_TIME}
import org.apache.spark.metrics.source.HiveCatalogMetrics
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.FileSourceOptions
import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
-import org.apache.spark.sql.execution.streaming.FileStreamSink
+import org.apache.spark.sql.execution.streaming.sinks.FileStreamSink
import org.apache.spark.sql.types.StructType
import org.apache.spark.util.HadoopFSUtils
@@ -103,7 +103,8 @@ class InMemoryFileIndex(
}
override def equals(other: Any): Boolean = other match {
- case hdfs: InMemoryFileIndex => rootPaths.toSet == hdfs.rootPaths.toSet
+ case hdfs: InMemoryFileIndex if rootPaths.size == hdfs.rootPaths.size =>
+ rootPaths.sorted == hdfs.rootPaths.sorted
case _ => false
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala
index 07be3f89872cc..1bf0d2f0301f2 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala
@@ -24,7 +24,7 @@ import scala.collection.mutable
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs._
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{COUNT, PERCENT, TOTAL}
import org.apache.spark.paths.SparkPath
import org.apache.spark.sql.SparkSession
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SQLHadoopMapReduceCommitProtocol.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SQLHadoopMapReduceCommitProtocol.scala
index 03e988eb0bd2b..db9a65a3814db 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SQLHadoopMapReduceCommitProtocol.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SQLHadoopMapReduceCommitProtocol.scala
@@ -21,7 +21,7 @@ import org.apache.hadoop.fs.Path
import org.apache.hadoop.mapreduce.{OutputCommitter, TaskAttemptContext}
import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.CLASS_NAME
import org.apache.spark.internal.io.HadoopMapReduceCommitProtocol
import org.apache.spark.sql.internal.SQLConf
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormat.scala
index a1f2e16449244..57e0efb993fb7 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormat.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormat.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources.binaryfile
import java.sql.Timestamp
-import com.google.common.io.{ByteStreams, Closeables}
+import com.google.common.io.Closeables
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileStatus, Path}
import org.apache.hadoop.mapreduce.Job
@@ -31,6 +31,7 @@ import org.apache.spark.sql.catalyst.types.DataTypeUtils
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.errors.QueryExecutionErrors
import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriterFactory, PartitionedFile}
+import org.apache.spark.sql.internal.SessionStateHelper
import org.apache.spark.sql.internal.SQLConf.SOURCES_BINARY_FILE_MAX_LENGTH
import org.apache.spark.sql.sources.{And, DataSourceRegister, EqualTo, Filter, GreaterThan, GreaterThanOrEqual, LessThan, LessThanOrEqual, Not, Or}
import org.apache.spark.sql.types._
@@ -55,7 +56,8 @@ import org.apache.spark.util.SerializableConfiguration
* .load("/path/to/fileDir");
* }}}
*/
-case class BinaryFileFormat() extends FileFormat with DataSourceRegister {
+case class BinaryFileFormat() extends FileFormat
+ with DataSourceRegister with SessionStateHelper {
import BinaryFileFormat._
@@ -98,7 +100,7 @@ case class BinaryFileFormat() extends FileFormat with DataSourceRegister {
val broadcastedHadoopConf =
SerializableConfiguration.broadcast(sparkSession.sparkContext, hadoopConf)
val filterFuncs = filters.flatMap(filter => createFilterFunction(filter))
- val maxLength = sparkSession.sessionState.conf.getConf(SOURCES_BINARY_FILE_MAX_LENGTH)
+ val maxLength = getSqlConf(sparkSession).getConf(SOURCES_BINARY_FILE_MAX_LENGTH)
file: PartitionedFile => {
val path = file.toPath
@@ -118,7 +120,7 @@ case class BinaryFileFormat() extends FileFormat with DataSourceRegister {
}
val stream = fs.open(status.getPath)
try {
- writer.write(i, ByteStreams.toByteArray(stream))
+ writer.write(i, stream.readAllBytes())
} finally {
Closeables.close(stream, true)
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala
index c6b9764bee2c9..596edc8beaa34 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala
@@ -30,7 +30,7 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
import org.apache.spark.TaskContext
import org.apache.spark.input.{PortableDataStream, StreamInputFormat}
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.PATH
import org.apache.spark.paths.SparkPath
import org.apache.spark.rdd.{BinaryFileRDD, RDD}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala
index 8aaeae3ae952f..bf189268b4d6a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala
@@ -43,23 +43,15 @@ case class CSVFileFormat() extends TextBasedFileFormat with DataSourceRegister {
sparkSession: SparkSession,
options: Map[String, String],
path: Path): Boolean = {
- val parsedOptions = new CSVOptions(
- options,
- columnPruning = sparkSession.sessionState.conf.csvColumnPruning,
- sparkSession.sessionState.conf.sessionLocalTimeZone)
- val csvDataSource = CSVDataSource(parsedOptions)
- csvDataSource.isSplitable && super.isSplitable(sparkSession, options, path)
+ val parsedOptions = getCsvOptions(sparkSession, options)
+ CSVDataSource(parsedOptions).isSplitable && super.isSplitable(sparkSession, options, path)
}
override def inferSchema(
sparkSession: SparkSession,
options: Map[String, String],
files: Seq[FileStatus]): Option[StructType] = {
- val parsedOptions = new CSVOptions(
- options,
- columnPruning = sparkSession.sessionState.conf.csvColumnPruning,
- sparkSession.sessionState.conf.sessionLocalTimeZone)
-
+ val parsedOptions = getCsvOptions(sparkSession, options)
CSVDataSource(parsedOptions).inferSchema(sparkSession, files, parsedOptions)
}
@@ -76,13 +68,9 @@ case class CSVFileFormat() extends TextBasedFileFormat with DataSourceRegister {
throw QueryCompilationErrors.dataTypeUnsupportedByDataSourceError("CSV", field)
}
}
- val conf = job.getConfiguration
- val csvOptions = new CSVOptions(
- options,
- columnPruning = sparkSession.sessionState.conf.csvColumnPruning,
- sparkSession.sessionState.conf.sessionLocalTimeZone)
- csvOptions.compressionCodec.foreach { codec =>
- CompressionCodecs.setCodecConfiguration(conf, codec)
+ val parsedOptions = getCsvOptions(sparkSession, options)
+ parsedOptions.compressionCodec.foreach { codec =>
+ CompressionCodecs.setCodecConfiguration(job.getConfiguration, codec)
}
new OutputWriterFactory {
@@ -90,11 +78,11 @@ case class CSVFileFormat() extends TextBasedFileFormat with DataSourceRegister {
path: String,
dataSchema: StructType,
context: TaskAttemptContext): OutputWriter = {
- new CsvOutputWriter(path, dataSchema, context, csvOptions)
+ new CsvOutputWriter(path, dataSchema, context, parsedOptions)
}
override def getFileExtension(context: TaskAttemptContext): String = {
- "." + csvOptions.extension + CodecStreams.getCompressionExtension(context)
+ "." + parsedOptions.extension + CodecStreams.getCompressionExtension(context)
}
}
}
@@ -109,11 +97,7 @@ case class CSVFileFormat() extends TextBasedFileFormat with DataSourceRegister {
hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = {
val broadcastedHadoopConf =
SerializableConfiguration.broadcast(sparkSession.sparkContext, hadoopConf)
- val parsedOptions = new CSVOptions(
- options,
- sparkSession.sessionState.conf.csvColumnPruning,
- sparkSession.sessionState.conf.sessionLocalTimeZone,
- sparkSession.sessionState.conf.columnNameOfCorruptRecord)
+ val parsedOptions = getCsvOptions(sparkSession, options)
val isColumnPruningEnabled = parsedOptions.isColumnPruningEnabled(requiredSchema)
// Check a field requirement for corrupt records here to throw an exception in a driver side
@@ -180,4 +164,15 @@ case class CSVFileFormat() extends TextBasedFileFormat with DataSourceRegister {
}
override def allowDuplicatedColumnNames: Boolean = true
+
+ private def getCsvOptions(
+ sparkSession: SparkSession,
+ options: Map[String, String]): CSVOptions = {
+ val conf = getSqlConf(sparkSession)
+ new CSVOptions(
+ options,
+ conf.csvColumnPruning,
+ conf.sessionLocalTimeZone,
+ conf.columnNameOfCorruptRecord)
+ }
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCDatabaseMetadata.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCDatabaseMetadata.scala
new file mode 100644
index 0000000000000..7aa184f57fc00
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCDatabaseMetadata.scala
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.jdbc
+
+import java.sql.Connection
+
+import scala.util.control.NonFatal
+
+import org.apache.spark.internal.Logging
+
+/**
+ * Object that contains metadata about the external database.
+ * The metadata is static database information such as the version, or the version
+ * of the JDBC driver.
+ *
+ * This object is stored in JDBCRDD.
+ */
+case class JDBCDatabaseMetadata(
+ databaseMajorVersion: Option[Int],
+ databaseMinorVersion: Option[Int],
+ databaseDriverMajorVersion: Option[Int],
+ databaseDriverMinorVersion: Option[Int]
+ )
+
+/**
+ * Companion object for DatabaseMetadata.
+ * Contains factory methods to build instances.
+ */
+object JDBCDatabaseMetadata extends Logging {
+
+ /**
+ * Safely retrieves a piece of metadata.
+ *
+ * @param f A function that retrieves an integer value from DatabaseMetaData.
+ * @return Some(value) on success, None on failure.
+ */
+ private def safeGet(f: => Int): Option[Int] = {
+ try {
+ Some(f)
+ } catch {
+ case NonFatal(e) =>
+ logWarning(log"Exception while getting specific database metadata", e)
+ None
+ }
+ }
+
+ /**
+ * Creates a DatabaseMetadata instance from a JDBC Connection,
+ * handling errors for each field individually.
+ *
+ * @param getConnection A JDBC connection factory.
+ * @return A new instance of DatabaseMetadata containing the version metadata.
+ */
+ def fromJDBCConnectionFactory(getConnection: Int => Connection): JDBCDatabaseMetadata = {
+ var conn: Connection = null
+
+ def closeConnection(): Unit = {
+ try {
+ if (null != conn) {
+ conn.close()
+ }
+ logInfo("closed connection during metadata fetch")
+ } catch {
+ case e: Exception => logWarning("Exception closing connection during metadata fetch", e)
+ }
+ }
+
+ try {
+ conn = getConnection(-1)
+ // getMetaData itself can throw, so we catch that and return None for all fields
+ val databaseMetadata = conn.getMetaData
+
+ JDBCDatabaseMetadata(
+ databaseMajorVersion = safeGet(databaseMetadata.getDatabaseMajorVersion),
+ databaseMinorVersion = safeGet(databaseMetadata.getDatabaseMinorVersion),
+ databaseDriverMajorVersion = safeGet(databaseMetadata.getDriverMajorVersion),
+ databaseDriverMinorVersion = safeGet(databaseMetadata.getDriverMinorVersion)
+ )
+ } catch {
+ case NonFatal(e) =>
+ logWarning(log"Exception while getting database metadata object from connection", e)
+ JDBCDatabaseMetadata(None, None, None, None)
+ } finally {
+ closeConnection()
+ }
+ }
+}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala
index f0c638b7d07c8..3c85b6e65dee7 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala
@@ -100,7 +100,7 @@ class JDBCOptions(
if (subquery.isEmpty) {
throw QueryExecutionErrors.emptyOptionError(JDBC_QUERY_STRING)
} else {
- s"(${subquery}) SPARK_GEN_SUBQ_${curId.getAndIncrement()}"
+ s"(${subquery.trim.replaceAll(";+$", "")}) SPARK_GEN_SUBQ_${curId.getAndIncrement()}"
}
}
@@ -215,6 +215,10 @@ class JDBCOptions(
// This only applies to Data Source V2 JDBC
val pushDownTableSample = parameters.getOrElse(JDBC_PUSHDOWN_TABLESAMPLE, "true").toBoolean
+ // An option to allow/disallow pushing down JOIN into JDBC data source
+ // This only applies to Data Source V2 JDBC
+ val pushDownJoin = parameters.getOrElse(JDBC_PUSHDOWN_JOIN, "true").toBoolean
+
// The local path of user's keytab file, which is assumed to be pre-uploaded to all nodes either
// by --files option of spark-submit or manually
val keytab = {
@@ -321,6 +325,7 @@ object JDBCOptions {
val JDBC_PUSHDOWN_LIMIT = newOption("pushDownLimit")
val JDBC_PUSHDOWN_OFFSET = newOption("pushDownOffset")
val JDBC_PUSHDOWN_TABLESAMPLE = newOption("pushDownTableSample")
+ val JDBC_PUSHDOWN_JOIN = newOption("pushDownJoin")
val JDBC_KEYTAB = newOption("keytab")
val JDBC_PRINCIPAL = newOption("principal")
val JDBC_TABLE_COMMENT = newOption("tableComment")
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala
index 8342ae06da01a..33cb5342fd3d1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala
@@ -23,7 +23,7 @@ import scala.util.Using
import scala.util.control.NonFatal
import org.apache.spark.{InterruptibleIterator, Partition, SparkContext, SparkException, TaskContext}
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.SQL_TEXT
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
@@ -54,7 +54,7 @@ object JDBCRDD extends Logging {
* @throws java.sql.SQLException if the table specification is garbage.
* @throws java.sql.SQLException if the table contains an unsupported type.
*/
- def resolveTable(options: JDBCOptions): StructType = {
+ def resolveTable(options: JDBCOptions, conn: Connection): StructType = {
val url = options.url
val prepareQuery = options.prepareQuery
val table = options.tableOrQuery
@@ -62,7 +62,7 @@ object JDBCRDD extends Logging {
val fullQuery = prepareQuery + dialect.getSchemaQuery(table)
try {
- getQueryOutputSchema(fullQuery, options, dialect)
+ getQueryOutputSchema(fullQuery, options, dialect, conn)
} catch {
case e: SQLException if dialect.isSyntaxErrorBestEffort(e) =>
throw new SparkException(
@@ -72,19 +72,31 @@ object JDBCRDD extends Logging {
}
}
+ def resolveTable(options: JDBCOptions): StructType = {
+ JdbcUtils.withConnection(options) {
+ resolveTable(options, _)
+ }
+ }
+
def getQueryOutputSchema(
- query: String, options: JDBCOptions, dialect: JdbcDialect): StructType = {
- Using.resource(dialect.createConnectionFactory(options)(-1)) { conn =>
- Using.resource(conn.prepareStatement(query)) { statement =>
- statement.setQueryTimeout(options.queryTimeout)
- Using.resource(statement.executeQuery()) { rs =>
- JdbcUtils.getSchema(conn, rs, dialect, alwaysNullable = true,
- isTimestampNTZ = options.preferTimestampNTZ)
- }
+ query: String, options: JDBCOptions, dialect: JdbcDialect, conn: Connection): StructType = {
+ logInfo(log"Generated JDBC query to get scan output schema: ${MDC(SQL_TEXT, query)}")
+ Using.resource(conn.prepareStatement(query)) { statement =>
+ statement.setQueryTimeout(options.queryTimeout)
+ Using.resource(statement.executeQuery()) { rs =>
+ JdbcUtils.getSchema(conn, rs, dialect, alwaysNullable = true,
+ isTimestampNTZ = options.preferTimestampNTZ)
}
}
}
+ def getQueryOutputSchema(
+ query: String, options: JDBCOptions, dialect: JdbcDialect): StructType = {
+ JdbcUtils.withConnection(options) {
+ getQueryOutputSchema(query, options, dialect, _)
+ }
+ }
+
/**
* Prune all but the specified columns from the specified Catalyst schema.
*
@@ -130,7 +142,8 @@ object JDBCRDD extends Logging {
sample: Option[TableSampleInfo] = None,
limit: Int = 0,
sortOrders: Array[String] = Array.empty[String],
- offset: Int = 0): RDD[InternalRow] = {
+ offset: Int = 0,
+ additionalMetrics: Map[String, SQLMetric] = Map()): RDD[InternalRow] = {
val url = options.url
val dialect = JdbcDialects.get(url)
val quotedColumns = if (groupByColumns.isEmpty) {
@@ -139,20 +152,24 @@ object JDBCRDD extends Logging {
// these are already quoted in JDBCScanBuilder
requiredColumns
}
+ val connectionFactory = dialect.createConnectionFactory(options)
+
new JDBCRDD(
sc,
- dialect.createConnectionFactory(options),
+ connectionFactory,
outputSchema.getOrElse(pruneSchema(schema, requiredColumns)),
quotedColumns,
predicates,
parts,
url,
options,
+ databaseMetadata = JDBCDatabaseMetadata.fromJDBCConnectionFactory(connectionFactory),
groupByColumns,
sample,
limit,
sortOrders,
- offset)
+ offset,
+ additionalMetrics)
}
// scalastyle:on argcount
}
@@ -171,11 +188,13 @@ class JDBCRDD(
partitions: Array[Partition],
url: String,
options: JDBCOptions,
+ databaseMetadata: JDBCDatabaseMetadata,
groupByColumns: Option[Array[String]],
sample: Option[TableSampleInfo],
limit: Int,
sortOrders: Array[String],
- offset: Int)
+ offset: Int,
+ additionalMetrics: Map[String, SQLMetric])
extends RDD[InternalRow](sc, Nil) with DataSourceMetricsMixin with ExternalEngineDatasourceRDD {
/**
@@ -185,6 +204,17 @@ class JDBCRDD(
sparkContext,
name = "JDBC query execution time")
+ /**
+ * Time needed to fetch the data and transform it into Spark's InternalRow format.
+ *
+ * Usually this is spent in network transfer time, but it can be spent in transformation time
+ * as well if we are transforming some more complex datatype such as structs.
+ */
+ val fetchAndTransformToInternalRowsMetric: SQLMetric = SQLMetrics.createNanoTimingMetric(
+ sparkContext,
+ // Message that user sees does not have to leak details about conversion
+ name = "JDBC remote data fetch and translation time")
+
private lazy val dialect = JdbcDialects.get(url)
def generateJdbcQuery(partition: Option[JDBCPartition]): String = {
@@ -219,6 +249,11 @@ class JDBCRDD(
generateJdbcQuery(partition = None)
}
+ /**
+ * Get the external engine database metadata.
+ */
+ def getDatabaseMetadata: JDBCDatabaseMetadata = databaseMetadata
+
/**
* Runs the SQL query against the JDBC driver.
*/
@@ -287,28 +322,31 @@ class JDBCRDD(
}
val sqlText = generateJdbcQuery(Some(part))
+ logInfo(log"Generated JDBC query to fetch data: ${MDC(SQL_TEXT, sqlText)}")
stmt = conn.prepareStatement(sqlText,
ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY)
stmt.setFetchSize(options.fetchSize)
stmt.setQueryTimeout(options.queryTimeout)
- val startTime = System.nanoTime
- rs = try {
- stmt.executeQuery()
- } catch {
- case e: SQLException if dialect.isSyntaxErrorBestEffort(e) =>
- throw new SparkException(
- errorClass = "JDBC_EXTERNAL_ENGINE_SYNTAX_ERROR.DURING_QUERY_EXECUTION",
- messageParameters = Map("jdbcQuery" -> sqlText),
- cause = e)
+ rs = SQLMetrics.withTimingNs(queryExecutionTimeMetric) {
+ try {
+ stmt.executeQuery()
+ } catch {
+ case e: SQLException if dialect.isSyntaxErrorBestEffort(e) =>
+ throw new SparkException(
+ errorClass = "JDBC_EXTERNAL_ENGINE_SYNTAX_ERROR.DURING_QUERY_EXECUTION",
+ messageParameters = Map("jdbcQuery" -> sqlText),
+ cause = e)
+ }
}
- val endTime = System.nanoTime
-
- val executionTime = endTime - startTime
- queryExecutionTimeMetric.add(executionTime)
val rowsIterator =
- JdbcUtils.resultSetToSparkInternalRows(rs, dialect, schema, inputMetrics)
+ JdbcUtils.resultSetToSparkInternalRows(
+ rs,
+ dialect,
+ schema,
+ inputMetrics,
+ Some(fetchAndTransformToInternalRowsMetric))
CompletionIterator[InternalRow, Iterator[InternalRow]](
new InterruptibleIterator(context, rowsIterator), close())
@@ -316,7 +354,8 @@ class JDBCRDD(
override def getMetrics: Seq[(String, SQLMetric)] = {
Seq(
+ "fetchAndTransformToInternalRowsNs" -> fetchAndTransformToInternalRowsMetric,
"queryExecutionTime" -> queryExecutionTimeMetric
- )
+ ) ++ additionalMetrics
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala
index 2c4158dfe1533..05e30207314a7 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala
@@ -21,7 +21,7 @@ import scala.collection.mutable.ArrayBuffer
import scala.math.BigDecimal.RoundingMode
import org.apache.spark.Partition
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{CLAUSES, LOWER_BOUND, NEW_VALUE, NUM_PARTITIONS, OLD_VALUE, UPPER_BOUND}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{DataFrame, Row, SaveMode, SparkSession, SQLContext}
@@ -31,6 +31,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate
import org.apache.spark.sql.connector.expressions.filter.Predicate
import org.apache.spark.sql.errors.QueryCompilationErrors
import org.apache.spark.sql.execution.datasources.v2.TableSampleInfo
+import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.jdbc.JdbcDialects
import org.apache.spark.sql.sources._
@@ -48,6 +49,9 @@ private[sql] case class JDBCPartitioningInfo(
numPartitions: Int)
private[sql] object JDBCRelation extends Logging {
+
+ val schemaFetchName = "Remote JDBC schema fetch time"
+ val schemaFetchKey = "remoteSchemaFetchTime"
/**
* Given a partitioning schematic (a column of integral type, a number of
* partitions, and upper and lower bounds on the column's value), generate
@@ -255,15 +259,20 @@ private[sql] object JDBCRelation extends Logging {
parts: Array[Partition],
jdbcOptions: JDBCOptions)(
sparkSession: SparkSession): JDBCRelation = {
- val schema = JDBCRelation.getSchema(sparkSession.sessionState.conf.resolver, jdbcOptions)
- JDBCRelation(schema, parts, jdbcOptions)(sparkSession)
+ val remoteSchemaFetchMetric = JdbcUtils.createSchemaFetchMetric(sparkSession.sparkContext)
+ val schema = SQLMetrics.withTimingNs(remoteSchemaFetchMetric) {
+ JDBCRelation.getSchema(sparkSession.sessionState.conf.resolver, jdbcOptions)
+ }
+ JDBCRelation(schema, parts, jdbcOptions,
+ Map(schemaFetchKey -> remoteSchemaFetchMetric))(sparkSession)
}
}
private[sql] case class JDBCRelation(
override val schema: StructType,
parts: Array[Partition],
- jdbcOptions: JDBCOptions)(@transient val sparkSession: SparkSession)
+ jdbcOptions: JDBCOptions,
+ additionalMetrics: Map[String, SQLMetric] = Map())(@transient val sparkSession: SparkSession)
extends BaseRelation
with PrunedFilteredScan
with InsertableRelation {
@@ -296,7 +305,8 @@ private[sql] case class JDBCRelation(
requiredColumns,
pushedPredicates,
parts,
- jdbcOptions).asInstanceOf[RDD[Row]]
+ jdbcOptions,
+ additionalMetrics = additionalMetrics).asInstanceOf[RDD[Row]]
}
def buildScan(
@@ -321,7 +331,8 @@ private[sql] case class JDBCRelation(
tableSample,
limit,
sortOrders,
- offset).asInstanceOf[RDD[Row]]
+ offset,
+ additionalMetrics).asInstanceOf[RDD[Row]]
}
override def insert(data: DataFrame, overwrite: Boolean): Unit = {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala
index d9be1a1e3f674..6b9d0520f2eb5 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala
@@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.datasources.jdbc
import org.apache.spark.sql.{DataFrame, SaveMode, SQLContext}
import org.apache.spark.sql.errors.QueryCompilationErrors
import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils._
+import org.apache.spark.sql.execution.metric.SQLMetrics
import org.apache.spark.sql.jdbc.JdbcDialects
import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, DataSourceRegister, RelationProvider}
@@ -35,9 +36,17 @@ class JdbcRelationProvider extends CreatableRelationProvider
val sparkSession = sqlContext.sparkSession
val resolver = sparkSession.sessionState.conf.resolver
val timeZoneId = sparkSession.sessionState.conf.sessionLocalTimeZone
- val schema = JDBCRelation.getSchema(resolver, jdbcOptions)
+ val remoteSchemaFetchMetric = JdbcUtils.createSchemaFetchMetric(sparkSession.sparkContext)
+ val schema = SQLMetrics.withTimingNs(remoteSchemaFetchMetric) {
+ JDBCRelation.getSchema(resolver, jdbcOptions)
+ }
val parts = JDBCRelation.columnPartition(schema, resolver, timeZoneId, jdbcOptions)
- JDBCRelation(schema, parts, jdbcOptions)(sparkSession)
+ JDBCRelation(
+ schema,
+ parts,
+ jdbcOptions,
+ Map(JDBCRelation.schemaFetchKey -> remoteSchemaFetchMetric)
+ )(sparkSession)
}
override def createRelation(
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
index 0077012e2b0e4..926c133d2875c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
@@ -29,9 +29,9 @@ import scala.jdk.CollectionConverters._
import scala.util.{Failure, Success, Try}
import scala.util.control.NonFatal
-import org.apache.spark.{SparkThrowable, SparkUnsupportedOperationException, TaskContext}
+import org.apache.spark.{SparkContext, SparkThrowable, SparkUnsupportedOperationException, TaskContext}
import org.apache.spark.executor.InputMetrics
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{DEFAULT_ISOLATION_LEVEL, ISOLATION_LEVEL}
import org.apache.spark.sql.{DataFrame, Row}
import org.apache.spark.sql.catalyst.{InternalRow, SQLConfHelper}
@@ -46,6 +46,7 @@ import org.apache.spark.sql.connector.catalog.{Identifier, TableChange}
import org.apache.spark.sql.connector.catalog.index.{SupportsIndex, TableIndex}
import org.apache.spark.sql.connector.expressions.NamedReference
import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects, JdbcType, NoopDialect}
import org.apache.spark.sql.types._
import org.apache.spark.sql.util.SchemaUtils
@@ -357,7 +358,8 @@ object JdbcUtils extends Logging with SQLConfHelper {
resultSet: ResultSet,
dialect: JdbcDialect,
schema: StructType,
- inputMetrics: InputMetrics): Iterator[InternalRow] = {
+ inputMetrics: InputMetrics,
+ fetchAndTransformToInternalRowsMetric: Option[SQLMetric] = None): Iterator[InternalRow] = {
new NextIterator[InternalRow] {
private[this] val rs = resultSet
private[this] val getters: Array[JDBCValueGetter] = makeGetters(dialect, schema)
@@ -372,7 +374,7 @@ object JdbcUtils extends Logging with SQLConfHelper {
}
}
- override protected def getNext(): InternalRow = {
+ private def getNextWithoutTiming: InternalRow = {
if (rs.next()) {
inputMetrics.incRecordsRead(1)
var i = 0
@@ -387,9 +389,26 @@ object JdbcUtils extends Logging with SQLConfHelper {
null.asInstanceOf[InternalRow]
}
}
+
+ override protected def getNext(): InternalRow = {
+ if (fetchAndTransformToInternalRowsMetric.isDefined) {
+ SQLMetrics.withTimingNs(fetchAndTransformToInternalRowsMetric.get) {
+ getNextWithoutTiming
+ }
+ } else {
+ getNextWithoutTiming
+ }
+ }
}
}
+ def createSchemaFetchMetric(sparkContext: SparkContext): SQLMetric = {
+ SQLMetrics.createNanoTimingMetric(
+ sparkContext,
+ JDBCRelation.schemaFetchName
+ )
+ }
+
// A `JDBCValueGetter` is responsible for getting a value from `ResultSet` into a field
// for `MutableRow`. The last argument `Int` means the index for the value to be set in
// the row and also used for the value in `ResultSet`.
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala
index bedf5ec62e4ee..aaa5af478dbb7 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala
@@ -20,7 +20,6 @@ package org.apache.spark.sql.execution.datasources.json
import java.io.InputStream
import com.fasterxml.jackson.core.{JsonFactory, JsonParser}
-import com.google.common.io.ByteStreams
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileStatus, Path}
import org.apache.hadoop.io.Text
@@ -222,7 +221,7 @@ object MultiLineJsonDataSource extends JsonDataSource {
CodecStreams.createInputStreamWithCloseResource(conf, file.toPath)
}
} { inputStream =>
- UTF8String.fromBytes(ByteStreams.toByteArray(inputStream))
+ UTF8String.fromBytes(inputStream.readAllBytes())
}
}
val streamParser = parser.options.encoding
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala
index ed096cf289b56..e3b78ef432505 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala
@@ -39,22 +39,15 @@ case class JsonFileFormat() extends TextBasedFileFormat with DataSourceRegister
sparkSession: SparkSession,
options: Map[String, String],
path: Path): Boolean = {
- val parsedOptions = new JSONOptionsInRead(
- options,
- sparkSession.sessionState.conf.sessionLocalTimeZone,
- sparkSession.sessionState.conf.columnNameOfCorruptRecord)
- val jsonDataSource = JsonDataSource(parsedOptions)
- jsonDataSource.isSplitable && super.isSplitable(sparkSession, options, path)
+ val parsedOptions = getJsonOptions(sparkSession, options)
+ JsonDataSource(parsedOptions).isSplitable && super.isSplitable(sparkSession, options, path)
}
override def inferSchema(
sparkSession: SparkSession,
options: Map[String, String],
files: Seq[FileStatus]): Option[StructType] = {
- val parsedOptions = new JSONOptionsInRead(
- options,
- sparkSession.sessionState.conf.sessionLocalTimeZone,
- sparkSession.sessionState.conf.columnNameOfCorruptRecord)
+ val parsedOptions = getJsonOptions(sparkSession, options)
JsonDataSource(parsedOptions).inferSchema(sparkSession, files, parsedOptions)
}
@@ -63,13 +56,9 @@ case class JsonFileFormat() extends TextBasedFileFormat with DataSourceRegister
job: Job,
options: Map[String, String],
dataSchema: StructType): OutputWriterFactory = {
- val conf = job.getConfiguration
- val parsedOptions = new JSONOptions(
- options,
- sparkSession.sessionState.conf.sessionLocalTimeZone,
- sparkSession.sessionState.conf.columnNameOfCorruptRecord)
+ val parsedOptions = getJsonOptions(sparkSession, options, inRead = false)
parsedOptions.compressionCodec.foreach { codec =>
- CompressionCodecs.setCodecConfiguration(conf, codec)
+ CompressionCodecs.setCodecConfiguration(job.getConfiguration, codec)
}
new OutputWriterFactory {
@@ -96,12 +85,7 @@ case class JsonFileFormat() extends TextBasedFileFormat with DataSourceRegister
hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = {
val broadcastedHadoopConf =
SerializableConfiguration.broadcast(sparkSession.sparkContext, hadoopConf)
-
- val parsedOptions = new JSONOptionsInRead(
- options,
- sparkSession.sessionState.conf.sessionLocalTimeZone,
- sparkSession.sessionState.conf.columnNameOfCorruptRecord)
-
+ val parsedOptions = getJsonOptions(sparkSession, options)
val actualSchema =
StructType(requiredSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord))
// Check a field requirement for corrupt records here to throw an exception in a driver side
@@ -147,4 +131,16 @@ case class JsonFileFormat() extends TextBasedFileFormat with DataSourceRegister
case _ => false
}
+
+ private def getJsonOptions(
+ spark: SparkSession,
+ options: Map[String, String],
+ inRead: Boolean = true): JSONOptions = {
+ val conf = getSqlConf(spark)
+ if (inRead) {
+ new JSONOptionsInRead(options, conf.sessionLocalTimeZone, conf.columnNameOfCorruptRecord)
+ } else {
+ new JSONOptions(options, conf.sessionLocalTimeZone, conf.columnNameOfCorruptRecord)
+ }
+ }
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonOutputWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonOutputWriter.scala
index 5727c502a7097..4abfd96993785 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonOutputWriter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonOutputWriter.scala
@@ -21,7 +21,7 @@ import java.nio.charset.{Charset, StandardCharsets}
import org.apache.hadoop.fs.Path
import org.apache.hadoop.mapreduce.TaskAttemptContext
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{ENCODING, PATH}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.json.{JacksonGenerator, JSONOptions, JSONOptionsInRead}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala
index a662ea3b8d2df..c44c5f780a6be 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala
@@ -27,6 +27,7 @@ import org.apache.spark.sql.internal.connector.{SimpleTableProvider, SupportsStr
import org.apache.spark.sql.sources.DataSourceRegister
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.util.CaseInsensitiveStringMap
+import org.apache.spark.util.Utils
/**
* This is no-op datasource. It does not do anything besides consuming its input.
@@ -48,6 +49,7 @@ private[noop] object NoopTable extends Table with SupportsWrite {
TableCapability.TRUNCATE,
TableCapability.ACCEPT_ANY_SCHEMA)
}
+ override def toString: String = Utils.getFormattedClassName(this)
}
private[noop] object NoopWriteBuilder extends WriteBuilder
@@ -59,6 +61,7 @@ private[noop] object NoopWriteBuilder extends WriteBuilder
private[noop] object NoopWrite extends Write {
override def toBatch: BatchWrite = NoopBatchWrite
override def toStreaming: StreamingWrite = NoopStreamingWrite
+ override def toString: String = Utils.getFormattedClassName(this)
}
private[noop] object NoopBatchWrite extends BatchWrite {
@@ -67,6 +70,7 @@ private[noop] object NoopBatchWrite extends BatchWrite {
override def useCommitCoordinator(): Boolean = false
override def commit(messages: Array[WriterCommitMessage]): Unit = {}
override def abort(messages: Array[WriterCommitMessage]): Unit = {}
+ override def toString: String = Utils.getFormattedClassName(this)
}
private[noop] object NoopWriterFactory extends DataWriterFactory {
@@ -86,6 +90,7 @@ private[noop] object NoopStreamingWrite extends StreamingWrite {
override def useCommitCoordinator(): Boolean = false
override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {}
override def abort(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {}
+ override def toString: String = Utils.getFormattedClassName(this)
}
private[noop] object NoopStreamingDataWriterFactory extends StreamingDataWriterFactory {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala
index 86528bf7a0af9..40d10d0e44034 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala
@@ -38,6 +38,7 @@ import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes
import org.apache.spark.sql.execution.datasources._
+import org.apache.spark.sql.internal.SessionStateHelper
import org.apache.spark.sql.sources._
import org.apache.spark.sql.types._
import org.apache.spark.util.{SerializableConfiguration, Utils}
@@ -48,6 +49,7 @@ import org.apache.spark.util.{SerializableConfiguration, Utils}
class OrcFileFormat
extends FileFormat
with DataSourceRegister
+ with SessionStateHelper
with Serializable {
override def shortName(): String = "orc"
@@ -70,7 +72,8 @@ class OrcFileFormat
job: Job,
options: Map[String, String],
dataSchema: StructType): OutputWriterFactory = {
- val orcOptions = new OrcOptions(options, sparkSession.sessionState.conf)
+ val sqlConf = getSqlConf(sparkSession)
+ val orcOptions = new OrcOptions(options, sqlConf)
val conf = job.getConfiguration
@@ -79,7 +82,7 @@ class OrcFileFormat
conf.asInstanceOf[JobConf]
.setOutputFormat(classOf[org.apache.orc.mapred.OrcOutputFormat[OrcStruct]])
- val batchSize = sparkSession.sessionState.conf.orcVectorizedWriterBatchSize
+ val batchSize = sqlConf.orcVectorizedWriterBatchSize
new OutputWriterFactory {
override def newInstance(
@@ -101,10 +104,10 @@ class OrcFileFormat
}
override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = {
- val conf = sparkSession.sessionState.conf
- conf.orcVectorizedReaderEnabled &&
+ val sqlConf = getSqlConf(sparkSession)
+ sqlConf.orcVectorizedReaderEnabled &&
schema.forall(s => OrcUtils.supportColumnarReads(
- s.dataType, sparkSession.sessionState.conf.orcVectorizedReaderNestedColumnEnabled))
+ s.dataType, sqlConf.orcVectorizedReaderNestedColumnEnabled))
}
override def isSplitable(
@@ -136,7 +139,7 @@ class OrcFileFormat
hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = {
val resultSchema = StructType(requiredSchema.fields ++ partitionSchema.fields)
- val sqlConf = sparkSession.sessionState.conf
+ val sqlConf = getSqlConf(sparkSession)
val capacity = sqlConf.orcVectorizedReaderBatchSize
// Should always be set by FileSourceScanExec creating this.
@@ -163,8 +166,8 @@ class OrcFileFormat
val broadcastedConf =
SerializableConfiguration.broadcast(sparkSession.sparkContext, hadoopConf)
- val isCaseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis
- val orcFilterPushDown = sparkSession.sessionState.conf.orcFilterPushDown
+ val isCaseSensitive = sqlConf.caseSensitiveAnalysis
+ val orcFilterPushDown = sqlConf.orcFilterPushDown
(file: PartitionedFile) => {
val conf = broadcastedConf.value.value
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala
index 50c28c783b4cd..20c40f65b0686 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala
@@ -31,7 +31,7 @@ import org.apache.orc.{BooleanColumnStatistics, ColumnStatistics, DateColumnStat
import org.apache.spark.{SPARK_VERSION_SHORT, SparkException}
import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.PATH
import org.apache.spark.sql.{SPARK_VERSION_METADATA_KEY, SparkSession}
import org.apache.spark.sql.catalyst.{FileSourceOptions, InternalRow}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
index c71d55a41c764..be6e5d1886670 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
@@ -32,7 +32,7 @@ import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GRO
import org.apache.parquet.hadoop._
import org.apache.spark.TaskContext
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{PATH, SCHEMA}
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
@@ -44,7 +44,7 @@ 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.internal.{SessionStateHelper, SQLConf}
import org.apache.spark.sql.sources._
import org.apache.spark.sql.types._
import org.apache.spark.util.{SerializableConfiguration, ThreadUtils}
@@ -52,6 +52,7 @@ import org.apache.spark.util.{SerializableConfiguration, ThreadUtils}
class ParquetFileFormat
extends FileFormat
with DataSourceRegister
+ with SessionStateHelper
with Logging
with Serializable {
@@ -68,7 +69,7 @@ class ParquetFileFormat
job: Job,
options: Map[String, String],
dataSchema: StructType): OutputWriterFactory = {
- val sqlConf = sparkSession.sessionState.conf
+ val sqlConf = getSqlConf(sparkSession)
val parquetOptions = new ParquetOptions(options, sqlConf)
ParquetUtils.prepareWrite(sqlConf, job, dataSchema, parquetOptions)
}
@@ -84,8 +85,7 @@ class ParquetFileFormat
* 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)
+ ParquetUtils.isBatchReadSupportedForSchema(getSqlConf(sparkSession), schema)
}
override def vectorTypes(
@@ -128,6 +128,7 @@ class ParquetFileFormat
filters: Seq[Filter],
options: Map[String, String],
hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = {
+ val sqlConf = getSqlConf(sparkSession)
hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName)
hadoopConf.set(
ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA,
@@ -137,27 +138,27 @@ class ParquetFileFormat
requiredSchema.json)
hadoopConf.set(
SQLConf.SESSION_LOCAL_TIMEZONE.key,
- sparkSession.sessionState.conf.sessionLocalTimeZone)
+ sqlConf.sessionLocalTimeZone)
hadoopConf.setBoolean(
SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key,
- sparkSession.sessionState.conf.nestedSchemaPruningEnabled)
+ sqlConf.nestedSchemaPruningEnabled)
hadoopConf.setBoolean(
SQLConf.CASE_SENSITIVE.key,
- sparkSession.sessionState.conf.caseSensitiveAnalysis)
+ sqlConf.caseSensitiveAnalysis)
// Sets flags for `ParquetToSparkSchemaConverter`
hadoopConf.setBoolean(
SQLConf.PARQUET_BINARY_AS_STRING.key,
- sparkSession.sessionState.conf.isParquetBinaryAsString)
+ sqlConf.isParquetBinaryAsString)
hadoopConf.setBoolean(
SQLConf.PARQUET_INT96_AS_TIMESTAMP.key,
- sparkSession.sessionState.conf.isParquetINT96AsTimestamp)
+ sqlConf.isParquetINT96AsTimestamp)
hadoopConf.setBoolean(
SQLConf.PARQUET_INFER_TIMESTAMP_NTZ_ENABLED.key,
- sparkSession.sessionState.conf.parquetInferTimestampNTZEnabled)
+ sqlConf.parquetInferTimestampNTZEnabled)
hadoopConf.setBoolean(
SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key,
- sparkSession.sessionState.conf.legacyParquetNanosAsLong)
+ sqlConf.legacyParquetNanosAsLong)
val broadcastedHadoopConf =
@@ -167,7 +168,6 @@ class ParquetFileFormat
// 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)
@@ -181,13 +181,13 @@ class ParquetFileFormat
val pushDownStringPredicate = sqlConf.parquetFilterPushDownStringPredicate
val pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold
val isCaseSensitive = sqlConf.caseSensitiveAnalysis
- val parquetOptions = new ParquetOptions(options, sparkSession.sessionState.conf)
+ val parquetOptions = new ParquetOptions(options, sqlConf)
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 &&
+ val returningBatch = sqlConf.parquetVectorizedReaderEnabled &&
options.getOrElse(FileFormat.OPTION_RETURNING_BATCH,
throw new IllegalArgumentException(
"OPTION_RETURNING_BATCH should always be set for ParquetFileFormat. " +
@@ -380,11 +380,12 @@ object ParquetFileFormat extends Logging {
private[parquet] def readSchema(
footers: Seq[Footer], sparkSession: SparkSession): Option[StructType] = {
+ val sqlConf = SessionStateHelper.getSqlConf(sparkSession)
val converter = new ParquetToSparkSchemaConverter(
- sparkSession.sessionState.conf.isParquetBinaryAsString,
- sparkSession.sessionState.conf.isParquetINT96AsTimestamp,
- inferTimestampNTZ = sparkSession.sessionState.conf.parquetInferTimestampNTZEnabled,
- nanosAsLong = sparkSession.sessionState.conf.legacyParquetNanosAsLong)
+ sqlConf.isParquetBinaryAsString,
+ sqlConf.isParquetINT96AsTimestamp,
+ inferTimestampNTZ = sqlConf.parquetInferTimestampNTZEnabled,
+ nanosAsLong = sqlConf.legacyParquetNanosAsLong)
val seen = mutable.HashSet[String]()
val finalSchemas: Seq[StructType] = footers.flatMap { footer =>
@@ -478,10 +479,11 @@ object ParquetFileFormat extends Logging {
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 sqlConf = SessionStateHelper.getSqlConf(sparkSession)
+ val assumeBinaryIsString = sqlConf.isParquetBinaryAsString
+ val assumeInt96IsTimestamp = sqlConf.isParquetINT96AsTimestamp
+ val inferTimestampNTZ = sqlConf.parquetInferTimestampNTZEnabled
+ val nanosAsLong = sqlConf.legacyParquetNanosAsLong
val reader = (files: Seq[FileStatus], conf: Configuration, ignoreCorruptFiles: Boolean) => {
// Converter used to convert Parquet `MessageType` to Spark SQL `StructType`
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala
index 565742671b9cd..4a9b17bf98e59 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala
@@ -22,6 +22,7 @@ import java.math.{BigDecimal => JBigDecimal}
import java.nio.charset.StandardCharsets.UTF_8
import java.sql.{Date, Timestamp}
import java.time.{Duration, Instant, LocalDate, LocalTime, Period}
+import java.time.temporal.ChronoField.MICRO_OF_DAY
import java.util.HashSet
import java.util.Locale
@@ -149,7 +150,7 @@ class ParquetFilters(
ParquetSchemaType(LogicalTypeAnnotation.timestampType(true, TimeUnit.MICROS), INT64, 0)
private val ParquetTimestampMillisType =
ParquetSchemaType(LogicalTypeAnnotation.timestampType(true, TimeUnit.MILLIS), INT64, 0)
- private val ParquetTimeType =
+ private val ParquetTimeMicrosType =
ParquetSchemaType(LogicalTypeAnnotation.timeType(false, TimeUnit.MICROS), INT64, 0)
private def dateToDays(date: Any): Int = {
@@ -176,7 +177,7 @@ class ParquetFilters(
}
private def localTimeToMicros(v: Any): JLong = {
- DateTimeUtils.localTimeToMicros(v.asInstanceOf[LocalTime])
+ v.asInstanceOf[LocalTime].getLong(MICRO_OF_DAY)
}
private def decimalToInt32(decimal: JBigDecimal): Integer = decimal.unscaledValue().intValue()
@@ -213,7 +214,7 @@ class ParquetFilters(
private def toLongValue(v: Any): JLong = v match {
case d: Duration => IntervalUtils.durationToMicros(d)
- case lt: LocalTime => DateTimeUtils.localTimeToMicros(lt)
+ case lt: LocalTime => localTimeToMicros(lt)
case l => l.asInstanceOf[JLong]
}
@@ -251,7 +252,7 @@ class ParquetFilters(
(n: Array[String], v: Any) => FilterApi.eq(
longColumn(n),
Option(v).map(timestampToMillis).orNull)
- case ParquetTimeType =>
+ case ParquetTimeMicrosType =>
(n: Array[String], v: Any) => FilterApi.eq(
longColumn(n),
Option(v).map(localTimeToMicros).orNull)
@@ -304,7 +305,7 @@ class ParquetFilters(
(n: Array[String], v: Any) => FilterApi.notEq(
longColumn(n),
Option(v).map(timestampToMillis).orNull)
- case ParquetTimeType =>
+ case ParquetTimeMicrosType =>
(n: Array[String], v: Any) => FilterApi.notEq(
longColumn(n),
Option(v).map(localTimeToMicros).orNull)
@@ -348,7 +349,7 @@ class ParquetFilters(
(n: Array[String], v: Any) => FilterApi.lt(longColumn(n), timestampToMicros(v))
case ParquetTimestampMillisType if pushDownTimestamp =>
(n: Array[String], v: Any) => FilterApi.lt(longColumn(n), timestampToMillis(v))
- case ParquetTimeType =>
+ case ParquetTimeMicrosType =>
(n: Array[String], v: Any) => FilterApi.lt(longColumn(n), localTimeToMicros(v))
case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT32, _) if pushDownDecimal =>
@@ -387,7 +388,7 @@ class ParquetFilters(
(n: Array[String], v: Any) => FilterApi.ltEq(longColumn(n), timestampToMicros(v))
case ParquetTimestampMillisType if pushDownTimestamp =>
(n: Array[String], v: Any) => FilterApi.ltEq(longColumn(n), timestampToMillis(v))
- case ParquetTimeType =>
+ case ParquetTimeMicrosType =>
(n: Array[String], v: Any) => FilterApi.ltEq(longColumn(n), localTimeToMicros(v))
case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT32, _) if pushDownDecimal =>
@@ -426,7 +427,7 @@ class ParquetFilters(
(n: Array[String], v: Any) => FilterApi.gt(longColumn(n), timestampToMicros(v))
case ParquetTimestampMillisType if pushDownTimestamp =>
(n: Array[String], v: Any) => FilterApi.gt(longColumn(n), timestampToMillis(v))
- case ParquetTimeType =>
+ case ParquetTimeMicrosType =>
(n: Array[String], v: Any) => FilterApi.gt(longColumn(n), localTimeToMicros(v))
case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT32, _) if pushDownDecimal =>
@@ -465,7 +466,7 @@ class ParquetFilters(
(n: Array[String], v: Any) => FilterApi.gtEq(longColumn(n), timestampToMicros(v))
case ParquetTimestampMillisType if pushDownTimestamp =>
(n: Array[String], v: Any) => FilterApi.gtEq(longColumn(n), timestampToMillis(v))
- case ParquetTimeType =>
+ case ParquetTimeMicrosType =>
(n: Array[String], v: Any) => FilterApi.gtEq(longColumn(n), localTimeToMicros(v))
case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT32, _) if pushDownDecimal =>
@@ -556,7 +557,7 @@ class ParquetFilters(
}
FilterApi.in(longColumn(n), set)
- case ParquetTimeType =>
+ case ParquetTimeMicrosType =>
(n: Array[String], values: Array[Any]) =>
val set = new HashSet[JLong]()
for (value <- values) {
@@ -661,7 +662,7 @@ class ParquetFilters(
value.isInstanceOf[Date] || value.isInstanceOf[LocalDate]
case ParquetTimestampMicrosType | ParquetTimestampMillisType =>
value.isInstanceOf[Timestamp] || value.isInstanceOf[Instant]
- case ParquetTimeType => value.isInstanceOf[LocalTime]
+ case ParquetTimeMicrosType => value.isInstanceOf[LocalTime]
case ParquetSchemaType(decimalType: DecimalLogicalTypeAnnotation, INT32, _) =>
isDecimalMatched(value, decimalType)
case ParquetSchemaType(decimalType: DecimalLogicalTypeAnnotation, INT64, _) =>
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala
index eaedd99d8628c..dd5669bda07c9 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala
@@ -24,7 +24,7 @@ import org.apache.parquet.hadoop.ParquetOutputFormat
import org.apache.spark.sql.catalyst.{DataSourceOptions, FileSourceOptions}
import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
import org.apache.spark.sql.errors.QueryExecutionErrors
-import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf}
+import org.apache.spark.sql.internal.SQLConf
/**
* Options for the Parquet data source.
@@ -74,16 +74,15 @@ class ParquetOptions(
/**
* The rebasing mode for the DATE and TIMESTAMP_MICROS, TIMESTAMP_MILLIS values in reads.
*/
- def datetimeRebaseModeInRead: LegacyBehaviorPolicy.Value = parameters
+ def datetimeRebaseModeInRead: String = parameters
.get(DATETIME_REBASE_MODE)
- .map(LegacyBehaviorPolicy.withName)
- .getOrElse(sqlConf.getConf(SQLConf.PARQUET_REBASE_MODE_IN_READ))
+ .getOrElse(sqlConf.getConf(SQLConf.PARQUET_REBASE_MODE_IN_READ).toString)
/**
* The rebasing mode for INT96 timestamp values in reads.
*/
- def int96RebaseModeInRead: LegacyBehaviorPolicy.Value = parameters
- .get(INT96_REBASE_MODE).map(LegacyBehaviorPolicy.withName)
- .getOrElse(sqlConf.getConf(SQLConf.PARQUET_INT96_REBASE_MODE_IN_READ))
+ def int96RebaseModeInRead: String = parameters
+ .get(INT96_REBASE_MODE)
+ .getOrElse(sqlConf.getConf(SQLConf.PARQUET_INT96_REBASE_MODE_IN_READ).toString)
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala
index af0bf0d51f077..09fd0eccec4b3 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala
@@ -423,7 +423,7 @@ object ParquetReadSupport extends Logging {
caseSensitiveParquetFieldMap
.get(f.name)
.map(clipParquetType(_, f.dataType, caseSensitive, useFieldId))
- .getOrElse(toParquet.convertField(f))
+ .getOrElse(toParquet.convertField(f, inShredded = false))
}
def matchCaseInsensitiveField(f: StructField): Type = {
@@ -439,7 +439,7 @@ object ParquetReadSupport extends Logging {
} else {
clipParquetType(parquetTypes.head, f.dataType, caseSensitive, useFieldId)
}
- }.getOrElse(toParquet.convertField(f))
+ }.getOrElse(toParquet.convertField(f, inShredded = false))
}
def matchIdField(f: StructField): Type = {
@@ -458,7 +458,7 @@ object ParquetReadSupport extends Logging {
}.getOrElse {
// When there is no ID match, we use a fake name to avoid a name match by accident
// We need this name to be unique as well, otherwise there will be type conflicts
- toParquet.convertField(f.copy(name = generateFakeColumnName))
+ toParquet.convertField(f.copy(name = generateFakeColumnName), inShredded = false)
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala
index 0927f5c3c963c..cb5e7bf532157 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala
@@ -487,7 +487,8 @@ private[parquet] class ParquetRowConverter(
.asInstanceOf[TimeLogicalTypeAnnotation].getUnit == TimeUnit.MICROS =>
new ParquetPrimitiveConverter(updater) {
override def addLong(value: Long): Unit = {
- this.updater.setLong(value)
+ val nanos = DateTimeUtils.microsToNanos(value)
+ this.updater.setLong(nanos)
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala
index e05d5fe2fd88a..0df21e2a52295 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala
@@ -196,8 +196,7 @@ class ParquetToSparkSchemaConverter(
field: ColumnIO,
sparkReadType: Option[DataType] = None): ParquetColumn = {
val targetType = sparkReadType.map {
- case udt: UserDefinedType[_] => udt.sqlType
- case otherType => otherType
+ _.transformRecursively { case t: UserDefinedType[_] => t.sqlType }
}
field match {
case primitiveColumn: PrimitiveColumnIO => convertPrimitiveField(primitiveColumn, targetType)
@@ -550,15 +549,16 @@ class SparkToParquetSchemaConverter(
def convert(catalystSchema: StructType): MessageType = {
Types
.buildMessage()
- .addFields(catalystSchema.map(convertField): _*)
+ .addFields(catalystSchema.map(f => convertField(f, inShredded = false)): _*)
.named(ParquetSchemaConverter.SPARK_PARQUET_SCHEMA_NAME)
}
/**
- * Converts a Spark SQL [[StructField]] to a Parquet [[Type]].
+ * Converts a Spark SQL [[StructField]] to a Parquet [[Type]]. `inShredded` indicates whether
+ * the field is within a shredded Variant schema.
*/
- def convertField(field: StructField): Type = {
- val converted = convertField(field, if (field.nullable) OPTIONAL else REQUIRED)
+ def convertField(field: StructField, inShredded: Boolean): Type = {
+ val converted = convertField(field, if (field.nullable) OPTIONAL else REQUIRED, inShredded)
if (useFieldId && ParquetUtils.hasFieldId(field)) {
converted.withId(ParquetUtils.getFieldId(field))
} else {
@@ -566,7 +566,10 @@ class SparkToParquetSchemaConverter(
}
}
- private def convertField(field: StructField, repetition: Type.Repetition): Type = {
+ private def convertField(
+ field: StructField,
+ repetition: Type.Repetition,
+ inShredded: Boolean): Type = {
field.dataType match {
// ===================
@@ -622,16 +625,26 @@ class SparkToParquetSchemaConverter(
// from Spark 1.5.0, we resort to a timestamp type with microsecond precision so that we can
// store a timestamp into a `Long`. This design decision is subject to change though, for
// example, we may resort to nanosecond precision in the future.
+ //
+ // The Parquet Variant Shredding spec states that timestamps coming from shredded Variants
+ // must be stored as INT64, and the Spark Timestamp type always has microsecond precision,
+ // therefore, the TIMESTAMP_MICROS configuration is used when writing shredded variant
+ // timestamp subfields.
case TimestampType =>
- outputTimestampType match {
- case SQLConf.ParquetOutputTimestampType.INT96 =>
- Types.primitive(INT96, repetition).named(field.name)
- case SQLConf.ParquetOutputTimestampType.TIMESTAMP_MICROS =>
- Types.primitive(INT64, repetition)
- .as(LogicalTypeAnnotation.timestampType(true, TimeUnit.MICROS)).named(field.name)
- case SQLConf.ParquetOutputTimestampType.TIMESTAMP_MILLIS =>
- Types.primitive(INT64, repetition)
- .as(LogicalTypeAnnotation.timestampType(true, TimeUnit.MILLIS)).named(field.name)
+ if (inShredded) {
+ Types.primitive(INT64, repetition)
+ .as(LogicalTypeAnnotation.timestampType(true, TimeUnit.MICROS)).named(field.name)
+ } else {
+ outputTimestampType match {
+ case SQLConf.ParquetOutputTimestampType.INT96 =>
+ Types.primitive(INT96, repetition).named(field.name)
+ case SQLConf.ParquetOutputTimestampType.TIMESTAMP_MICROS =>
+ Types.primitive(INT64, repetition)
+ .as(LogicalTypeAnnotation.timestampType(true, TimeUnit.MICROS)).named(field.name)
+ case SQLConf.ParquetOutputTimestampType.TIMESTAMP_MILLIS =>
+ Types.primitive(INT64, repetition)
+ .as(LogicalTypeAnnotation.timestampType(true, TimeUnit.MILLIS)).named(field.name)
+ }
}
case TimestampNTZType =>
@@ -707,7 +720,7 @@ class SparkToParquetSchemaConverter(
.buildGroup(repetition).as(LogicalTypeAnnotation.listType())
.addField(Types
.buildGroup(REPEATED)
- .addField(convertField(StructField("array", elementType, nullable)))
+ .addField(convertField(StructField("array", elementType, nullable), inShredded))
.named("bag"))
.named(field.name)
@@ -723,7 +736,7 @@ class SparkToParquetSchemaConverter(
Types
.buildGroup(repetition).as(LogicalTypeAnnotation.listType())
// "array" is the name chosen by parquet-avro (1.7.0 and prior version)
- .addField(convertField(StructField("array", elementType, nullable), REPEATED))
+ .addField(convertField(StructField("array", elementType, nullable), REPEATED, inShredded))
.named(field.name)
// Spark 1.4.x and prior versions convert MapType into a 3-level group annotated by
@@ -738,8 +751,8 @@ class SparkToParquetSchemaConverter(
ConversionPatterns.mapType(
repetition,
field.name,
- convertField(StructField("key", keyType, nullable = false)),
- convertField(StructField("value", valueType, valueContainsNull)))
+ convertField(StructField("key", keyType, nullable = false), inShredded),
+ convertField(StructField("value", valueType, valueContainsNull), inShredded))
// =====================================
// ArrayType and MapType (standard mode)
@@ -755,7 +768,7 @@ class SparkToParquetSchemaConverter(
.buildGroup(repetition).as(LogicalTypeAnnotation.listType())
.addField(
Types.repeatedGroup()
- .addField(convertField(StructField("element", elementType, containsNull)))
+ .addField(convertField(StructField("element", elementType, containsNull), inShredded))
.named("list"))
.named(field.name)
@@ -771,9 +784,10 @@ class SparkToParquetSchemaConverter(
.addField(
Types
.repeatedGroup()
- .addField(convertField(StructField("key", keyType, nullable = false)))
- .addField(convertField(StructField("value", valueType, valueContainsNull)))
- .named("key_value"))
+ .addField(convertField(StructField("key", keyType, nullable = false), inShredded))
+ .addField(
+ convertField(StructField("value", valueType, valueContainsNull), inShredded)
+ ).named("key_value"))
.named(field.name)
// ===========
@@ -782,25 +796,25 @@ class SparkToParquetSchemaConverter(
case VariantType =>
Types.buildGroup(repetition)
- .addField(convertField(StructField("value", BinaryType, nullable = false)))
- .addField(convertField(StructField("metadata", BinaryType, nullable = false)))
+ .addField(convertField(StructField("value", BinaryType, nullable = false), inShredded))
+ .addField(convertField(StructField("metadata", BinaryType, nullable = false), inShredded))
.named(field.name)
case s: StructType if SparkShreddingUtils.isVariantShreddingStruct(s) =>
// Variant struct takes a Variant and writes to Parquet as a shredded schema.
val group = Types.buildGroup(repetition)
s.fields.foreach { f =>
- group.addField(convertField(f))
+ group.addField(convertField(f, inShredded = true))
}
group.named(field.name)
case StructType(fields) =>
fields.foldLeft(Types.buildGroup(repetition)) { (builder, field) =>
- builder.addField(convertField(field))
+ builder.addField(convertField(field, inShredded))
}.named(field.name)
case udt: UserDefinedType[_] =>
- convertField(field.copy(dataType = udt.sqlType))
+ convertField(field.copy(dataType = udt.sqlType), inShredded)
case _ =>
throw QueryCompilationErrors.cannotConvertDataTypeToParquetTypeError(field)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala
index bad883534115c..5db3b0671db9f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala
@@ -33,7 +33,7 @@ import org.apache.parquet.schema.{PrimitiveType, Types}
import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName
import org.apache.spark.{SparkException, SparkUnsupportedOperationException}
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{CLASS_NAME, CONFIG}
import org.apache.spark.sql.{Row, SparkSession}
import org.apache.spark.sql.catalyst.InternalRow
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala
index 4022f7ea30032..2ab9fb64da43d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala
@@ -120,7 +120,8 @@ class ParquetWriteSupport extends WriteSupport[InternalRow] with Logging {
SQLConf.ParquetOutputTimestampType.withName(configuration.get(key))
}
- this.rootFieldWriters = shreddedSchema.map(_.dataType).map(makeWriter).toArray[ValueWriter]
+ this.rootFieldWriters =
+ shreddedSchema.map(_.dataType).map(makeWriter(_, inShredded = false)).toArray[ValueWriter]
val messageType = new SparkToParquetSchemaConverter(configuration).convert(shreddedSchema)
val metadata = Map(
@@ -187,7 +188,9 @@ class ParquetWriteSupport extends WriteSupport[InternalRow] with Logging {
}
}
- private def makeWriter(dataType: DataType): ValueWriter = {
+ // `inShredded` indicates whether the current traversal is nested within a shredded Variant
+ // schema. This affects how timestamp values are written.
+ private def makeWriter(dataType: DataType, inShredded: Boolean): ValueWriter = {
dataType match {
case BooleanType =>
(row: SpecializedGetters, ordinal: Int) =>
@@ -209,7 +212,7 @@ class ParquetWriteSupport extends WriteSupport[InternalRow] with Logging {
(row: SpecializedGetters, ordinal: Int) =>
recordConsumer.addInteger(row.getInt(ordinal))
- case LongType | _: DayTimeIntervalType | _: TimeType =>
+ case LongType | _: DayTimeIntervalType =>
(row: SpecializedGetters, ordinal: Int) =>
recordConsumer.addLong(row.getLong(ordinal))
@@ -227,25 +230,34 @@ class ParquetWriteSupport extends WriteSupport[InternalRow] with Logging {
Binary.fromReusedByteArray(row.getUTF8String(ordinal).getBytes))
case TimestampType =>
- outputTimestampType match {
- case SQLConf.ParquetOutputTimestampType.INT96 =>
- (row: SpecializedGetters, ordinal: Int) =>
- val micros = int96RebaseFunc(row.getLong(ordinal))
- val (julianDay, timeOfDayNanos) = DateTimeUtils.toJulianDay(micros)
- val buf = ByteBuffer.wrap(timestampBuffer)
- buf.order(ByteOrder.LITTLE_ENDIAN).putLong(timeOfDayNanos).putInt(julianDay)
- recordConsumer.addBinary(Binary.fromReusedByteArray(timestampBuffer))
-
- case SQLConf.ParquetOutputTimestampType.TIMESTAMP_MICROS =>
- (row: SpecializedGetters, ordinal: Int) =>
- val micros = row.getLong(ordinal)
- recordConsumer.addLong(timestampRebaseFunc(micros))
-
- case SQLConf.ParquetOutputTimestampType.TIMESTAMP_MILLIS =>
- (row: SpecializedGetters, ordinal: Int) =>
- val micros = row.getLong(ordinal)
- val millis = DateTimeUtils.microsToMillis(timestampRebaseFunc(micros))
- recordConsumer.addLong(millis)
+ if (inShredded) {
+ // The Parquet Variant Shredding schema states that timestamp types must be stored as
+ // INT64. The Spark Timestamp type always has microsecond granularity and therefore, we
+ // always write shredded timestamps in the TIMESTAMP_MICROS configuration.
+ (row: SpecializedGetters, ordinal: Int) =>
+ val micros = row.getLong(ordinal)
+ recordConsumer.addLong(timestampRebaseFunc(micros))
+ } else {
+ outputTimestampType match {
+ case SQLConf.ParquetOutputTimestampType.INT96 =>
+ (row: SpecializedGetters, ordinal: Int) =>
+ val micros = int96RebaseFunc(row.getLong(ordinal))
+ val (julianDay, timeOfDayNanos) = DateTimeUtils.toJulianDay(micros)
+ val buf = ByteBuffer.wrap(timestampBuffer)
+ buf.order(ByteOrder.LITTLE_ENDIAN).putLong(timeOfDayNanos).putInt(julianDay)
+ recordConsumer.addBinary(Binary.fromReusedByteArray(timestampBuffer))
+
+ case SQLConf.ParquetOutputTimestampType.TIMESTAMP_MICROS =>
+ (row: SpecializedGetters, ordinal: Int) =>
+ val micros = row.getLong(ordinal)
+ recordConsumer.addLong(timestampRebaseFunc(micros))
+
+ case SQLConf.ParquetOutputTimestampType.TIMESTAMP_MILLIS =>
+ (row: SpecializedGetters, ordinal: Int) =>
+ val micros = row.getLong(ordinal)
+ val millis = DateTimeUtils.microsToMillis(timestampRebaseFunc(micros))
+ recordConsumer.addLong(millis)
+ }
}
case TimestampNTZType =>
@@ -253,6 +265,10 @@ class ParquetWriteSupport extends WriteSupport[InternalRow] with Logging {
// MICROS time unit.
(row: SpecializedGetters, ordinal: Int) => recordConsumer.addLong(row.getLong(ordinal))
+ case _: TimeType =>
+ (row: SpecializedGetters, ordinal: Int) =>
+ recordConsumer.addLong(DateTimeUtils.nanosToMicros(row.getLong(ordinal)))
+
case BinaryType =>
(row: SpecializedGetters, ordinal: Int) =>
recordConsumer.addBinary(Binary.fromReusedByteArray(row.getBinary(ordinal)))
@@ -273,7 +289,8 @@ class ParquetWriteSupport extends WriteSupport[InternalRow] with Logging {
}
case s: StructType if SparkShreddingUtils.isVariantShreddingStruct(s) =>
- val fieldWriters = s.map(_.dataType).map(makeWriter).toArray[ValueWriter]
+ val fieldWriters =
+ s.map(_.dataType).map(makeWriter(_, inShredded = true)).toArray[ValueWriter]
val variantShreddingSchema = SparkShreddingUtils.buildVariantSchema(s)
(row: SpecializedGetters, ordinal: Int) =>
val v = row.getVariant(ordinal)
@@ -284,17 +301,18 @@ class ParquetWriteSupport extends WriteSupport[InternalRow] with Logging {
}
case t: StructType =>
- val fieldWriters = t.map(_.dataType).map(makeWriter).toArray[ValueWriter]
+ val fieldWriters =
+ t.map(_.dataType).map(makeWriter(_, inShredded)).toArray[ValueWriter]
(row: SpecializedGetters, ordinal: Int) =>
consumeGroup {
writeFields(row.getStruct(ordinal, t.length), t, fieldWriters)
}
- case t: ArrayType => makeArrayWriter(t)
+ case t: ArrayType => makeArrayWriter(t, inShredded)
- case t: MapType => makeMapWriter(t)
+ case t: MapType => makeMapWriter(t, inShredded)
- case t: UserDefinedType[_] => makeWriter(t.sqlType)
+ case t: UserDefinedType[_] => makeWriter(t.sqlType, inShredded)
case _ => throw SparkException.internalError(s"Unsupported data type $dataType.")
}
@@ -373,8 +391,9 @@ class ParquetWriteSupport extends WriteSupport[InternalRow] with Logging {
}
}
- def makeArrayWriter(arrayType: ArrayType): ValueWriter = {
- val elementWriter = makeWriter(arrayType.elementType)
+ def makeArrayWriter(arrayType: ArrayType, inShredded: Boolean): ValueWriter = {
+ // The shredded schema should not have an array inside
+ val elementWriter = makeWriter(arrayType.elementType, inShredded)
def threeLevelArrayWriter(repeatedGroupName: String, elementFieldName: String): ValueWriter =
(row: SpecializedGetters, ordinal: Int) => {
@@ -453,9 +472,12 @@ class ParquetWriteSupport extends WriteSupport[InternalRow] with Logging {
}
}
- private def makeMapWriter(mapType: MapType): ValueWriter = {
- val keyWriter = makeWriter(mapType.keyType)
- val valueWriter = makeWriter(mapType.valueType)
+ private def makeMapWriter(mapType: MapType, inShredded: Boolean): ValueWriter = {
+ // TODO: If maps are ever supported in the shredded schema, we should add a test in
+ // `ParquetVariantShreddingSuite` to make sure that timestamps within maps are shredded
+ // correctly as INT64.
+ val keyWriter = makeWriter(mapType.keyType, inShredded)
+ val valueWriter = makeWriter(mapType.valueType, inShredded)
val repeatedGroupName = if (writeLegacyParquetFormat) {
// Legacy mode:
//
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala
index 24f4fd8274d28..1132f074f29d1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala
@@ -473,13 +473,15 @@ case object SparkShreddingUtils {
* b: struct>>
*
*/
- def variantShreddingSchema(dataType: DataType, isTopLevel: Boolean = true): StructType = {
+ def variantShreddingSchema(dataType: DataType,
+ isTopLevel: Boolean = true,
+ isObjectField : Boolean = false): StructType = {
val fields = dataType match {
case ArrayType(elementType, _) =>
// Always set containsNull to false. One of value or typed_value must always be set for
// array elements.
val arrayShreddingSchema =
- ArrayType(variantShreddingSchema(elementType, false), containsNull = false)
+ ArrayType(variantShreddingSchema(elementType, false, false), containsNull = false)
Seq(
StructField(VariantValueFieldName, BinaryType, nullable = true),
StructField(TypedValueFieldName, arrayShreddingSchema, nullable = true)
@@ -489,15 +491,17 @@ case object SparkShreddingUtils {
// "value" columna as "00", and missing values are represented by setting both "value" and
// "typed_value" to null.
val objectShreddingSchema = StructType(fields.map(f =>
- f.copy(dataType = variantShreddingSchema(f.dataType, false), nullable = false)))
+ f.copy(dataType = variantShreddingSchema(f.dataType, false, true), nullable = false)))
Seq(
StructField(VariantValueFieldName, BinaryType, nullable = true),
StructField(TypedValueFieldName, objectShreddingSchema, nullable = true)
)
case VariantType =>
- // For Variant, we don't need a typed column
+ // For Variant, we don't need a typed column. If there is no typed column, value is required
+ // for array elements or top-level fields, but optional for objects (where a null represents
+ // a missing field).
Seq(
- StructField(VariantValueFieldName, BinaryType, nullable = true)
+ StructField(VariantValueFieldName, BinaryType, nullable = isObjectField)
)
case _: NumericType | BooleanType | _: StringType | BinaryType | _: DatetimeType =>
Seq(
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala
index cf7bad0ecfe60..c8cb5d7ce7c51 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala
@@ -35,7 +35,6 @@ import org.apache.spark.sql.classic.SparkSession
import org.apache.spark.sql.connector.expressions.{FieldReference, RewritableTransform}
import org.apache.spark.sql.errors.QueryCompilationErrors
import org.apache.spark.sql.execution.command.DDLUtils
-import org.apache.spark.sql.execution.command.ViewHelper.generateViewProperties
import org.apache.spark.sql.execution.datasources.{CreateTable => CreateTableV1}
import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2
import org.apache.spark.sql.internal.SQLConf
@@ -705,16 +704,6 @@ object ViewSyncSchemaToMetaStore extends (LogicalPlan => Unit) {
}
if (redo) {
- val newProperties = if (viewSchemaMode == SchemaEvolution) {
- generateViewProperties(
- metaData.properties,
- session,
- fieldNames,
- fieldNames,
- metaData.viewSchemaMode)
- } else {
- metaData.properties
- }
val newSchema = if (viewSchemaMode == SchemaTypeEvolution) {
val newFields = viewQuery.schema.map {
case StructField(name, dataType, nullable, _) =>
@@ -727,9 +716,7 @@ object ViewSyncSchemaToMetaStore extends (LogicalPlan => Unit) {
}
SchemaUtils.checkColumnNameDuplication(fieldNames.toImmutableArraySeq,
session.sessionState.conf.resolver)
- val updatedViewMeta = metaData.copy(
- properties = newProperties,
- schema = newSchema)
+ val updatedViewMeta = metaData.copy(schema = newSchema)
session.sessionState.catalog.alterTable(updatedViewMeta)
}
case _ => // OK
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala
index 6a502a44fad58..55866cc858405 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala
@@ -17,18 +17,19 @@
package org.apache.spark.sql.execution.datasources.v2
-import com.google.common.base.Objects
+import java.util.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, KeyGroupedShuffleSpec, Partitioning, SinglePartition}
+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.catalog.functions.Reducer
import org.apache.spark.sql.connector.read._
+import org.apache.spark.sql.execution.KeyGroupedPartitionedScan
+import org.apache.spark.sql.execution.joins.StoragePartitionJoinParams
import org.apache.spark.util.ArrayImplicits._
/**
@@ -41,7 +42,7 @@ case class BatchScanExec(
ordering: Option[Seq[SortOrder]] = None,
@transient table: Table,
spjParams: StoragePartitionJoinParams = StoragePartitionJoinParams()
- ) extends DataSourceV2ScanExecBase {
+ ) extends DataSourceV2ScanExecBase with KeyGroupedPartitionedScan[InputPartition] {
@transient lazy val batch: Batch = if (scan == null) null else scan.toBatch
@@ -55,7 +56,7 @@ case class BatchScanExec(
false
}
- override def hashCode(): Int = Objects.hashCode(batch, runtimeFilters)
+ override def hashCode(): Int = Objects.hash(batch, runtimeFilters)
@transient override lazy val inputPartitions: Seq[InputPartition] =
batch.planInputPartitions().toImmutableArraySeq
@@ -118,31 +119,7 @@ case class BatchScanExec(
override def outputPartitioning: Partitioning = {
super.outputPartitioning match {
- case k: KeyGroupedPartitioning =>
- val expressions = spjParams.joinKeyPositions match {
- case Some(projectionPositions) => projectionPositions.map(i => k.expressions(i))
- case _ => k.expressions
- }
-
- val newPartValues = spjParams.commonPartitionValues match {
- case Some(commonPartValues) =>
- // We allow duplicated partition values if
- // `spark.sql.sources.v2.bucketing.partiallyClusteredDistribution.enabled` is true
- commonPartValues.flatMap {
- case (partValue, numSplits) => Seq.fill(numSplits)(partValue)
- }
- case None =>
- spjParams.joinKeyPositions match {
- case Some(projectionPositions) => k.partitionValues.map{r =>
- val projectedRow = KeyGroupedPartitioning.project(expressions,
- projectionPositions, r)
- InternalRowComparableWrapper(projectedRow, expressions)
- }.distinct.map(_.row)
- case _ => k.partitionValues
- }
- }
- k.copy(expressions = expressions, numPartitions = newPartValues.length,
- partitionValues = newPartValues)
+ case k: KeyGroupedPartitioning => getOutputKeyGroupedPartitioning(k, spjParams)
case p => p
}
}
@@ -155,104 +132,8 @@ case class BatchScanExec(
sparkContext.parallelize(Array.empty[InternalRow].toImmutableArraySeq, 1)
} else {
val finalPartitions = outputPartitioning match {
- case p: KeyGroupedPartitioning =>
- assert(spjParams.keyGroupedPartitioning.isDefined)
- val expressions = spjParams.keyGroupedPartitioning.get
-
- // Re-group the input partitions if we are projecting on a subset of join keys
- val (groupedPartitions, partExpressions) = spjParams.joinKeyPositions match {
- case Some(projectPositions) =>
- val projectedExpressions = projectPositions.map(i => expressions(i))
- val parts = filteredPartitions.flatten.groupBy(part => {
- val row = part.asInstanceOf[HasPartitionKey].partitionKey()
- val projectedRow = KeyGroupedPartitioning.project(
- expressions, projectPositions, row)
- InternalRowComparableWrapper(projectedRow, projectedExpressions)
- }).map { case (wrapper, splits) => (wrapper.row, splits) }.toSeq
- (parts, projectedExpressions)
- case _ =>
- val groupedParts = filteredPartitions.map(splits => {
- assert(splits.nonEmpty && splits.head.isInstanceOf[HasPartitionKey])
- (splits.head.asInstanceOf[HasPartitionKey].partitionKey(), splits)
- })
- (groupedParts, expressions)
- }
-
- // Also re-group the partitions if we are reducing compatible partition expressions
- val finalGroupedPartitions = spjParams.reducers match {
- case Some(reducers) =>
- val result = groupedPartitions.groupBy { case (row, _) =>
- KeyGroupedShuffleSpec.reducePartitionValue(row, partExpressions, reducers)
- }.map { case (wrapper, splits) => (wrapper.row, splits.flatMap(_._2)) }.toSeq
- val rowOrdering = RowOrdering.createNaturalAscendingOrdering(
- partExpressions.map(_.dataType))
- result.sorted(rowOrdering.on((t: (InternalRow, _)) => t._1))
- case _ => groupedPartitions
- }
-
- // When partially clustered, the input partitions are not grouped by partition
- // values. Here we'll need to check `commonPartitionValues` and decide how to group
- // and replicate splits within a partition.
- if (spjParams.commonPartitionValues.isDefined && spjParams.applyPartialClustering) {
- // A mapping from the common partition values to how many splits the partition
- // should contain.
- val commonPartValuesMap = spjParams.commonPartitionValues
- .get
- .map(t => (InternalRowComparableWrapper(t._1, partExpressions), t._2))
- .toMap
- val filteredGroupedPartitions = finalGroupedPartitions.filter {
- case (partValues, _) =>
- commonPartValuesMap.keySet.contains(
- InternalRowComparableWrapper(partValues, partExpressions))
- }
- val nestGroupedPartitions = filteredGroupedPartitions.map { case (partValue, splits) =>
- // `commonPartValuesMap` should contain the part value since it's the super set.
- val numSplits = commonPartValuesMap
- .get(InternalRowComparableWrapper(partValue, partExpressions))
- assert(numSplits.isDefined, s"Partition value $partValue does not exist in " +
- "common partition values from Spark plan")
-
- val newSplits = if (spjParams.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, partExpressions), newSplits)
- }
-
- // Now fill missing partition keys with empty partitions
- val partitionMapping = nestGroupedPartitions.toMap
- spjParams.commonPartitionValues.get.flatMap {
- case (partValue, numSplits) =>
- // Use empty partition for those partition values that are not present.
- partitionMapping.getOrElse(
- InternalRowComparableWrapper(partValue, partExpressions),
- Seq.fill(numSplits)(Seq.empty))
- }
- } else {
- // either `commonPartitionValues` is not defined, or it is defined but
- // `applyPartialClustering` is false.
- val partitionMapping = finalGroupedPartitions.map { case (partValue, splits) =>
- InternalRowComparableWrapper(partValue, partExpressions) -> splits
- }.toMap
-
- // In case `commonPartitionValues` is not defined (e.g., SPJ is not used), there
- // could exist duplicated partition values, as partition grouping is not done
- // at the beginning and postponed to this method. It is important to use unique
- // partition values here so that grouped partitions won't get duplicated.
- p.uniquePartitionValues.map { partValue =>
- // Use empty partition for those partition values that are not present
- partitionMapping.getOrElse(
- InternalRowComparableWrapper(partValue, partExpressions), Seq.empty)
- }
- }
-
+ case p: KeyGroupedPartitioning => getInputPartitionGrouping(
+ p, spjParams, filteredPartitions, p => p.asInstanceOf[HasPartitionKey].partitionKey())
case _ => filteredPartitions
}
@@ -285,28 +166,3 @@ case class BatchScanExec(
s"BatchScan ${table.name()}".trim
}
}
-
-case class StoragePartitionJoinParams(
- keyGroupedPartitioning: Option[Seq[Expression]] = None,
- joinKeyPositions: Option[Seq[Int]] = None,
- commonPartitionValues: Option[Seq[(InternalRow, Int)]] = None,
- reducers: Option[Seq[Option[Reducer[_, _]]]] = None,
- applyPartialClustering: Boolean = false,
- replicatePartitions: Boolean = false) {
- override def equals(other: Any): Boolean = other match {
- case other: StoragePartitionJoinParams =>
- this.commonPartitionValues == other.commonPartitionValues &&
- this.replicatePartitions == other.replicatePartitions &&
- this.applyPartialClustering == other.applyPartialClustering &&
- this.joinKeyPositions == other.joinKeyPositions
- case _ =>
- false
- }
-
- override def hashCode(): Int = Objects.hashCode(
- joinKeyPositions: Option[Seq[Int]],
- commonPartitionValues: Option[Seq[(InternalRow, Int)]],
- applyPartialClustering: java.lang.Boolean,
- replicatePartitions: java.lang.Boolean)
-}
-
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CacheTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CacheTableExec.scala
index c7f47d2eaaaad..a28b40dc7cbf5 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CacheTableExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CacheTableExec.scala
@@ -19,8 +19,9 @@ package org.apache.spark.sql.execution.datasources.v2
import java.util.Locale
+import scala.util.control.NonFatal
+
import org.apache.spark.internal.LogKeys.OPTIONS
-import org.apache.spark.internal.MDC
import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
import org.apache.spark.sql.catalyst.analysis.{LocalTempView, UnresolvedRelation}
import org.apache.spark.sql.catalyst.expressions.Attribute
@@ -28,8 +29,10 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
import org.apache.spark.sql.classic.Dataset
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper
-import org.apache.spark.sql.execution.command.CreateViewCommand
+import org.apache.spark.sql.connector.catalog.Identifier
+import org.apache.spark.sql.execution.command.{CreateViewCommand, DropTempViewCommand}
import org.apache.spark.storage.StorageLevel
+import org.apache.spark.util.Utils
trait BaseCacheTableExec extends LeafV2CommandExec {
def relationName: String
@@ -53,7 +56,16 @@ trait BaseCacheTableExec extends LeafV2CommandExec {
if (!isLazy) {
// Performs eager caching.
- df.count()
+ try {
+ df.count()
+ } catch {
+ case NonFatal(e) =>
+ // If the query fails, we should remove the cached table.
+ Utils.tryLogNonFatalError {
+ session.sharedState.cacheManager.uncacheQuery(session, planToCache, cascade = false)
+ }
+ throw e
+ }
}
Seq.empty
@@ -99,7 +111,15 @@ case class CacheTableAsSelectExec(
isAnalyzed = true,
referredTempFunctions = referredTempFunctions
).run(session)
- super.run()
+ try {
+ super.run()
+ } catch {
+ case NonFatal(e) =>
+ Utils.tryLogNonFatalError {
+ DropTempViewCommand(Identifier.of(Array.empty, tempViewName)).run(session)
+ }
+ throw e
+ }
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateIndexExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateIndexExec.scala
index 60d44101da3b1..1de97ba4189ea 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateIndexExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateIndexExec.scala
@@ -22,7 +22,6 @@ import java.util
import scala.jdk.CollectionConverters._
import org.apache.spark.internal.LogKeys.{INDEX_NAME, TABLE_NAME}
-import org.apache.spark.internal.MDC
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.IndexAlreadyExistsException
import org.apache.spark.sql.catalyst.expressions.Attribute
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateNamespaceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateNamespaceExec.scala
index 76ba53ef99a00..ef069fd2fe23b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateNamespaceExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateNamespaceExec.scala
@@ -20,7 +20,6 @@ package org.apache.spark.sql.execution.datasources.v2
import scala.jdk.CollectionConverters.MapHasAsJava
import org.apache.spark.internal.LogKeys.NAMESPACE
-import org.apache.spark.internal.MDC
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.NamespaceAlreadyExistsException
import org.apache.spark.sql.catalyst.expressions.Attribute
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala
index 3298a56d95992..58c63be38833f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala
@@ -20,7 +20,6 @@ package org.apache.spark.sql.execution.datasources.v2
import scala.jdk.CollectionConverters._
import org.apache.spark.internal.LogKeys.TABLE_NAME
-import org.apache.spark.internal.MDC
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException
import org.apache.spark.sql.catalyst.expressions.Attribute
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala
index e46c0806ba2d9..6a07d3c3931a1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala
@@ -61,6 +61,14 @@ class DataSourceRDD(
private var currentIter: Option[Iterator[Object]] = None
private var currentIndex: Int = 0
+ private val partitionMetricCallback = new PartitionMetricCallback(customMetrics)
+
+ // In case of early stopping before consuming the entire iterator,
+ // we need to do one more metric update at the end of the task.
+ context.addTaskCompletionListener[Unit] { _ =>
+ partitionMetricCallback.execute()
+ }
+
override def hasNext: Boolean = currentIter.exists(_.hasNext) || advanceToNextIter()
override def next(): Object = {
@@ -87,14 +95,10 @@ class DataSourceRDD(
new PartitionIterator[InternalRow](rowReader, customMetrics))
(iter, rowReader)
}
- context.addTaskCompletionListener[Unit] { _ =>
- // In case of early stopping before consuming the entire iterator,
- // we need to do one more metric update at the end of the task.
- CustomMetrics
- .updateMetrics(reader.currentMetricsValues.toImmutableArraySeq, customMetrics)
- iter.forceUpdateMetrics()
- reader.close()
- }
+
+ // Once we advance to the next partition, update the metric callback for early finish
+ partitionMetricCallback.advancePartition(iter, reader)
+
currentIter = Some(iter)
hasNext
}
@@ -109,6 +113,28 @@ class DataSourceRDD(
}
}
+private class PartitionMetricCallback
+ (customMetrics: Map[String, SQLMetric]) {
+ private var iter: MetricsIterator[_] = null
+ private var reader: PartitionReader[_] = null
+
+ def advancePartition(iter: MetricsIterator[_], reader: PartitionReader[_]): Unit = {
+ execute()
+
+ this.iter = iter
+ this.reader = reader
+ }
+
+ def execute(): Unit = {
+ if (iter != null && reader != null) {
+ CustomMetrics
+ .updateMetrics(reader.currentMetricsValues.toImmutableArraySeq, customMetrics)
+ iter.forceUpdateMetrics()
+ reader.close()
+ }
+ }
+}
+
private class PartitionIterator[T](
reader: PartitionReader[T],
customMetrics: Map[String, SQLMetric]) extends Iterator[T] {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
index 9cbea3b69ab79..b07e0442d4f01 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
@@ -19,11 +19,10 @@ package org.apache.spark.sql.execution.datasources.v2
import scala.collection.mutable
-import org.apache.commons.lang3.StringUtils
import org.apache.hadoop.fs.Path
import org.apache.spark.SparkException
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.EXPR
import org.apache.spark.sql.catalyst.analysis.{ResolvedIdentifier, ResolvedNamespace, ResolvedPartitionSpec, ResolvedTable}
import org.apache.spark.sql.catalyst.catalog.CatalogUtils
@@ -46,12 +45,14 @@ import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors
import org.apache.spark.sql.execution.{FilterExec, InSubqueryExec, LeafExecNode, LocalTableScanExec, ProjectExec, RowDataSourceScanExec, SparkPlan, SparkStrategy => Strategy}
import org.apache.spark.sql.execution.command.CommandUtils
import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, LogicalRelationWithTable, PushableColumnAndNestedColumn}
+import org.apache.spark.sql.execution.joins.StoragePartitionJoinParams
import org.apache.spark.sql.execution.streaming.continuous.{WriteToContinuousDataSource, WriteToContinuousDataSourceExec}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.StaticSQLConf.WAREHOUSE_PATH
import org.apache.spark.sql.sources.{BaseRelation, TableScan}
import org.apache.spark.storage.StorageLevel
import org.apache.spark.util.ArrayImplicits._
+import org.apache.spark.util.SparkStringUtils
class DataSourceV2Strategy(session: SparkSession) extends Strategy with PredicateHelper {
@@ -391,7 +392,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
AlterNamespaceSetPropertiesExec(catalog.asNamespaceCatalog, ns, properties) :: Nil
case SetNamespaceLocation(ResolvedNamespace(catalog, ns, _), location) =>
- if (StringUtils.isEmpty(location)) {
+ if (SparkStringUtils.isEmpty(location)) {
throw QueryExecutionErrors.invalidEmptyLocationError(location)
}
AlterNamespaceSetPropertiesExec(
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala
index 7cfd601ef774f..3cc767380d8bb 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala
@@ -45,6 +45,7 @@ case class DescribeTableExec(
addMetadataColumns(rows)
addTableDetails(rows)
addTableStats(rows)
+ addTableConstraints(rows)
}
rows.toSeq
}
@@ -76,15 +77,25 @@ case class DescribeTableExec(
rows += toCatalystRow("Table Properties", properties, "")
// If any columns have default values, append them to the result.
- ResolveDefaultColumns.getDescribeMetadata(table.schema).foreach { row =>
+ ResolveDefaultColumns.getDescribeMetadata(table.columns()).foreach { row =>
rows += toCatalystRow(row._1, row._2, row._3)
}
}
private def addSchema(rows: ArrayBuffer[InternalRow]): Unit = {
- rows ++= table.schema.map{ column =>
+ rows ++= table.columns().map{ column =>
toCatalystRow(
- column.name, column.dataType.simpleString, column.getComment().orNull)
+ column.name, column.dataType.simpleString, column.comment)
+ }
+ }
+
+ private def addTableConstraints(rows: ArrayBuffer[InternalRow]): Unit = {
+ if (table.constraints.nonEmpty) {
+ rows += emptyRow()
+ rows += toCatalystRow("# Constraints", "", "")
+ rows ++= table.constraints().map{ constraint =>
+ toCatalystRow(constraint.name(), constraint.toDescription, "")
+ }
}
}
@@ -107,11 +118,12 @@ case class DescribeTableExec(
rows += toCatalystRow("# Clustering Information", "", "")
rows += toCatalystRow(s"# ${output.head.name}", output(1).name, output(2).name)
rows ++= clusterBySpec.columnNames.map { fieldNames =>
- val nestedField = table.schema.findNestedField(fieldNames.fieldNames.toIndexedSeq)
+ val schema = CatalogV2Util.v2ColumnsToStructType(table.columns())
+ val nestedField = schema.findNestedField(fieldNames.fieldNames.toIndexedSeq)
assert(nestedField.isDefined,
"The clustering column " +
s"${fieldNames.fieldNames.map(quoteIfNeeded).mkString(".")} " +
- s"was not found in the table schema ${table.schema.catalogString}.")
+ s"was not found in the table schema ${schema.catalogString}.")
nestedField.get
}.map { case (path, field) =>
toCatalystRow(
@@ -153,15 +165,15 @@ case class DescribeTableExec(
if (partitionColumnsOnly) {
rows += toCatalystRow("# Partition Information", "", "")
rows += toCatalystRow(s"# ${output(0).name}", output(1).name, output(2).name)
+ val schema = CatalogV2Util.v2ColumnsToStructType(table.columns())
rows ++= table.partitioning
.map(_.asInstanceOf[IdentityTransform].ref.fieldNames())
.map { fieldNames =>
- val nestedField = table.schema.findNestedField(fieldNames.toImmutableArraySeq)
+ val nestedField = schema.findNestedField(fieldNames.toImmutableArraySeq)
if (nestedField.isEmpty) {
throw QueryExecutionErrors.partitionColumnNotFoundInTheTableSchemaError(
fieldNames.toSeq,
- table.schema()
- )
+ schema)
}
nestedField.get
}.map { case (path, field) =>
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIndexExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIndexExec.scala
index 4fe6c3cd4a0e8..b10f59686062e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIndexExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIndexExec.scala
@@ -18,7 +18,6 @@
package org.apache.spark.sql.execution.datasources.v2
import org.apache.spark.internal.LogKeys.INDEX_NAME
-import org.apache.spark.internal.MDC
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.NoSuchIndexException
import org.apache.spark.sql.catalyst.expressions.Attribute
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileBatchWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileBatchWrite.scala
index b9f058b55ed02..345a0b0a4a290 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileBatchWrite.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileBatchWrite.scala
@@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.datasources.v2
import org.apache.hadoop.mapreduce.Job
-import org.apache.spark.internal.{Logging, LogKeys, MDC}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.internal.io.FileCommitProtocol
import org.apache.spark.sql.connector.write.{BatchWrite, DataWriterFactory, PhysicalWriteInfo, WriterCommitMessage}
import org.apache.spark.sql.execution.datasources.{WriteJobDescription, WriteTaskResult}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala
index be4f5dcb65aa1..37060f9bd0816 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala
@@ -16,17 +16,17 @@
*/
package org.apache.spark.sql.execution.datasources.v2
-import java.io.{FileNotFoundException, IOException}
+import java.io.FileNotFoundException
import org.apache.hadoop.hdfs.BlockMissingException
import org.apache.hadoop.security.AccessControlException
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{CURRENT_FILE, PARTITIONED_FILE_READER}
import org.apache.spark.rdd.InputFileBlockHolder
import org.apache.spark.sql.catalyst.FileSourceOptions
import org.apache.spark.sql.connector.read.PartitionReader
-import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.execution.datasources.{DataSourceUtils, PartitionedFile}
class FilePartitionReader[T](
files: Iterator[PartitionedFile],
@@ -53,7 +53,7 @@ class FilePartitionReader[T](
currentReader = null
case e @ (_ : AccessControlException | _ : BlockMissingException) =>
throw FileDataSourceV2.attachFilePath(file.urlEncodedPath, e)
- case e @ (_: RuntimeException | _: IOException) if ignoreCorruptFiles =>
+ case e if ignoreCorruptFiles && DataSourceUtils.shouldIgnoreCorruptFileException(e) =>
logWarning(
s"Skipped the rest of the content in the corrupted file.", e)
currentReader = null
@@ -71,7 +71,7 @@ class FilePartitionReader[T](
} catch {
case e @ (_ : AccessControlException | _ : BlockMissingException) =>
throw FileDataSourceV2.attachFilePath(currentReader.file.urlEncodedPath, e)
- case e @ (_: RuntimeException | _: IOException) if ignoreCorruptFiles =>
+ case e if ignoreCorruptFiles && DataSourceUtils.shouldIgnoreCorruptFileException(e) =>
logWarning(log"Skipped the rest of the content in the corrupted file: " +
log"${MDC(PARTITIONED_FILE_READER, currentReader)}", e)
false
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala
index 9ace0540ec243..5348f9ab6df62 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala
@@ -18,10 +18,9 @@ package org.apache.spark.sql.execution.datasources.v2
import java.util.{Locale, OptionalLong}
-import org.apache.commons.lang3.StringUtils
import org.apache.hadoop.fs.Path
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{PATH, REASON}
import org.apache.spark.internal.config.IO_WARNING_LARGEFILETHRESHOLD
import org.apache.spark.sql.SparkSession
@@ -30,11 +29,11 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeSet, Expression, Expr
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
import org.apache.spark.sql.catalyst.plans.QueryPlan
import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes
-import org.apache.spark.sql.connector.read.{Batch, InputPartition, Scan, Statistics, SupportsReportStatistics}
+import org.apache.spark.sql.connector.read._
import org.apache.spark.sql.errors.QueryCompilationErrors
import org.apache.spark.sql.execution.PartitionedFileUtil
import org.apache.spark.sql.execution.datasources._
-import org.apache.spark.sql.internal.{SessionState, SQLConf}
+import org.apache.spark.sql.internal.{SessionStateHelper, SQLConf}
import org.apache.spark.sql.internal.connector.SupportsMetadata
import org.apache.spark.sql.sources.Filter
import org.apache.spark.sql.types.StructType
@@ -113,10 +112,7 @@ trait FileScan extends Scan
override def hashCode(): Int = getClass.hashCode()
- override def conf: SQLConf = {
- val sessionState: SessionState = sparkSession.sessionState
- sessionState.conf
- }
+ override def conf: SQLConf = SessionStateHelper.getSqlConf(sparkSession)
val maxMetadataValueLength = conf.maxMetadataStringLength
@@ -125,7 +121,7 @@ trait FileScan extends Scan
case (key, value) =>
val redactedValue =
Utils.redact(conf.stringRedactionPattern, value)
- key + ": " + StringUtils.abbreviate(redactedValue, maxMetadataValueLength)
+ key + ": " + Utils.abbreviate(redactedValue, maxMetadataValueLength)
}.mkString(", ")
s"${this.getClass.getSimpleName} $metadataStr"
}
@@ -177,7 +173,7 @@ trait FileScan extends Scan
if (splitFiles.length == 1) {
val path = splitFiles(0).toPath
if (!isSplitable(path) && splitFiles(0).length >
- sparkSession.sparkContext.conf.get(IO_WARNING_LARGEFILETHRESHOLD)) {
+ SessionStateHelper.getSparkConf(sparkSession).get(IO_WARNING_LARGEFILETHRESHOLD)) {
logWarning(log"Loading one large unsplittable file ${MDC(PATH, path.toString)} with only " +
log"one partition, the reason is: ${MDC(REASON, getFileUnSplittableReason(path))}")
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala
index 863104da80c2e..0af728c1958d4 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala
@@ -29,7 +29,8 @@ import org.apache.spark.sql.connector.expressions.Transform
import org.apache.spark.sql.connector.write.{LogicalWriteInfo, LogicalWriteInfoImpl}
import org.apache.spark.sql.errors.QueryCompilationErrors
import org.apache.spark.sql.execution.datasources._
-import org.apache.spark.sql.execution.streaming.{FileStreamSink, MetadataLogFileIndex}
+import org.apache.spark.sql.execution.streaming.runtime.MetadataLogFileIndex
+import org.apache.spark.sql.execution.streaming.sinks.FileStreamSink
import org.apache.spark.sql.types.{DataType, StructType}
import org.apache.spark.sql.util.CaseInsensitiveStringMap
import org.apache.spark.sql.util.SchemaUtils
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/GroupBasedRowLevelOperationScanPlanning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/GroupBasedRowLevelOperationScanPlanning.scala
index 8b8cdc06d398b..77e21bcbbdb8f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/GroupBasedRowLevelOperationScanPlanning.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/GroupBasedRowLevelOperationScanPlanning.scala
@@ -17,7 +17,7 @@
package org.apache.spark.sql.execution.datasources.v2
-import org.apache.spark.internal.{LogKeys, MDC}
+import org.apache.spark.internal.{LogKeys}
import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, AttributeSet, Expression, ExpressionSet, PredicateHelper, SubqueryExpression}
import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
import org.apache.spark.sql.catalyst.planning.{GroupBasedRowLevelOperation, PhysicalOperation}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeRowsExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeRowsExec.scala
index 8a25170fcee80..311bab084bbf7 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeRowsExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeRowsExec.scala
@@ -29,11 +29,12 @@ import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.catalyst.expressions.Projection
import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
-import org.apache.spark.sql.catalyst.plans.logical.MergeRows.{Discard, Instruction, Keep, ROW_ID, Split}
+import org.apache.spark.sql.catalyst.plans.logical.MergeRows.{Context, Copy, Delete, Discard, Insert, Instruction, Keep, ROW_ID, Split, Update}
import org.apache.spark.sql.catalyst.util.truncatedString
import org.apache.spark.sql.errors.QueryExecutionErrors
import org.apache.spark.sql.execution.SparkPlan
import org.apache.spark.sql.execution.UnaryExecNode
+import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
case class MergeRowsExec(
isSourceRowPresent: Expression,
@@ -45,6 +46,24 @@ case class MergeRowsExec(
output: Seq[Attribute],
child: SparkPlan) extends UnaryExecNode {
+ override lazy val metrics: Map[String, SQLMetric] = Map(
+ "numTargetRowsCopied" -> SQLMetrics.createMetric(sparkContext,
+ "number of target rows copied unmodified because they did not match any action"),
+ "numTargetRowsInserted" -> SQLMetrics.createMetric(sparkContext,
+ "number of target rows inserted"),
+ "numTargetRowsDeleted" -> SQLMetrics.createMetric(sparkContext,
+ "number of target rows deleted"),
+ "numTargetRowsUpdated" -> SQLMetrics.createMetric(sparkContext,
+ "number of target rows updated"),
+ "numTargetRowsMatchedUpdated" -> SQLMetrics.createMetric(sparkContext,
+ "number of target rows updated by a matched clause"),
+ "numTargetRowsMatchedDeleted" -> SQLMetrics.createMetric(sparkContext,
+ "number of target rows deleted by a matched clause"),
+ "numTargetRowsNotMatchedBySourceUpdated" -> SQLMetrics.createMetric(sparkContext,
+ "number of target rows updated by a not matched by source clause"),
+ "numTargetRowsNotMatchedBySourceDeleted" -> SQLMetrics.createMetric(sparkContext,
+ "number of target rows deleted by a not matched by source clause"))
+
@transient override lazy val producedAttributes: AttributeSet = {
AttributeSet(output.filterNot(attr => inputSet.contains(attr)))
}
@@ -107,8 +126,8 @@ case class MergeRowsExec(
private def planInstructions(instructions: Seq[Instruction]): Seq[InstructionExec] = {
instructions.map {
- case Keep(cond, output) =>
- KeepExec(createPredicate(cond), createProjection(output))
+ case Keep(context, cond, output) =>
+ KeepExec(context, createPredicate(cond), createProjection(output))
case Discard(cond) =>
DiscardExec(createPredicate(cond))
@@ -127,7 +146,10 @@ case class MergeRowsExec(
def condition: BasePredicate
}
- case class KeepExec(condition: BasePredicate, projection: Projection) extends InstructionExec {
+ case class KeepExec(
+ context: Context,
+ condition: BasePredicate,
+ projection: Projection) extends InstructionExec {
def apply(row: InternalRow): InternalRow = projection.apply(row)
}
@@ -203,9 +225,9 @@ case class MergeRowsExec(
if (isTargetRowPresent && isSourceRowPresent) {
cardinalityValidator.validate(row)
- applyInstructions(row, matchedInstructions)
+ applyInstructions(row, matchedInstructions, sourcePresent = true)
} else if (isSourceRowPresent) {
- applyInstructions(row, notMatchedInstructions)
+ applyInstructions(row, notMatchedInstructions, sourcePresent = true)
} else if (isTargetRowPresent) {
applyInstructions(row, notMatchedBySourceInstructions)
} else {
@@ -215,18 +237,29 @@ case class MergeRowsExec(
private def applyInstructions(
row: InternalRow,
- instructions: Seq[InstructionExec]): InternalRow = {
+ instructions: Seq[InstructionExec],
+ sourcePresent: Boolean = false): InternalRow = {
for (instruction <- instructions) {
if (instruction.condition.eval(row)) {
instruction match {
case keep: KeepExec =>
+ keep.context match {
+ case Copy => incrementCopyMetric()
+ case Update => incrementUpdateMetric(sourcePresent)
+ case Insert => incrementInsertMetric()
+ case Delete => incrementDeleteMetric(sourcePresent)
+ case _ => throw new IllegalArgumentException(
+ s"Unexpected context for KeepExec: ${keep.context}")
+ }
return keep.apply(row)
case _: DiscardExec =>
+ incrementDeleteMetric(sourcePresent)
return null
case split: SplitExec =>
+ incrementUpdateMetric(sourcePresent)
cachedExtraRow = split.projectExtraRow(row)
return split.projectRow(row)
}
@@ -236,4 +269,27 @@ case class MergeRowsExec(
null
}
}
+
+ // For group based merge, copy is inserted if row matches no other case
+ private def incrementCopyMetric(): Unit = longMetric("numTargetRowsCopied") += 1
+
+ private def incrementInsertMetric(): Unit = longMetric("numTargetRowsInserted") += 1
+
+ private def incrementDeleteMetric(sourcePresent: Boolean): Unit = {
+ longMetric("numTargetRowsDeleted") += 1
+ if (sourcePresent) {
+ longMetric("numTargetRowsMatchedDeleted") += 1
+ } else {
+ longMetric("numTargetRowsNotMatchedBySourceDeleted") += 1
+ }
+ }
+
+ private def incrementUpdateMetric(sourcePresent: Boolean): Unit = {
+ longMetric("numTargetRowsUpdated") += 1
+ if (sourcePresent) {
+ longMetric("numTargetRowsMatchedUpdated") += 1
+ } else {
+ longMetric("numTargetRowsNotMatchedBySourceUpdated") += 1
+ }
+ }
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushedDownOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushedDownOperators.scala
index 49044c6e24db6..c7d2c56b89858 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushedDownOperators.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushedDownOperators.scala
@@ -30,6 +30,9 @@ case class PushedDownOperators(
limit: Option[Int],
offset: Option[Int],
sortValues: Seq[SortOrder],
- pushedPredicates: Seq[Predicate]) {
+ pushedPredicates: Seq[Predicate],
+ joinedRelationPushedDownOperators: Seq[PushedDownOperators],
+ // Relation name in case of leaf relation. For join nodes, this is empty.
+ relationName: Option[String]) {
assert((limit.isEmpty && sortValues.isEmpty) || limit.isDefined)
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateTableExec.scala
index 4195560c5cc1c..56e786d3e9334 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateTableExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateTableExec.scala
@@ -65,7 +65,8 @@ case class ShowCreateTableExec(
private def showTableDataColumns(table: Table, builder: StringBuilder): Unit = {
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
val columns = CharVarcharUtils.getRawSchema(table.columns.asSchema, conf).fields.map(_.toDDL)
- builder ++= concatByMultiLines(columns)
+ val constraints = table.constraints().map(_.toDDL)
+ builder ++= concatByMultiLines(columns ++ constraints)
}
private def showTableUsing(table: Table, builder: StringBuilder): Unit = {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala
index 0b2d11a597d5c..ed62890c87d89 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala
@@ -114,8 +114,8 @@ case class ShowTablesExtendedExec(
field => quoteIdentifier(field.name)).mkString("[", ", ", "]"))
}
- if (table.schema().nonEmpty) {
- results.put("Schema", table.schema().treeString)
+ if (table.columns().nonEmpty) {
+ results.put("Schema", CatalogV2Util.v2ColumnsToStructType(table.columns()).treeString)
}
results.map { case (key, value) =>
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala
index e332c6b8014a6..072dcc995f41c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala
@@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.streaming.StreamingRelationV2
import org.apache.spark.sql.connector.catalog.Table
import org.apache.spark.sql.connector.catalog.TableCapability._
import org.apache.spark.sql.errors.QueryCompilationErrors
-import org.apache.spark.sql.execution.streaming.StreamingRelation
+import org.apache.spark.sql.execution.streaming.runtime.StreamingRelation
import org.apache.spark.sql.types.BooleanType
/**
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanPartitioningAndOrdering.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanPartitioningAndOrdering.scala
index 7ee4374b45789..5d06c8786d894 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanPartitioningAndOrdering.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanPartitioningAndOrdering.scala
@@ -16,7 +16,7 @@
*/
package org.apache.spark.sql.execution.datasources.v2
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.CLASS_NAME
import org.apache.spark.sql.catalyst.expressions.V2ExpressionUtils
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala
index 5f7e86cab5240..31a98e1ff96cb 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala
@@ -17,21 +17,22 @@
package org.apache.spark.sql.execution.datasources.v2
+import java.util.Locale
+
import scala.collection.mutable
-import org.apache.spark.internal.LogKeys.{AGGREGATE_FUNCTIONS, GROUP_BY_EXPRS, POST_SCAN_FILTERS, PUSHED_FILTERS, RELATION_NAME, RELATION_OUTPUT}
-import org.apache.spark.internal.MDC
+import org.apache.spark.internal.LogKeys.{AGGREGATE_FUNCTIONS, COLUMN_NAMES, GROUP_BY_EXPRS, JOIN_CONDITION, JOIN_TYPE, POST_SCAN_FILTERS, PUSHED_FILTERS, RELATION_NAME, RELATION_OUTPUT}
import org.apache.spark.sql.catalyst.expressions.{aggregate, Alias, And, Attribute, AttributeMap, AttributeReference, AttributeSet, Cast, Expression, IntegerLiteral, Literal, NamedExpression, PredicateHelper, ProjectionOverSchema, SortOrder, SubqueryExpression}
import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
import org.apache.spark.sql.catalyst.optimizer.CollapseProject
import org.apache.spark.sql.catalyst.planning.{PhysicalOperation, ScanOperation}
-import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, LeafNode, Limit, LimitAndOffset, LocalLimit, LogicalPlan, Offset, OffsetAndLimit, Project, Sample, Sort}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LeafNode, Limit, LimitAndOffset, LocalLimit, LogicalPlan, Offset, OffsetAndLimit, Project, Sample, Sort}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes
import org.apache.spark.sql.connector.expressions.{SortOrder => V2SortOrder}
import org.apache.spark.sql.connector.expressions.aggregate.{Aggregation, Avg, Count, CountStar, Max, Min, Sum}
import org.apache.spark.sql.connector.expressions.filter.Predicate
-import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownAggregates, SupportsPushDownFilters, V1Scan}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownAggregates, SupportsPushDownFilters, SupportsPushDownJoin, V1Scan}
import org.apache.spark.sql.execution.datasources.DataSourceStrategy
import org.apache.spark.sql.sources
import org.apache.spark.sql.types.{DataType, DecimalType, IntegerType, StructType}
@@ -46,9 +47,11 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper {
createScanBuilder,
pushDownSample,
pushDownFilters,
+ pushDownJoin,
pushDownAggregates,
pushDownLimitAndOffset,
buildScanWithPushedAggregate,
+ buildScanWithPushedJoin,
pruneColumns)
pushdownRules.foldLeft(plan) { (newPlan, pushDownRule) =>
@@ -98,6 +101,218 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper {
filterCondition.map(Filter(_, sHolder)).getOrElse(sHolder)
}
+ def pushDownJoin(plan: LogicalPlan): LogicalPlan = plan.transformUp {
+ // Join can be attempted to be pushed down only if left and right side of join are
+ // compatible (same data source, for example). Also, another requirement is that if
+ // there are projections between Join and ScanBuilderHolder, these projections need to be
+ // AttributeReferences. We could probably support Alias as well, but this should be on
+ // TODO list.
+ // Alias can exist between Join and sHolder node because the query below is not valid:
+ // SELECT * FROM
+ // (SELECT * FROM tbl t1 JOIN tbl2 t2) p
+ // JOIN
+ // (SELECT * FROM tbl t3 JOIN tbl3 t4) q
+ // ON p.t1.col = q.t3.col (this is not possible)
+ // It's because there are duplicated columns in both sides of top level join and it's not
+ // possible to fully qualified the column names in condition. Therefore, query should be
+ // rewritten so that each of the outputs of child joins are aliased, so there would be a
+ // projection with aliases between top level join and scanBuilderHolder (that has pushed
+ // child joins).
+ case node @ Join(
+ PhysicalOperation(
+ leftProjections,
+ Nil,
+ leftHolder @ ScanBuilderHolder(_, _, lBuilder: SupportsPushDownJoin)
+ ),
+ PhysicalOperation(
+ rightProjections,
+ Nil,
+ rightHolder @ ScanBuilderHolder(_, _, rBuilder: SupportsPushDownJoin)
+ ),
+ joinType,
+ condition,
+ _) if conf.dataSourceV2JoinPushdown &&
+ // We do not support pushing down anything besides AttributeReference.
+ leftProjections.forall(_.isInstanceOf[AttributeReference]) &&
+ rightProjections.forall(_.isInstanceOf[AttributeReference]) &&
+ // Cross joins are not supported because they increase the amount of data.
+ condition.isDefined &&
+ lBuilder.isOtherSideCompatibleForJoin(rBuilder) =>
+ // Process left and right columns in original order
+ val (leftSideRequiredColumnsWithAliases, rightSideRequiredColumnsWithAliases) =
+ generateColumnAliasesForDuplicatedName(
+ getRequiredColumnNames(leftProjections, leftHolder),
+ getRequiredColumnNames(rightProjections, rightHolder))
+
+ // Create the AttributeMap that holds (Attribute -> Attribute with up to date name) mapping.
+ val pushedJoinOutputMap = AttributeMap[Expression](
+ node.output
+ .zip(leftSideRequiredColumnsWithAliases ++ rightSideRequiredColumnsWithAliases)
+ .collect {
+ case (attr, columnWithAlias) =>
+ if (columnWithAlias.alias() != null) {
+ (attr, attr.withName(columnWithAlias.alias()))
+ } else {
+ (attr, attr.withName(columnWithAlias.colName()))
+ }
+ }
+ .toMap
+ )
+
+ // Reuse the previously calculated map to update the condition with attributes
+ // with up-to-date names
+ val normalizedCondition = condition.map { e =>
+ DataSourceStrategy.normalizeExprs(
+ Seq(e),
+ (leftHolder.output ++ rightHolder.output).map { a =>
+ pushedJoinOutputMap.getOrElse(a, a).asInstanceOf[AttributeReference]
+ }
+ ).head
+ }
+
+ val translatedCondition =
+ normalizedCondition.flatMap(DataSourceV2Strategy.translateFilterV2(_))
+ val translatedJoinType = DataSourceStrategy.translateJoinType(joinType)
+
+ logInfo(log"DSv2 Join pushdown - translated join condition " +
+ log"${MDC(JOIN_CONDITION, translatedCondition)}")
+ logInfo(log"DSv2 Join pushdown - translated join type " +
+ log"${MDC(JOIN_TYPE, translatedJoinType)}")
+
+ logInfo(log"DSv2 Join pushdown - left side required columns with aliases: " +
+ log"${MDC(
+ COLUMN_NAMES,
+ leftSideRequiredColumnsWithAliases.map(_.prettyString()).mkString(", ")
+ )}")
+ logInfo(log"DSv2 Join pushdown - right side required columns with aliases: " +
+ log"${MDC(
+ COLUMN_NAMES,
+ rightSideRequiredColumnsWithAliases.map(_.prettyString()).mkString(", ")
+ )}")
+
+ if (translatedJoinType.isDefined &&
+ translatedCondition.isDefined &&
+ lBuilder.pushDownJoin(
+ rBuilder,
+ translatedJoinType.get,
+ leftSideRequiredColumnsWithAliases,
+ rightSideRequiredColumnsWithAliases,
+ translatedCondition.get)
+ ) {
+ val leftSidePushedDownOperators = getPushedDownOperators(leftHolder)
+ val rightSidePushedDownOperators = getPushedDownOperators(rightHolder)
+
+ leftHolder.joinedRelations = leftHolder.joinedRelations ++ rightHolder.joinedRelations
+ leftHolder.joinedRelationsPushedDownOperators =
+ Seq(leftSidePushedDownOperators, rightSidePushedDownOperators)
+
+ leftHolder.pushedPredicates = Seq(translatedCondition.get)
+ leftHolder.pushedSample = None
+
+ leftHolder.output = node.output.asInstanceOf[Seq[AttributeReference]]
+ leftHolder.pushedJoinOutputMap = pushedJoinOutputMap
+
+ // TODO: for cascade joins, already joined relations will still have the name of the
+ // original(leaf) relation. It should be thought of if we want to change the name of the
+ // relation when join is pushed down.
+ logInfo(log"DSv2 Join pushdown - successfully pushed down join between relations " +
+ log"${MDC(RELATION_NAME, leftHolder.relation.name)} and " +
+ log"${MDC(RELATION_NAME, rightHolder.relation.name)}.")
+
+ leftHolder
+ } else {
+ logInfo(log"DSv2 Join pushdown - failed to push down join.")
+ node
+ }
+ }
+ /**
+ * Generates unique column aliases for join operations to avoid naming conflicts.
+ * Handles case sensitivity issues across different databases (SQL Server, MySQL, etc.).
+ *
+ * @param leftSideRequiredColumnNames Columns from the left side of the join
+ * @param rightSideRequiredColumnNames Columns from the right side of the join
+ * @return Tuple of (leftColumnsWithAliases, rightColumnsWithAliases)
+ */
+ private[v2] def generateColumnAliasesForDuplicatedName(
+ leftSideRequiredColumnNames: Array[String],
+ rightSideRequiredColumnNames: Array[String]
+ ): (Array[SupportsPushDownJoin.ColumnWithAlias],
+ Array[SupportsPushDownJoin.ColumnWithAlias]) = {
+ // Normalize all column names to lowercase for case-insensitive comparison
+ val normalizeCase: String => String = _.toLowerCase(Locale.ROOT)
+
+ // Count occurrences of each column name (case-insensitive)
+ val allRequiredColumnNames = leftSideRequiredColumnNames ++ rightSideRequiredColumnNames
+ val allNameCounts: Map[String, Int] =
+ allRequiredColumnNames.map(normalizeCase)
+ .groupBy(identity)
+ .view
+ .mapValues(_.length)
+ .toMap
+
+ // Track claimed aliases using normalized names.
+ // Use Set for O(1) lookups when checking existing column names, claim all names
+ // that appears only once to ensure they have highest priority.
+ val allClaimedAliases = mutable.Set.from(
+ allNameCounts.filter(_._2 == 1).keys
+ )
+
+ // Track suffix index for each base column name (starts at 0) to avoid extreme worst
+ // case of O(n^2) alias generation.
+ val aliasSuffixIndex = mutable.HashMap[String, Int]().withDefaultValue(0)
+
+ def processColumn(originalName: String): SupportsPushDownJoin.ColumnWithAlias = {
+ val normalizedName = normalizeCase(originalName)
+
+ // No alias needed for unique column names
+ if (allNameCounts(normalizedName) == 1) {
+ new SupportsPushDownJoin.ColumnWithAlias(originalName, null)
+ } else {
+ var attempt = aliasSuffixIndex(normalizedName)
+ var candidate = if (attempt == 0) originalName else s"${originalName}_$attempt"
+ var normalizedCandidate = normalizeCase(candidate)
+
+ // Find first available unique alias, use original name for the first attempt, then append
+ // suffix for more attempts.
+ while (allClaimedAliases.contains(normalizedCandidate)) {
+ attempt += 1
+ candidate = s"${originalName}_$attempt"
+ normalizedCandidate = normalizeCase(candidate)
+ }
+
+ // Update tracking state
+ aliasSuffixIndex(normalizedName) = attempt + 1
+ allClaimedAliases.add(normalizedCandidate)
+
+ if (originalName == candidate) {
+ new SupportsPushDownJoin.ColumnWithAlias(originalName, null)
+ } else {
+ new SupportsPushDownJoin.ColumnWithAlias(originalName, candidate)
+ }
+ }
+ }
+
+ (
+ leftSideRequiredColumnNames.map(processColumn),
+ rightSideRequiredColumnNames.map(processColumn)
+ )
+ }
+
+ // Projections' names are maybe not up to date if the joins have been previously pushed down.
+ // For this reason, we need to use pushedJoinOutputMap to get up to date names.
+ def getRequiredColumnNames(
+ projections: Seq[NamedExpression],
+ sHolder: ScanBuilderHolder): Array[String] = {
+ val normalizedProjections = DataSourceStrategy.normalizeExprs(
+ projections,
+ sHolder.output.map { a =>
+ sHolder.pushedJoinOutputMap.getOrElse(a, a).asInstanceOf[AttributeReference]
+ }
+ ).asInstanceOf[Seq[AttributeReference]]
+
+ normalizedProjections.map(_.name).toArray
+ }
+
def pushDownAggregates(plan: LogicalPlan): LogicalPlan = plan.transform {
// update the scan builder with agg pushdown and return a new plan with agg pushed
case agg: Aggregate => rewriteAggregate(agg)
@@ -113,10 +328,9 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper {
val aggExprToOutputOrdinal = mutable.HashMap.empty[Expression, Int]
val aggregates = collectAggregates(actualResultExprs, aggExprToOutputOrdinal)
- val normalizedAggExprs = DataSourceStrategy.normalizeExprs(
- aggregates, holder.relation.output).asInstanceOf[Seq[AggregateExpression]]
- val normalizedGroupingExpr = DataSourceStrategy.normalizeExprs(
- actualGroupExprs, holder.relation.output)
+ val normalizedAggExprs =
+ normalizeExpressions(aggregates, holder).asInstanceOf[Seq[AggregateExpression]]
+ val normalizedGroupingExpr = normalizeExpressions(actualGroupExprs, holder)
val translatedAggOpt = DataSourceStrategy.translateAggregation(
normalizedAggExprs, normalizedGroupingExpr)
if (translatedAggOpt.isEmpty) {
@@ -356,6 +570,25 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper {
Project(projectList, scanRelation)
}
+ def buildScanWithPushedJoin(plan: LogicalPlan): LogicalPlan = plan.transform {
+ case holder: ScanBuilderHolder if holder.joinedRelations.length > 1 =>
+ val scan = holder.builder.build()
+ val realOutput = toAttributes(scan.readSchema())
+ assert(realOutput.length == holder.output.length,
+ "The data source returns unexpected number of columns")
+ val wrappedScan = getWrappedScan(scan, holder)
+ val scanRelation = DataSourceV2ScanRelation(holder.relation, wrappedScan, realOutput)
+
+ // When join is pushed down, the real output is going to be, for example,
+ // SALARY_01234#0, NAME_ab123#1, DEPT_cd123#2.
+ // We should revert these names back to original names. For example,
+ // SALARY#0, NAME#1, DEPT#1. This is done by adding projection with appropriate aliases.
+ val projectList = realOutput.zip(holder.output).map { case (a1, a2) =>
+ Alias(a1, a2.name)(a2.exprId)
+ }
+ Project(projectList, scanRelation)
+ }
+
def pruneColumns(plan: LogicalPlan): LogicalPlan = plan.transform {
case ScanOperation(project, filtersStayUp, filtersPushDown, sHolder: ScanBuilderHolder) =>
// column pruning
@@ -441,8 +674,7 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper {
} else {
aliasReplacedOrder.asInstanceOf[Seq[SortOrder]]
}
- val normalizedOrders = DataSourceStrategy.normalizeExprs(
- newOrder, sHolder.relation.output).asInstanceOf[Seq[SortOrder]]
+ val normalizedOrders = normalizeExpressions(newOrder, sHolder).asInstanceOf[Seq[SortOrder]]
val orders = DataSourceStrategy.translateSortOrders(normalizedOrders)
if (orders.length == order.length) {
val (isPushed, isPartiallyPushed) =
@@ -540,6 +772,23 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper {
}
}
+ private def normalizeExpressions(
+ expressions: Seq[Expression],
+ sHolder: ScanBuilderHolder): Seq[Expression] = {
+ val output = if (sHolder.joinedRelations.length == 1) {
+ // Join is not pushed down
+ sHolder.relation.output
+ } else {
+ // sHolder.output's names can be out of date if the joins has previously been pushed down.
+ // For this reason, we need to use pushedJoinOutputMap to get up to date names.
+ sHolder.output.map { a =>
+ sHolder.pushedJoinOutputMap.getOrElse(a, a).asInstanceOf[AttributeReference]
+ }
+ }
+
+ DataSourceStrategy.normalizeExprs(expressions, output)
+ }
+
private def getWrappedScan(scan: Scan, sHolder: ScanBuilderHolder): Scan = {
scan match {
case v1: V1Scan =>
@@ -548,12 +797,18 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper {
f.pushedFilters()
case _ => Array.empty[sources.Filter]
}
- val pushedDownOperators = PushedDownOperators(sHolder.pushedAggregate, sHolder.pushedSample,
- sHolder.pushedLimit, sHolder.pushedOffset, sHolder.sortOrders, sHolder.pushedPredicates)
+ val pushedDownOperators = getPushedDownOperators(sHolder)
V1ScanWrapper(v1, pushedFilters.toImmutableArraySeq, pushedDownOperators)
case _ => scan
}
}
+
+ private def getPushedDownOperators(sHolder: ScanBuilderHolder): PushedDownOperators = {
+ val optRelationName = Option.when(sHolder.joinedRelations.length <= 1)(sHolder.relation.name)
+ PushedDownOperators(sHolder.pushedAggregate, sHolder.pushedSample,
+ sHolder.pushedLimit, sHolder.pushedOffset, sHolder.sortOrders, sHolder.pushedPredicates,
+ sHolder.joinedRelationsPushedDownOperators, optRelationName)
+ }
}
case class ScanBuilderHolder(
@@ -573,6 +828,12 @@ case class ScanBuilderHolder(
var pushedAggregate: Option[Aggregation] = None
var pushedAggOutputMap: AttributeMap[Expression] = AttributeMap.empty[Expression]
+
+ var joinedRelations: Seq[DataSourceV2RelationBase] = Seq(relation)
+
+ var joinedRelationsPushedDownOperators: Seq[PushedDownOperators] = Seq.empty[PushedDownOperators]
+
+ var pushedJoinOutputMap: AttributeMap[Expression] = AttributeMap.empty[Expression]
}
// A wrapper for v1 scan to carry the translated filters and the handled ones, along with
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala
index 1088e3f7a7206..891728e6f7e19 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala
@@ -309,7 +309,7 @@ class V2SessionCatalog(catalog: SessionCatalog)
collation = collation, storage = storage))
}
if (changes.exists(_.isInstanceOf[TableChange.ColumnChange])) {
- catalog.alterTableDataSchema(ident.asTableIdentifier, schema)
+ catalog.alterTableSchema(ident.asTableIdentifier, schema)
}
} catch {
case _: NoSuchTableException =>
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala
index 4436c6b24f7c8..68e8c1c6e1047 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala
@@ -17,10 +17,13 @@
package org.apache.spark.sql.execution.datasources.v2
+import java.lang
+import java.util
+
import scala.jdk.CollectionConverters._
import org.apache.spark.{SparkEnv, SparkException, TaskContext}
-import org.apache.spark.internal.{Logging, LogKeys, MDC}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.{InternalRow, ProjectingInternalRow}
import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
@@ -34,6 +37,7 @@ import org.apache.spark.sql.connector.metric.CustomMetric
import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, DeltaWrite, DeltaWriter, PhysicalWriteInfoImpl, Write, WriterCommitMessage}
import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
import org.apache.spark.sql.execution.{SparkPlan, SQLExecution, UnaryExecNode}
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
import org.apache.spark.sql.execution.metric.{CustomMetrics, SQLMetric, SQLMetrics}
import org.apache.spark.sql.types.StructType
import org.apache.spark.util.{LongAccumulator, Utils}
@@ -303,8 +307,6 @@ case class ReplaceDataExec(
projections: ReplaceDataProjections,
write: Write) extends V2ExistingTableWriteExec {
- override val stringArgs: Iterator[Any] = Iterator(query, write)
-
override def writingTask: WritingSparkTask[_] = {
projections match {
case ReplaceDataProjections(dataProj, Some(metadataProj)) =>
@@ -328,8 +330,6 @@ case class WriteDeltaExec(
projections: WriteDeltaProjections,
write: DeltaWrite) extends V2ExistingTableWriteExec {
- override lazy val stringArgs: Iterator[Any] = Iterator(query, write)
-
override lazy val writingTask: WritingSparkTask[_] = {
if (projections.metadataProjection.isDefined) {
DeltaWithMetadataWritingSparkTask(projections)
@@ -349,6 +349,8 @@ case class WriteToDataSourceV2Exec(
query: SparkPlan,
writeMetrics: Seq[CustomMetric]) extends V2TableWriteExec {
+ override val stringArgs: Iterator[Any] = Iterator(batchWrite, query)
+
override val customMetrics: Map[String, SQLMetric] = writeMetrics.map { customMetric =>
customMetric.name() -> SQLMetrics.createV2CustomMetric(sparkContext, customMetric)
}.toMap
@@ -367,6 +369,8 @@ trait V2ExistingTableWriteExec extends V2TableWriteExec {
def refreshCache: () => Unit
def write: Write
+ override val stringArgs: Iterator[Any] = Iterator(query, write)
+
override val customMetrics: Map[String, SQLMetric] =
write.supportedCustomMetrics().map { customMetric =>
customMetric.name() -> SQLMetrics.createV2CustomMetric(sparkContext, customMetric)
@@ -398,7 +402,7 @@ trait V2ExistingTableWriteExec extends V2TableWriteExec {
/**
* The base physical plan for writing data into data source v2.
*/
-trait V2TableWriteExec extends V2CommandExec with UnaryExecNode {
+trait V2TableWriteExec extends V2CommandExec with UnaryExecNode with AdaptiveSparkPlanHelper {
def query: SparkPlan
def writingTask: WritingSparkTask[_] = DataWritingSparkTask
@@ -451,8 +455,9 @@ trait V2TableWriteExec extends V2CommandExec with UnaryExecNode {
}
)
+ val operationMetrics = getOperationMetrics(query)
logInfo(log"Data source write support ${MDC(LogKeys.BATCH_WRITE, batchWrite)} is committing.")
- batchWrite.commit(messages)
+ batchWrite.commit(messages, operationMetrics)
logInfo(log"Data source write support ${MDC(LogKeys.BATCH_WRITE, batchWrite)} committed.")
commitProgress = Some(StreamWriterCommitProgress(totalNumRowsAccumulator.value))
} catch {
@@ -474,6 +479,12 @@ trait V2TableWriteExec extends V2CommandExec with UnaryExecNode {
Nil
}
+
+ private def getOperationMetrics(query: SparkPlan): util.Map[String, lang.Long] = {
+ collectFirst(query) { case m: MergeRowsExec => m }.map{ n =>
+ n.metrics.map { case (name, metric) => s"merge.$name" -> lang.Long.valueOf(metric.value) }
+ }.getOrElse(Map.empty[String, lang.Long]).asJava
+ }
}
trait WritingSparkTask[W <: DataWriter[InternalRow]] extends Logging with Serializable {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala
index 230f30fb1d069..b758ddd35e0d2 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala
@@ -19,21 +19,25 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc
import scala.util.control.NonFatal
import org.apache.spark.internal.Logging
+import org.apache.spark.internal.LogKeys.{JOIN_CONDITION, JOIN_TYPE, SCHEMA}
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.connector.expressions.{FieldReference, SortOrder}
import org.apache.spark.sql.connector.expressions.aggregate.Aggregation
import org.apache.spark.sql.connector.expressions.filter.Predicate
-import org.apache.spark.sql.connector.read.{ScanBuilder, SupportsPushDownAggregates, SupportsPushDownLimit, SupportsPushDownOffset, SupportsPushDownRequiredColumns, SupportsPushDownTableSample, SupportsPushDownTopN, SupportsPushDownV2Filters}
+import org.apache.spark.sql.connector.join.JoinType
+import org.apache.spark.sql.connector.read.{ScanBuilder, SupportsPushDownAggregates, SupportsPushDownJoin, SupportsPushDownLimit, SupportsPushDownOffset, SupportsPushDownRequiredColumns, SupportsPushDownTableSample, SupportsPushDownTopN, SupportsPushDownV2Filters}
import org.apache.spark.sql.execution.datasources.PartitioningUtils
-import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JDBCRDD, JDBCRelation}
+import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JDBCPartition, JDBCRDD, JDBCRelation}
import org.apache.spark.sql.execution.datasources.v2.TableSampleInfo
-import org.apache.spark.sql.jdbc.JdbcDialects
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.sql.jdbc.{JdbcDialects, JdbcSQLQueryBuilder, JoinPushdownAliasGenerator}
import org.apache.spark.sql.types.StructType
case class JDBCScanBuilder(
session: SparkSession,
schema: StructType,
- jdbcOptions: JDBCOptions)
+ var jdbcOptions: JDBCOptions,
+ additionalMetrics: Map[String, SQLMetric] = Map())
extends ScanBuilder
with SupportsPushDownV2Filters
with SupportsPushDownRequiredColumns
@@ -42,6 +46,7 @@ case class JDBCScanBuilder(
with SupportsPushDownOffset
with SupportsPushDownTableSample
with SupportsPushDownTopN
+ with SupportsPushDownJoin
with Logging {
private val dialect = JdbcDialects.get(jdbcOptions.url)
@@ -121,6 +126,151 @@ case class JDBCScanBuilder(
}
}
+ // TODO: currently we check that all the options are same (besides dbtable and query options).
+ // That is too strict, so in the future we should relax this check by asserting only specific
+ // options are some (e.g. host, port, username, password, database...).
+ // Also, we need to check if join is done on 2 tables from 2 different databases within same
+ // host. These shouldn't be allowed.
+ override def isOtherSideCompatibleForJoin(other: SupportsPushDownJoin): Boolean = {
+ if (!jdbcOptions.pushDownJoin ||
+ !dialect.supportsJoin ||
+ !other.isInstanceOf[JDBCScanBuilder]) {
+ return false
+ }
+
+ val filteredJDBCOptions = jdbcOptions.parameters -
+ JDBCOptions.JDBC_TABLE_NAME -
+ JDBCOptions.JDBC_QUERY_STRING
+
+ val otherSideFilteredJDBCOptions = other.asInstanceOf[JDBCScanBuilder].jdbcOptions.parameters -
+ JDBCOptions.JDBC_TABLE_NAME -
+ JDBCOptions.JDBC_QUERY_STRING
+
+ filteredJDBCOptions == otherSideFilteredJDBCOptions
+ };
+
+ /**
+ * Helper method to calculate StructType based on the SupportsPushDownJoin.ColumnWithAlias and
+ * the given schema.
+ *
+ * If ColumnWithAlias object has defined alias, new field with new name being equal to alias
+ * should be returned. Otherwise, original field is returned.
+ */
+ private def calculateJoinOutputSchema(
+ columnsWithAliases: Array[SupportsPushDownJoin.ColumnWithAlias],
+ schema: StructType): StructType = {
+ var newSchema = StructType(Seq())
+ columnsWithAliases.foreach { columnWithAlias =>
+ val colName = columnWithAlias.colName()
+ val alias = columnWithAlias.alias()
+ val field = schema(colName)
+
+ if (alias == null) {
+ newSchema = newSchema.add(field)
+ } else {
+ newSchema = newSchema.add(alias, field.dataType, field.nullable, field.metadata)
+ }
+ }
+
+ newSchema
+ }
+
+ override def pushDownJoin(
+ other: SupportsPushDownJoin,
+ joinType: JoinType,
+ leftSideRequiredColumnsWithAliases: Array[SupportsPushDownJoin.ColumnWithAlias],
+ rightSideRequiredColumnsWithAliases: Array[SupportsPushDownJoin.ColumnWithAlias],
+ condition: Predicate ): Boolean = {
+ if (!jdbcOptions.pushDownJoin || !dialect.supportsJoin) {
+ return false
+ }
+
+ val joinTypeStringOption = joinType match {
+ case JoinType.INNER_JOIN => Some("INNER JOIN")
+ case JoinType.LEFT_OUTER_JOIN => Some("LEFT JOIN")
+ case JoinType.RIGHT_OUTER_JOIN => Some("RIGHT JOIN")
+ case _ => None
+ }
+ if (!joinTypeStringOption.isDefined) {
+ logError(log"Failed to push down join to JDBC due to unsupported join type " +
+ log"${MDC(JOIN_TYPE, joinType)}")
+ return false
+ }
+
+ val compiledCondition = dialect.compileExpression(condition)
+ if (!compiledCondition.isDefined) {
+ logError(log"Failed to push down join to JDBC due to unsupported join condition " +
+ log"${MDC(JOIN_CONDITION, condition)}")
+ return false
+ }
+
+ val otherJdbcScanBuilder = other.asInstanceOf[JDBCScanBuilder]
+
+ // requiredSchema will become the finalSchema of this JDBCScanBuilder
+ var requiredSchema = StructType(Seq())
+ requiredSchema = calculateJoinOutputSchema(leftSideRequiredColumnsWithAliases, finalSchema)
+ requiredSchema = requiredSchema.merge(
+ calculateJoinOutputSchema(
+ rightSideRequiredColumnsWithAliases,
+ otherJdbcScanBuilder.finalSchema
+ )
+ )
+
+ val joinOutputColumns = requiredSchema.fields.map(f => dialect.quoteIdentifier(f.name))
+ val conditionString = compiledCondition.get
+
+ // Get left side and right side of join sql query builders and recursively build them when
+ // crafting join sql query.
+ val leftSideJdbcSQLBuilder = getJoinPushdownJdbcSQLBuilder(leftSideRequiredColumnsWithAliases)
+ val otherSideJdbcSQLBuilder = otherJdbcScanBuilder
+ .getJoinPushdownJdbcSQLBuilder(rightSideRequiredColumnsWithAliases)
+
+ val joinQuery = dialect
+ .getJdbcSQLQueryBuilder(jdbcOptions)
+ .withJoin(
+ leftSideJdbcSQLBuilder,
+ otherSideJdbcSQLBuilder,
+ JoinPushdownAliasGenerator.getSubqueryQualifier,
+ JoinPushdownAliasGenerator.getSubqueryQualifier,
+ joinOutputColumns,
+ joinTypeStringOption.get,
+ conditionString
+ )
+ .build()
+
+ val newJdbcOptionsMap = jdbcOptions.parameters.originalMap +
+ (JDBCOptions.JDBC_QUERY_STRING -> joinQuery) - JDBCOptions.JDBC_TABLE_NAME
+
+ jdbcOptions = new JDBCOptions(newJdbcOptionsMap)
+ finalSchema = requiredSchema
+ logInfo(log"Updated JDBC schema due to join pushdown. " +
+ log"New schema: ${MDC(SCHEMA, finalSchema.toDDL)}")
+
+ // We need to reset the pushedPredicate because it has already been consumed in previously
+ // crafted SQL query.
+ pushedPredicate = Array.empty[Predicate]
+ // Table sample is pushed down already as well, so we need to reset it to None to not push it
+ // down again when join pushdown is triggered again on this JDBCScanBuilder.
+ tableSample = None
+
+ true
+ }
+
+ def getJoinPushdownJdbcSQLBuilder(
+ columnsWithAliases: Array[SupportsPushDownJoin.ColumnWithAlias]): JdbcSQLQueryBuilder = {
+ val quotedColumns = columnsWithAliases.map(col => dialect.quoteIdentifier(col.colName()))
+ val quotedAliases = columnsWithAliases
+ .map(col => Option(col.alias()).map(dialect.quoteIdentifier))
+
+ // Only filters can be pushed down before join pushdown, so we need to craft SQL query
+ // that contains filters as well.
+ // Joins on top of samples are not supported so we don't need to provide tableSample here.
+ dialect
+ .getJdbcSQLQueryBuilder(jdbcOptions)
+ .withPredicates(pushedPredicate, JDBCPartition(whereClause = null, idx = 1))
+ .withAliasedColumns(quotedColumns, quotedAliases)
+ }
+
override def pushTableSample(
lowerBound: Double,
upperBound: Double,
@@ -191,7 +341,9 @@ case class JDBCScanBuilder(
// "DEPT","NAME",MAX("SALARY"),MIN("BONUS"), instead of getting column names from
// prunedSchema and quote them (will become "MAX(SALARY)", "MIN(BONUS)" and can't
// be used in sql string.
- JDBCScan(JDBCRelation(schema, parts, jdbcOptions)(session), finalSchema, pushedPredicate,
- pushedAggregateList, pushedGroupBys, tableSample, pushedLimit, sortOrders, pushedOffset)
+ JDBCScan(JDBCRelation(schema, parts, jdbcOptions, additionalMetrics)(session),
+ finalSchema, pushedPredicate, pushedAggregateList, pushedGroupBys,
+ tableSample, pushedLimit, sortOrders, pushedOffset)
}
+
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTable.scala
index cf9aa4f8682bf..a242a24baf9cd 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTable.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTable.scala
@@ -28,17 +28,30 @@ import org.apache.spark.sql.connector.expressions.NamedReference
import org.apache.spark.sql.connector.write.{LogicalWriteInfo, WriteBuilder}
import org.apache.spark.sql.errors.DataTypeErrorsBase
import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcOptionsInWrite, JdbcUtils}
+import org.apache.spark.sql.execution.metric.SQLMetric
import org.apache.spark.sql.jdbc.JdbcDialects
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.util.CaseInsensitiveStringMap
-case class JDBCTable(ident: Identifier, schema: StructType, jdbcOptions: JDBCOptions)
+case class JDBCTable(
+ ident: Identifier,
+ override val schema: StructType,
+ jdbcOptions: JDBCOptions,
+ additionalMetrics: Map[String, SQLMetric] = Map())
extends Table
with SupportsRead
with SupportsWrite
with SupportsIndex
with DataTypeErrorsBase {
+ override def hashCode(): Int = (ident, schema, jdbcOptions).##
+
+ override def equals(obj: Any): Boolean = obj match {
+ case that: JDBCTable =>
+ this.ident == that.ident && this.schema == that.schema && this.jdbcOptions == that.jdbcOptions
+ case _ => false
+ }
+
override def name(): String = ident.toString
override def capabilities(): util.Set[TableCapability] = {
@@ -48,7 +61,7 @@ case class JDBCTable(ident: Identifier, schema: StructType, jdbcOptions: JDBCOpt
override def newScanBuilder(options: CaseInsensitiveStringMap): JDBCScanBuilder = {
val mergedOptions = new JDBCOptions(
jdbcOptions.parameters.originalMap ++ options.asCaseSensitiveMap().asScala)
- JDBCScanBuilder(SparkSession.active, schema, mergedOptions)
+ JDBCScanBuilder(SparkSession.active, schema, mergedOptions, additionalMetrics)
}
override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala
index 63048028c5877..7f9a408a597d8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala
@@ -16,7 +16,7 @@
*/
package org.apache.spark.sql.execution.datasources.v2.jdbc
-import java.sql.SQLException
+import java.sql.{Connection, SQLException}
import java.util
import scala.collection.mutable
@@ -24,11 +24,13 @@ import scala.jdk.CollectionConverters._
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException
+import org.apache.spark.sql.classic.SparkSession
import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Column, FunctionCatalog, Identifier, NamespaceChange, SupportsNamespaces, Table, TableCatalog, TableChange, TableSummary}
import org.apache.spark.sql.connector.catalog.functions.UnboundFunction
import org.apache.spark.sql.connector.expressions.Transform
import org.apache.spark.sql.errors.{DataTypeErrorsBase, QueryCompilationErrors, QueryExecutionErrors}
-import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcOptionsInWrite, JDBCRDD, JdbcUtils}
+import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcOptionsInWrite, JDBCRDD, JDBCRelation, JdbcUtils}
+import org.apache.spark.sql.execution.metric.SQLMetrics
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects}
import org.apache.spark.sql.util.CaseInsensitiveStringMap
@@ -91,19 +93,21 @@ class JDBCTableCatalog extends TableCatalog
}
override def tableExists(ident: Identifier): Boolean = {
+ JdbcUtils.withConnection(options)(tableExists(ident, _))
+ }
+
+ private def tableExists(ident: Identifier, conn: Connection): Boolean = {
checkNamespace(ident.namespace())
val writeOptions = new JdbcOptionsInWrite(
options.parameters + (JDBCOptions.JDBC_TABLE_NAME -> getTableName(ident)))
- JdbcUtils.withConnection(options) {
- JdbcUtils.classifyException(
- condition = "FAILED_JDBC.TABLE_EXISTS",
- messageParameters = Map(
- "url" -> options.getRedactUrl(),
- "tableName" -> toSQLId(ident)),
- dialect,
- description = s"Failed table existence check: $ident",
- isRuntime = false)(JdbcUtils.tableExists(_, writeOptions))
- }
+ JdbcUtils.classifyException(
+ condition = "FAILED_JDBC.TABLE_EXISTS",
+ messageParameters = Map(
+ "url" -> options.getRedactUrl(),
+ "tableName" -> toSQLId(ident)),
+ dialect,
+ description = s"Failed table existence check: $ident",
+ isRuntime = false)(JdbcUtils.tableExists(conn, writeOptions))
}
override def dropTable(ident: Identifier): Boolean = {
@@ -136,23 +140,30 @@ class JDBCTableCatalog extends TableCatalog
}
override def loadTable(ident: Identifier): Table = {
- if (!tableExists(ident)) {
- throw QueryCompilationErrors.noSuchTableError(ident)
- }
+ JdbcUtils.withConnection(options) { conn =>
+ if (!tableExists(ident, conn)) {
+ throw QueryCompilationErrors.noSuchTableError(ident)
+ }
- val optionsWithTableName = new JDBCOptions(
- options.parameters + (JDBCOptions.JDBC_TABLE_NAME -> getTableName(ident)))
- JdbcUtils.classifyException(
- condition = "FAILED_JDBC.LOAD_TABLE",
- messageParameters = Map(
- "url" -> options.getRedactUrl(),
- "tableName" -> toSQLId(ident)),
- dialect,
- description = s"Failed to load table: $ident",
- isRuntime = false
- ) {
- val schema = JDBCRDD.resolveTable(optionsWithTableName)
- JDBCTable(ident, schema, optionsWithTableName)
+ val optionsWithTableName = new JDBCOptions(
+ options.parameters + (JDBCOptions.JDBC_TABLE_NAME -> getTableName(ident)))
+ JdbcUtils.classifyException(
+ condition = "FAILED_JDBC.LOAD_TABLE",
+ messageParameters = Map(
+ "url" -> options.getRedactUrl(),
+ "tableName" -> toSQLId(ident)),
+ dialect,
+ description = s"Failed to load table: $ident",
+ isRuntime = false
+ ) {
+ val remoteSchemaFetchMetric = JdbcUtils
+ .createSchemaFetchMetric(SparkSession.active.sparkContext)
+ val schema = SQLMetrics.withTimingNs(remoteSchemaFetchMetric) {
+ JDBCRDD.resolveTable(optionsWithTableName, conn)
+ }
+ JDBCTable(ident, schema, optionsWithTableName,
+ Map(JDBCRelation.schemaFetchKey -> remoteSchemaFetchMetric))
+ }
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala
index ac15456f0c3d4..70ae8068a03a0 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala
@@ -84,6 +84,8 @@ case class ParquetPartitionReaderFactory(
private val datetimeRebaseModeInRead = options.datetimeRebaseModeInRead
private val int96RebaseModeInRead = options.int96RebaseModeInRead
+ private val parquetReaderCallback = new ParquetReaderCallback()
+
private def getFooter(file: PartitionedFile): ParquetMetadata = {
val conf = broadcastedConf.value.value
if (aggregation.isDefined || enableVectorizedReader) {
@@ -309,7 +311,8 @@ case class ParquetPartitionReaderFactory(
reader, readDataSchema)
val iter = new RecordReaderIterator(readerWithRowIndexes)
// SPARK-23457 Register a task completion listener before `initialization`.
- taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close()))
+ parquetReaderCallback.advanceFile(iter)
+ taskContext.foreach(parquetReaderCallback.initIfNotAlready)
readerWithRowIndexes
}
@@ -337,8 +340,39 @@ case class ParquetPartitionReaderFactory(
capacity)
val iter = new RecordReaderIterator(vectorizedReader)
// SPARK-23457 Register a task completion listener before `initialization`.
- taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close()))
+ parquetReaderCallback.advanceFile(iter)
+ taskContext.foreach(parquetReaderCallback.initIfNotAlready)
logDebug(s"Appending $partitionSchema $partitionValues")
vectorizedReader
}
}
+
+/**
+ * A callback class to handle the cleanup of Parquet readers.
+ *
+ * This class is used to ensure that the Parquet readers are closed properly when the task
+ * completes, and it also allows for the initialization of the reader callback only once per task.
+ */
+private class ParquetReaderCallback extends Serializable {
+ private var init: Boolean = false
+ private var iter: RecordReaderIterator[_] = null
+
+ def initIfNotAlready(taskContext: TaskContext): Unit = {
+ if (!init) {
+ taskContext.addTaskCompletionListener[Unit](_ => closeCurrent())
+ init = true
+ }
+ }
+
+ def advanceFile(iter: RecordReaderIterator[_]): Unit = {
+ closeCurrent()
+
+ this.iter = iter
+ }
+
+ def closeCurrent(): Unit = {
+ if (iter != null) {
+ iter.close()
+ }
+ }
+}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonStreamingPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonStreamingPartitionReaderFactory.scala
index 7d80cc2728102..466ecf6090938 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonStreamingPartitionReaderFactory.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonStreamingPartitionReaderFactory.scala
@@ -19,7 +19,7 @@
package org.apache.spark.sql.execution.datasources.v2.python
import org.apache.spark.SparkEnv
-import org.apache.spark.internal.{Logging, LogKeys, MDC}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.connector.metric.CustomTaskMetric
import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala
index 937eb1fc042d5..828c06ab834ab 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala
@@ -29,13 +29,18 @@ import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.DataSourceOptions
import org.apache.spark.sql.connector.catalog.{Table, TableProvider}
import org.apache.spark.sql.connector.expressions.Transform
-import org.apache.spark.sql.execution.datasources.v2.state.StateSourceOptions.{JoinSideValues, READ_REGISTERED_TIMERS, STATE_VAR_NAME}
+import org.apache.spark.sql.execution.datasources.v2.state.StateSourceOptions.{JoinSideValues, READ_REGISTERED_TIMERS, STATE_VAR_NAME, STORE_NAME}
import org.apache.spark.sql.execution.datasources.v2.state.StateSourceOptions.JoinSideValues.JoinSideValues
import org.apache.spark.sql.execution.datasources.v2.state.metadata.{StateMetadataPartitionReader, StateMetadataTableEntry}
import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
-import org.apache.spark.sql.execution.streaming.{OffsetSeqMetadata, StreamingQueryCheckpointMetadata, TimerStateUtils, TransformWithStateOperatorProperties, TransformWithStateVariableInfo}
-import org.apache.spark.sql.execution.streaming.StreamingCheckpointConstants.DIR_NAME_STATE
-import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.checkpointing.OffsetSeqMetadata
+import org.apache.spark.sql.execution.streaming.operators.stateful.StatefulOperatorsUtils
+import org.apache.spark.sql.execution.streaming.operators.stateful.join.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.operators.stateful.join.SymmetricHashJoinStateManager
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.{TransformWithStateOperatorProperties, TransformWithStateVariableInfo}
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.timers.TimerStateUtils
+import org.apache.spark.sql.execution.streaming.runtime.StreamingCheckpointConstants.DIR_NAME_STATE
+import org.apache.spark.sql.execution.streaming.runtime.StreamingQueryCheckpointMetadata
import org.apache.spark.sql.execution.streaming.state.{InMemoryStateSchemaProvider, KeyStateEncoderSpec, NoPrefixKeyStateEncoderSpec, PrefixKeyScanStateEncoderSpec, StateSchemaCompatibilityChecker, StateSchemaMetadata, StateSchemaProvider, StateStore, StateStoreColFamilySchema, StateStoreConf, StateStoreId, StateStoreProviderId}
import org.apache.spark.sql.sources.DataSourceRegister
import org.apache.spark.sql.streaming.TimeMode
@@ -51,25 +56,17 @@ class StateDataSource extends TableProvider with DataSourceRegister with Logging
private lazy val hadoopConf: Configuration = session.sessionState.newHadoopConf()
- private lazy val serializedHadoopConf = new SerializableConfiguration(hadoopConf)
-
- // Seq of operator names who uses state schema v3 and TWS related options.
- // This Seq was used in checks before reading state schema files.
- private val twsShortNameSeq = Seq(
- "transformWithStateExec",
- "transformWithStateInPandasExec",
- "transformWithStateInPySparkExec"
- )
-
override def shortName(): String = "statestore"
override def getTable(
schema: StructType,
partitioning: Array[Transform],
properties: util.Map[String, String]): Table = {
- val sourceOptions = StateSourceOptions.apply(session, hadoopConf, properties)
+ val sourceOptions = StateSourceOptions.modifySourceOptions(hadoopConf,
+ StateSourceOptions.apply(session, hadoopConf, properties))
val stateConf = buildStateStoreConf(sourceOptions.resolvedCpLocation, sourceOptions.batchId)
- val stateStoreReaderInfo: StateStoreReaderInfo = getStoreMetadataAndRunChecks(sourceOptions)
+ val stateStoreReaderInfo: StateStoreReaderInfo = getStoreMetadataAndRunChecks(
+ sourceOptions)
// The key state encoder spec should be available for all operators except stream-stream joins
val keyStateEncoderSpec = if (stateStoreReaderInfo.keyStateEncoderSpecOpt.isDefined) {
@@ -82,25 +79,28 @@ class StateDataSource extends TableProvider with DataSourceRegister with Logging
new StateTable(session, schema, sourceOptions, stateConf, keyStateEncoderSpec,
stateStoreReaderInfo.transformWithStateVariableInfoOpt,
stateStoreReaderInfo.stateStoreColFamilySchemaOpt,
- stateStoreReaderInfo.stateSchemaProviderOpt)
+ stateStoreReaderInfo.stateSchemaProviderOpt,
+ stateStoreReaderInfo.joinColFamilyOpt)
}
override def inferSchema(options: CaseInsensitiveStringMap): StructType = {
- val sourceOptions = StateSourceOptions.apply(session, hadoopConf, options)
+ val sourceOptions = StateSourceOptions.modifySourceOptions(hadoopConf,
+ StateSourceOptions.apply(session, hadoopConf, options))
- val stateStoreReaderInfo: StateStoreReaderInfo = getStoreMetadataAndRunChecks(sourceOptions)
+ val stateStoreReaderInfo: StateStoreReaderInfo = getStoreMetadataAndRunChecks(
+ sourceOptions)
+ val oldSchemaFilePaths = StateDataSource.getOldSchemaFilePaths(sourceOptions, hadoopConf)
val stateCheckpointLocation = sourceOptions.stateCheckpointLocation
try {
- // SPARK-51779 TODO: Support stream-stream joins with virtual column families
val (keySchema, valueSchema) = sourceOptions.joinSide match {
case JoinSideValues.left =>
StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
- sourceOptions.operatorId, LeftSide)
+ sourceOptions.operatorId, LeftSide, oldSchemaFilePaths)
case JoinSideValues.right =>
StreamStreamJoinStateHelper.readKeyValueSchema(session, stateCheckpointLocation.toString,
- sourceOptions.operatorId, RightSide)
+ sourceOptions.operatorId, RightSide, oldSchemaFilePaths)
case JoinSideValues.none =>
// we should have the schema for the state store if joinSide is none
@@ -141,19 +141,7 @@ class StateDataSource extends TableProvider with DataSourceRegister with Logging
private def runStateVarChecks(
sourceOptions: StateSourceOptions,
stateStoreMetadata: Array[StateMetadataTableEntry]): Unit = {
- if (sourceOptions.stateVarName.isDefined || sourceOptions.readRegisteredTimers) {
- // Perform checks for transformWithState operator in case state variable name is provided
- require(stateStoreMetadata.size == 1)
- val opMetadata = stateStoreMetadata.head
- if (!twsShortNameSeq.contains(opMetadata.operatorName)) {
- // if we are trying to query state source with state variable name, then the operator
- // should be transformWithState
- val errorMsg = "Providing state variable names is only supported with the " +
- s"transformWithState operator. Found operator=${opMetadata.operatorName}. " +
- s"Please remove this option and re-run the query."
- throw StateDataSourceErrors.invalidOptionValue(STATE_VAR_NAME, errorMsg)
- }
-
+ def runTWSChecks(opMetadata: StateMetadataTableEntry): Unit = {
// if the operator is transformWithState, but the operator properties are empty, then
// the user has not defined any state variables for the operator
val operatorProperties = opMetadata.operatorPropertiesJson
@@ -183,35 +171,74 @@ class StateDataSource extends TableProvider with DataSourceRegister with Logging
throw StateDataSourceErrors.invalidOptionValue(STATE_VAR_NAME,
s"State variable $stateVarName is not defined for the transformWithState operator.")
}
- } else {
- // if the operator is transformWithState, then a state variable argument is mandatory
- if (stateStoreMetadata.size == 1 &&
- twsShortNameSeq.contains(stateStoreMetadata.head.operatorName)) {
- throw StateDataSourceErrors.requiredOptionUnspecified("stateVarName")
- }
}
- }
- private def getStateStoreMetadata(stateSourceOptions: StateSourceOptions):
- Array[StateMetadataTableEntry] = {
- val allStateStoreMetadata = new StateMetadataPartitionReader(
- stateSourceOptions.stateCheckpointLocation.getParent.toString,
- serializedHadoopConf, stateSourceOptions.batchId).stateMetadata.toArray
- val stateStoreMetadata = allStateStoreMetadata.filter { entry =>
- entry.operatorId == stateSourceOptions.operatorId &&
- entry.stateStoreName == stateSourceOptions.storeName
+ sourceOptions.stateVarName match {
+ case Some(name) =>
+ // Check that stateStoreMetadata exists
+ require(stateStoreMetadata.size == 1)
+ val opMetadata = stateStoreMetadata.head
+ opMetadata.operatorName match {
+ case opName: String if opName ==
+ StatefulOperatorsUtils.SYMMETRIC_HASH_JOIN_EXEC_OP_NAME =>
+ // Verify that the storename is valid
+ val possibleStoreNames = SymmetricHashJoinStateManager.allStateStoreNames(
+ LeftSide, RightSide)
+ if (!possibleStoreNames.contains(name)) {
+ val errorMsg = s"Store name $name not allowed for join operator. Allowed names are " +
+ s"$possibleStoreNames. " +
+ s"Please remove this option and re-run the query."
+ throw StateDataSourceErrors.invalidOptionValue(STORE_NAME, errorMsg)
+ }
+ case opName: String if StatefulOperatorsUtils.TRANSFORM_WITH_STATE_OP_NAMES
+ .contains(opName) =>
+ runTWSChecks(opMetadata)
+ case _ =>
+ // if we are trying to query state source with state variable name, then the operator
+ // should be transformWithState
+ val errorMsg = "Providing state variable names is only supported with the " +
+ s"transformWithState operator. Found operator=${opMetadata.operatorName}. " +
+ s"Please remove this option and re-run the query."
+ throw StateDataSourceErrors.invalidOptionValue(STATE_VAR_NAME, errorMsg)
+ }
+ case None =>
+ if (sourceOptions.readRegisteredTimers) {
+ // Check that stateStoreMetadata exists
+ require(stateStoreMetadata.size == 1)
+ val opMetadata = stateStoreMetadata.head
+ opMetadata.operatorName match {
+ case opName: String if StatefulOperatorsUtils.TRANSFORM_WITH_STATE_OP_NAMES
+ .contains(opName) =>
+ runTWSChecks(opMetadata)
+ case _ =>
+ // if we are trying to query state source with state variable name, then the operator
+ // should be transformWithState
+ val errorMsg = "Providing readRegisteredTimers=true is only supported with the " +
+ s"transformWithState operator. Found operator=${opMetadata.operatorName}. " +
+ s"Please remove this option and re-run the query."
+ throw StateDataSourceErrors.invalidOptionValue(READ_REGISTERED_TIMERS, errorMsg)
+ }
+ } else {
+ // if the operator is transformWithState, then a state variable argument is mandatory
+ if (stateStoreMetadata.size == 1 &&
+ StatefulOperatorsUtils.TRANSFORM_WITH_STATE_OP_NAMES.contains(
+ stateStoreMetadata.head.operatorName)) {
+ throw StateDataSourceErrors.requiredOptionUnspecified("stateVarName")
+ }
+ }
}
- stateStoreMetadata
}
private def getStoreMetadataAndRunChecks(sourceOptions: StateSourceOptions):
StateStoreReaderInfo = {
- val storeMetadata = getStateStoreMetadata(sourceOptions)
+ val storeMetadata = StateDataSource.getStateStoreMetadata(sourceOptions, hadoopConf)
runStateVarChecks(sourceOptions, storeMetadata)
+
var keyStateEncoderSpecOpt: Option[KeyStateEncoderSpec] = None
var stateStoreColFamilySchemaOpt: Option[StateStoreColFamilySchema] = None
var transformWithStateVariableInfoOpt: Option[TransformWithStateVariableInfo] = None
var stateSchemaProvider: Option[StateSchemaProvider] = None
+ var joinColFamilyOpt: Option[String] = None
var timeMode: String = TimeMode.None.toString
if (sourceOptions.joinSide == JoinSideValues.none) {
@@ -220,34 +247,41 @@ class StateDataSource extends TableProvider with DataSourceRegister with Logging
// Read the schema file path from operator metadata version v2 onwards
// for the transformWithState operator
- val oldSchemaFilePaths = if (storeMetadata.length > 0 && storeMetadata.head.version == 2
- && twsShortNameSeq.exists(storeMetadata.head.operatorName.contains)) {
- val storeMetadataEntry = storeMetadata.head
- val operatorProperties = TransformWithStateOperatorProperties.fromJson(
- storeMetadataEntry.operatorPropertiesJson)
- timeMode = operatorProperties.timeMode
-
- if (sourceOptions.readRegisteredTimers) {
- stateVarName = TimerStateUtils.getTimerStateVarNames(timeMode)._1
+ val oldSchemaFilePaths = if (storeMetadata.length > 0 && storeMetadata.head.version == 2) {
+ val opName = storeMetadata.head.operatorName
+ if (StatefulOperatorsUtils.TRANSFORM_WITH_STATE_OP_NAMES.exists(opName.contains)) {
+ val storeMetadataEntry = storeMetadata.head
+ val operatorProperties = TransformWithStateOperatorProperties.fromJson(
+ storeMetadataEntry.operatorPropertiesJson)
+ timeMode = operatorProperties.timeMode
+
+ if (sourceOptions.readRegisteredTimers) {
+ stateVarName = TimerStateUtils.getTimerStateVarNames(timeMode)._1
+ }
+
+ val stateVarInfoList = operatorProperties.stateVariables
+ .filter(stateVar => stateVar.stateName == stateVarName)
+ require(stateVarInfoList.size == 1, s"Failed to find unique state variable info " +
+ s"for state variable $stateVarName in operator ${sourceOptions.operatorId}")
+ val stateVarInfo = stateVarInfoList.head
+ transformWithStateVariableInfoOpt = Some(stateVarInfo)
+ val schemaFilePaths = storeMetadataEntry.stateSchemaFilePaths
+ val stateSchemaMetadata = StateSchemaMetadata.createStateSchemaMetadata(
+ sourceOptions.stateCheckpointLocation.toString,
+ hadoopConf,
+ schemaFilePaths
+ )
+ stateSchemaProvider = Some(new InMemoryStateSchemaProvider(stateSchemaMetadata))
+ schemaFilePaths.map(new Path(_))
+ } else {
+ if (opName == StatefulOperatorsUtils.SYMMETRIC_HASH_JOIN_EXEC_OP_NAME) {
+ joinColFamilyOpt = Some(stateVarName)
+ }
+ StateDataSource.getOldSchemaFilePaths(sourceOptions, hadoopConf)
}
-
- val stateVarInfoList = operatorProperties.stateVariables
- .filter(stateVar => stateVar.stateName == stateVarName)
- require(stateVarInfoList.size == 1, s"Failed to find unique state variable info " +
- s"for state variable $stateVarName in operator ${sourceOptions.operatorId}")
- val stateVarInfo = stateVarInfoList.head
- transformWithStateVariableInfoOpt = Some(stateVarInfo)
- val schemaFilePaths = storeMetadataEntry.stateSchemaFilePaths
- val stateSchemaMetadata = StateSchemaMetadata.createStateSchemaMetadata(
- sourceOptions.stateCheckpointLocation.toString,
- hadoopConf,
- schemaFilePaths
- )
- stateSchemaProvider = Some(new InMemoryStateSchemaProvider(stateSchemaMetadata))
- schemaFilePaths.map(new Path(_))
} else {
- None
- }.toList
+ StateDataSource.getOldSchemaFilePaths(sourceOptions, hadoopConf)
+ }
try {
// Read the actual state schema from the provided path for v2 or from the dedicated path
@@ -276,7 +310,8 @@ class StateDataSource extends TableProvider with DataSourceRegister with Logging
keyStateEncoderSpecOpt,
stateStoreColFamilySchemaOpt,
transformWithStateVariableInfoOpt,
- stateSchemaProvider
+ stateSchemaProvider,
+ joinColFamilyOpt
)
}
@@ -335,7 +370,8 @@ case class StateSourceOptions(
readChangeFeedOptions: Option[ReadChangeFeedOptions],
stateVarName: Option[String],
readRegisteredTimers: Boolean,
- flattenCollectionTypes: Boolean) {
+ flattenCollectionTypes: Boolean,
+ operatorStateUniqueIds: Option[Array[Array[String]]] = None) {
def stateCheckpointLocation: Path = new Path(resolvedCpLocation, DIR_NAME_STATE)
override def toString: String = {
@@ -532,10 +568,37 @@ object StateSourceOptions extends DataSourceOptions {
}
}
+ val startBatchId = if (fromSnapshotOptions.isDefined) {
+ fromSnapshotOptions.get.snapshotStartBatchId
+ } else if (readChangeFeedOptions.isDefined) {
+ readChangeFeedOptions.get.changeStartBatchId
+ } else {
+ batchId.get
+ }
+
+ val operatorStateUniqueIds = getOperatorStateUniqueIds(
+ sparkSession,
+ startBatchId,
+ operatorId,
+ resolvedCpLocation)
+
+ if (operatorStateUniqueIds.isDefined) {
+ if (fromSnapshotOptions.isDefined) {
+ throw StateDataSourceErrors.invalidOptionValue(
+ SNAPSHOT_START_BATCH_ID,
+ "Snapshot reading is currently not supported with checkpoint v2.")
+ }
+ if (readChangeFeedOptions.isDefined) {
+ throw StateDataSourceErrors.invalidOptionValue(
+ READ_CHANGE_FEED,
+ "Read change feed is currently not supported with checkpoint v2.")
+ }
+ }
+
StateSourceOptions(
resolvedCpLocation, batchId.get, operatorId, storeName, joinSide,
readChangeFeed, fromSnapshotOptions, readChangeFeedOptions,
- stateVarName, readRegisteredTimers, flattenCollectionTypes)
+ stateVarName, readRegisteredTimers, flattenCollectionTypes, operatorStateUniqueIds)
}
private def resolvedCheckpointLocation(
@@ -553,6 +616,41 @@ object StateSourceOptions extends DataSourceOptions {
case None => throw StateDataSourceErrors.committedBatchUnavailable(checkpointLocation)
}
}
+
+ private def getOperatorStateUniqueIds(
+ session: SparkSession,
+ batchId: Long,
+ operatorId: Long,
+ checkpointLocation: String): Option[Array[Array[String]]] = {
+ val commitLog = new StreamingQueryCheckpointMetadata(session, checkpointLocation).commitLog
+ val commitMetadata = commitLog.get(batchId) match {
+ case Some(commitMetadata) => commitMetadata
+ case None => throw StateDataSourceErrors.committedBatchUnavailable(checkpointLocation)
+ }
+
+ commitMetadata.stateUniqueIds.flatMap(_.get(operatorId))
+ }
+
+ // Modifies options due to external data. Returns modified options.
+ // If this is a join operator specifying a store name using state format v3,
+ // we need to modify the options.
+ private[state] def modifySourceOptions(
+ hadoopConf: Configuration, sourceOptions: StateSourceOptions): StateSourceOptions = {
+ // If a storeName is specified (e.g. right-keyToNumValues) and v3 is used,
+ // we are using join with virtual column families not diff stores. Therefore,
+ // options will be modified to set stateVarName to that storeName and storeName
+ // to default.
+ if (sourceOptions.storeName != StateStoreId.DEFAULT_STORE_NAME &&
+ StreamStreamJoinStateHelper.usesVirtualColumnFamilies(
+ hadoopConf, sourceOptions.stateCheckpointLocation.toString,
+ sourceOptions.operatorId)) {
+ sourceOptions.copy(
+ stateVarName = Some(sourceOptions.storeName),
+ storeName = StateStoreId.DEFAULT_STORE_NAME)
+ } else {
+ sourceOptions
+ }
+ }
}
// Case class to store information around the key state encoder, col family schema and
@@ -561,5 +659,28 @@ case class StateStoreReaderInfo(
keyStateEncoderSpecOpt: Option[KeyStateEncoderSpec],
stateStoreColFamilySchemaOpt: Option[StateStoreColFamilySchema],
transformWithStateVariableInfoOpt: Option[TransformWithStateVariableInfo],
- stateSchemaProviderOpt: Option[StateSchemaProvider]
+ stateSchemaProviderOpt: Option[StateSchemaProvider],
+ joinColFamilyOpt: Option[String] // Only used for join op with state format v3
)
+
+object StateDataSource {
+ private def getStateStoreMetadata(
+ stateSourceOptions: StateSourceOptions,
+ hadoopConf: Configuration): Array[StateMetadataTableEntry] = {
+ val allStateStoreMetadata = new StateMetadataPartitionReader(
+ stateSourceOptions.stateCheckpointLocation.getParent.toString,
+ new SerializableConfiguration(hadoopConf), stateSourceOptions.batchId).stateMetadata.toArray
+ val stateStoreMetadata = allStateStoreMetadata.filter { entry =>
+ entry.operatorId == stateSourceOptions.operatorId &&
+ entry.stateStoreName == stateSourceOptions.storeName
+ }
+ stateStoreMetadata
+ }
+
+ def getOldSchemaFilePaths(
+ stateSourceOptions: StateSourceOptions,
+ hadoopConf: Configuration): List[Path] = {
+ val metadata = getStateStoreMetadata(stateSourceOptions, hadoopConf)
+ metadata.headOption.map(_.stateSchemaFilePaths.map(new Path(_))).getOrElse(List.empty)
+ }
+}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala
index 4aa95ad42ec7f..ebef6e3dac552 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala
@@ -21,7 +21,8 @@ import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeRow}
import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory}
import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
-import org.apache.spark.sql.execution.streaming.{StateVariableType, TransformWithStateVariableInfo}
+import org.apache.spark.sql.execution.streaming.operators.stateful.join.SymmetricHashJoinStateManager
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.{StateVariableType, TransformWithStateVariableInfo}
import org.apache.spark.sql.execution.streaming.state._
import org.apache.spark.sql.execution.streaming.state.RecordType.{getRecordTypeAsString, RecordType}
import org.apache.spark.sql.types.{NullType, StructField, StructType}
@@ -42,7 +43,8 @@ class StatePartitionReaderFactory(
keyStateEncoderSpec: KeyStateEncoderSpec,
stateVariableInfoOpt: Option[TransformWithStateVariableInfo],
stateStoreColFamilySchemaOpt: Option[StateStoreColFamilySchema],
- stateSchemaProviderOpt: Option[StateSchemaProvider])
+ stateSchemaProviderOpt: Option[StateSchemaProvider],
+ joinColFamilyOpt: Option[String])
extends PartitionReaderFactory {
override def createReader(partition: InputPartition): PartitionReader[InternalRow] = {
@@ -50,11 +52,11 @@ class StatePartitionReaderFactory(
if (stateStoreInputPartition.sourceOptions.readChangeFeed) {
new StateStoreChangeDataPartitionReader(storeConf, hadoopConf,
stateStoreInputPartition, schema, keyStateEncoderSpec, stateVariableInfoOpt,
- stateStoreColFamilySchemaOpt, stateSchemaProviderOpt)
+ stateStoreColFamilySchemaOpt, stateSchemaProviderOpt, joinColFamilyOpt)
} else {
new StatePartitionReader(storeConf, hadoopConf,
stateStoreInputPartition, schema, keyStateEncoderSpec, stateVariableInfoOpt,
- stateStoreColFamilySchemaOpt, stateSchemaProviderOpt)
+ stateStoreColFamilySchemaOpt, stateSchemaProviderOpt, joinColFamilyOpt)
}
}
}
@@ -71,7 +73,8 @@ abstract class StatePartitionReaderBase(
keyStateEncoderSpec: KeyStateEncoderSpec,
stateVariableInfoOpt: Option[TransformWithStateVariableInfo],
stateStoreColFamilySchemaOpt: Option[StateStoreColFamilySchema],
- stateSchemaProviderOpt: Option[StateSchemaProvider])
+ stateSchemaProviderOpt: Option[StateSchemaProvider],
+ joinColFamilyOpt: Option[String])
extends PartitionReader[InternalRow] with Logging {
// Used primarily as a placeholder for the value schema in the context of
// state variables used within the transformWithState operator.
@@ -93,16 +96,19 @@ abstract class StatePartitionReaderBase(
schema, "value").asInstanceOf[StructType]
}
+ protected val getStoreUniqueId : Option[String] = {
+ SymmetricHashJoinStateManager.getStateStoreCheckpointId(
+ storeName = partition.sourceOptions.storeName,
+ partitionId = partition.partition,
+ stateStoreCkptIds = partition.sourceOptions.operatorStateUniqueIds)
+ }
+
protected lazy val provider: StateStoreProvider = {
val stateStoreId = StateStoreId(partition.sourceOptions.stateCheckpointLocation.toString,
partition.sourceOptions.operatorId, partition.partition, partition.sourceOptions.storeName)
val stateStoreProviderId = StateStoreProviderId(stateStoreId, partition.queryId)
- val useColFamilies = if (stateVariableInfoOpt.isDefined) {
- true
- } else {
- false
- }
+ val useColFamilies = stateVariableInfoOpt.isDefined || joinColFamilyOpt.isDefined
val useMultipleValuesPerKey = SchemaUtil.checkVariableType(stateVariableInfoOpt,
StateVariableType.ListState)
@@ -115,7 +121,9 @@ abstract class StatePartitionReaderBase(
val isInternal = partition.sourceOptions.readRegisteredTimers
if (useColFamilies) {
- val store = provider.getStore(partition.sourceOptions.batchId + 1)
+ val store = provider.getStore(
+ partition.sourceOptions.batchId + 1,
+ getStoreUniqueId)
require(stateStoreColFamilySchemaOpt.isDefined)
val stateStoreColFamilySchema = stateStoreColFamilySchemaOpt.get
require(stateStoreColFamilySchema.keyStateEncoderSpec.isDefined)
@@ -126,6 +134,7 @@ abstract class StatePartitionReaderBase(
stateStoreColFamilySchema.keyStateEncoderSpec.get,
useMultipleValuesPerKey = useMultipleValuesPerKey,
isInternal = isInternal)
+ store.abort()
}
provider
}
@@ -164,14 +173,19 @@ class StatePartitionReader(
keyStateEncoderSpec: KeyStateEncoderSpec,
stateVariableInfoOpt: Option[TransformWithStateVariableInfo],
stateStoreColFamilySchemaOpt: Option[StateStoreColFamilySchema],
- stateSchemaProviderOpt: Option[StateSchemaProvider])
+ stateSchemaProviderOpt: Option[StateSchemaProvider],
+ joinColFamilyOpt: Option[String])
extends StatePartitionReaderBase(storeConf, hadoopConf, partition, schema,
keyStateEncoderSpec, stateVariableInfoOpt, stateStoreColFamilySchemaOpt,
- stateSchemaProviderOpt) {
+ stateSchemaProviderOpt, joinColFamilyOpt) {
private lazy val store: ReadStateStore = {
partition.sourceOptions.fromSnapshotOptions match {
- case None => provider.getReadStore(partition.sourceOptions.batchId + 1)
+ case None =>
+ provider.getReadStore(
+ partition.sourceOptions.batchId + 1,
+ getStoreUniqueId
+ )
case Some(fromSnapshotOptions) =>
if (!provider.isInstanceOf[SupportsFineGrainedReplay]) {
@@ -186,17 +200,18 @@ class StatePartitionReader(
}
override lazy val iter: Iterator[InternalRow] = {
- val stateVarName = stateVariableInfoOpt
- .map(_.stateName).getOrElse(StateStore.DEFAULT_COL_FAMILY_NAME)
+ val colFamilyName = stateStoreColFamilySchemaOpt
+ .map(_.colFamilyName).getOrElse(
+ joinColFamilyOpt.getOrElse(StateStore.DEFAULT_COL_FAMILY_NAME))
if (stateVariableInfoOpt.isDefined) {
val stateVariableInfo = stateVariableInfoOpt.get
val stateVarType = stateVariableInfo.stateVariableType
- SchemaUtil.processStateEntries(stateVarType, stateVarName, store,
+ SchemaUtil.processStateEntries(stateVarType, colFamilyName, store,
keySchema, partition.partition, partition.sourceOptions)
} else {
store
- .iterator(stateVarName)
+ .iterator(colFamilyName)
.map { pair =>
SchemaUtil.unifyStateRowPair((pair.key, pair.value), partition.partition)
}
@@ -204,7 +219,7 @@ class StatePartitionReader(
}
override def close(): Unit = {
- store.abort()
+ store.release()
super.close()
}
}
@@ -221,10 +236,11 @@ class StateStoreChangeDataPartitionReader(
keyStateEncoderSpec: KeyStateEncoderSpec,
stateVariableInfoOpt: Option[TransformWithStateVariableInfo],
stateStoreColFamilySchemaOpt: Option[StateStoreColFamilySchema],
- stateSchemaProviderOpt: Option[StateSchemaProvider])
+ stateSchemaProviderOpt: Option[StateSchemaProvider],
+ joinColFamilyOpt: Option[String])
extends StatePartitionReaderBase(storeConf, hadoopConf, partition, schema,
keyStateEncoderSpec, stateVariableInfoOpt, stateStoreColFamilySchemaOpt,
- stateSchemaProviderOpt) {
+ stateSchemaProviderOpt, joinColFamilyOpt) {
private lazy val changeDataReader:
NextIterator[(RecordType.Value, UnsafeRow, UnsafeRow, Long)] = {
@@ -235,6 +251,8 @@ class StateStoreChangeDataPartitionReader(
val colFamilyNameOpt = if (stateVariableInfoOpt.isDefined) {
Some(stateVariableInfoOpt.get.stateName)
+ } else if (joinColFamilyOpt.isDefined) {
+ Some(joinColFamilyOpt.get)
} else {
None
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala
index 3b8dad7a18090..a1bd88f40740a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala
@@ -25,8 +25,8 @@ import org.apache.hadoop.fs.{Path, PathFilter}
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReaderFactory, Scan, ScanBuilder}
import org.apache.spark.sql.execution.datasources.v2.state.StateSourceOptions.JoinSideValues
-import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
-import org.apache.spark.sql.execution.streaming.TransformWithStateVariableInfo
+import org.apache.spark.sql.execution.streaming.operators.stateful.join.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateVariableInfo
import org.apache.spark.sql.execution.streaming.state.{KeyStateEncoderSpec, StateSchemaProvider, StateStoreColFamilySchema, StateStoreConf, StateStoreErrors}
import org.apache.spark.sql.types.StructType
import org.apache.spark.util.SerializableConfiguration
@@ -45,9 +45,11 @@ class StateScanBuilder(
keyStateEncoderSpec: KeyStateEncoderSpec,
stateVariableInfoOpt: Option[TransformWithStateVariableInfo],
stateStoreColFamilySchemaOpt: Option[StateStoreColFamilySchema],
- stateSchemaProviderOpt: Option[StateSchemaProvider]) extends ScanBuilder {
+ stateSchemaProviderOpt: Option[StateSchemaProvider],
+ joinColFamilyOpt: Option[String]) extends ScanBuilder {
override def build(): Scan = new StateScan(session, schema, sourceOptions, stateStoreConf,
- keyStateEncoderSpec, stateVariableInfoOpt, stateStoreColFamilySchemaOpt, stateSchemaProviderOpt)
+ keyStateEncoderSpec, stateVariableInfoOpt, stateStoreColFamilySchemaOpt, stateSchemaProviderOpt,
+ joinColFamilyOpt)
}
/** An implementation of [[InputPartition]] for State Store data source. */
@@ -65,7 +67,8 @@ class StateScan(
keyStateEncoderSpec: KeyStateEncoderSpec,
stateVariableInfoOpt: Option[TransformWithStateVariableInfo],
stateStoreColFamilySchemaOpt: Option[StateStoreColFamilySchema],
- stateSchemaProviderOpt: Option[StateSchemaProvider])
+ stateSchemaProviderOpt: Option[StateSchemaProvider],
+ joinColFamilyOpt: Option[String])
extends Scan with Batch {
// A Hadoop Configuration can be about 10 KB, which is pretty big, so broadcast it
@@ -120,24 +123,28 @@ class StateScan(
override def createReaderFactory(): PartitionReaderFactory = sourceOptions.joinSide match {
case JoinSideValues.left =>
val userFacingSchema = schema
+ val oldSchemaFilePaths = StateDataSource.getOldSchemaFilePaths(sourceOptions,
+ hadoopConfBroadcast.value.value)
val stateSchema = StreamStreamJoinStateHelper.readSchema(session,
sourceOptions.stateCheckpointLocation.toString, sourceOptions.operatorId, LeftSide,
- excludeAuxColumns = false)
+ oldSchemaFilePaths, excludeAuxColumns = false)
new StreamStreamJoinStatePartitionReaderFactory(stateStoreConf,
hadoopConfBroadcast.value, userFacingSchema, stateSchema)
case JoinSideValues.right =>
val userFacingSchema = schema
+ val oldSchemaFilePaths = StateDataSource.getOldSchemaFilePaths(sourceOptions,
+ hadoopConfBroadcast.value.value)
val stateSchema = StreamStreamJoinStateHelper.readSchema(session,
sourceOptions.stateCheckpointLocation.toString, sourceOptions.operatorId, RightSide,
- excludeAuxColumns = false)
+ oldSchemaFilePaths, excludeAuxColumns = false)
new StreamStreamJoinStatePartitionReaderFactory(stateStoreConf,
hadoopConfBroadcast.value, userFacingSchema, stateSchema)
case JoinSideValues.none =>
new StatePartitionReaderFactory(stateStoreConf, hadoopConfBroadcast.value, schema,
keyStateEncoderSpec, stateVariableInfoOpt, stateStoreColFamilySchemaOpt,
- stateSchemaProviderOpt)
+ stateSchemaProviderOpt, joinColFamilyOpt)
}
override def toBatch: Batch = this
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateTable.scala
index 71b18be7fdf58..dfa378e8a5972 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateTable.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateTable.scala
@@ -25,7 +25,7 @@ import org.apache.spark.sql.connector.catalog.{MetadataColumn, SupportsMetadataC
import org.apache.spark.sql.connector.read.ScanBuilder
import org.apache.spark.sql.execution.datasources.v2.state.StateSourceOptions.JoinSideValues
import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
-import org.apache.spark.sql.execution.streaming.TransformWithStateVariableInfo
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateVariableInfo
import org.apache.spark.sql.execution.streaming.state.{KeyStateEncoderSpec, StateSchemaProvider, StateStoreColFamilySchema, StateStoreConf}
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.util.CaseInsensitiveStringMap
@@ -44,7 +44,8 @@ class StateTable(
keyStateEncoderSpec: KeyStateEncoderSpec,
stateVariableInfoOpt: Option[TransformWithStateVariableInfo],
stateStoreColFamilySchemaOpt: Option[StateStoreColFamilySchema],
- stateSchemaProviderOpt: Option[StateSchemaProvider])
+ stateSchemaProviderOpt: Option[StateSchemaProvider],
+ joinColFamilyOpt: Option[String])
extends Table with SupportsRead with SupportsMetadataColumns {
import StateTable._
@@ -85,7 +86,8 @@ class StateTable(
override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder =
new StateScanBuilder(session, schema, sourceOptions, stateConf, keyStateEncoderSpec,
- stateVariableInfoOpt, stateStoreColFamilySchemaOpt, stateSchemaProviderOpt)
+ stateVariableInfoOpt, stateStoreColFamilySchemaOpt, stateSchemaProviderOpt,
+ joinColFamilyOpt)
override def properties(): util.Map[String, String] = Map.empty[String, String].asJava
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStateHelper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStateHelper.scala
index 1a04d24f0048d..5cb38022159cb 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStateHelper.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStateHelper.scala
@@ -18,9 +18,14 @@ package org.apache.spark.sql.execution.datasources.v2.state
import java.util.UUID
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.JoinSide
-import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreId, StateStoreProviderId, SymmetricHashJoinStateManager}
+import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager
+import org.apache.spark.sql.execution.streaming.operators.stateful.join.StreamingSymmetricHashJoinHelper.{JoinSide, LeftSide}
+import org.apache.spark.sql.execution.streaming.operators.stateful.join.SymmetricHashJoinStateManager
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreId, StateStoreProviderId}
import org.apache.spark.sql.types.{BooleanType, StructType}
/**
@@ -35,52 +40,92 @@ object StreamStreamJoinStateHelper {
stateCheckpointLocation: String,
operatorId: Int,
side: JoinSide,
+ oldSchemaFilePaths: List[Path],
excludeAuxColumns: Boolean = true): StructType = {
val (keySchema, valueSchema) = readKeyValueSchema(session, stateCheckpointLocation,
- operatorId, side, excludeAuxColumns)
+ operatorId, side, oldSchemaFilePaths, excludeAuxColumns)
new StructType()
.add("key", keySchema)
.add("value", valueSchema)
}
+ // Returns whether the checkpoint uses stateFormatVersion 3 which uses VCF for the join.
+ def usesVirtualColumnFamilies(
+ hadoopConf: Configuration,
+ stateCheckpointLocation: String,
+ operatorId: Int): Boolean = {
+ // If the schema exists for operatorId/partitionId/left-keyToNumValues, it is not
+ // stateFormatVersion 3.
+ val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
+ val storeId = new StateStoreId(stateCheckpointLocation, operatorId,
+ partitionId, SymmetricHashJoinStateManager.allStateStoreNames(LeftSide).toList.head)
+ val schemaFilePath = StateSchemaCompatibilityChecker.schemaFile(
+ storeId.storeCheckpointLocation())
+ val fm = CheckpointFileManager.create(schemaFilePath, hadoopConf)
+ !fm.exists(schemaFilePath)
+ }
+
def readKeyValueSchema(
session: SparkSession,
stateCheckpointLocation: String,
operatorId: Int,
side: JoinSide,
+ oldSchemaFilePaths: List[Path],
excludeAuxColumns: Boolean = true): (StructType, StructType) = {
+ val newHadoopConf = session.sessionState.newHadoopConf()
+ val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
// KeyToNumValuesType, KeyWithIndexToValueType
val storeNames = SymmetricHashJoinStateManager.allStateStoreNames(side).toList
- val partitionId = StateStore.PARTITION_ID_TO_CHECK_SCHEMA
- val storeIdForKeyToNumValues = new StateStoreId(stateCheckpointLocation, operatorId,
- partitionId, storeNames(0))
- val providerIdForKeyToNumValues = new StateStoreProviderId(storeIdForKeyToNumValues,
- UUID.randomUUID())
+ val (keySchema, valueSchema) =
+ if (!usesVirtualColumnFamilies(
+ newHadoopConf, stateCheckpointLocation, operatorId)) {
+ val storeIdForKeyToNumValues = new StateStoreId(stateCheckpointLocation, operatorId,
+ partitionId, storeNames(0))
+ val providerIdForKeyToNumValues = new StateStoreProviderId(storeIdForKeyToNumValues,
+ UUID.randomUUID())
- val storeIdForKeyWithIndexToValue = new StateStoreId(stateCheckpointLocation,
- operatorId, partitionId, storeNames(1))
- val providerIdForKeyWithIndexToValue = new StateStoreProviderId(storeIdForKeyWithIndexToValue,
- UUID.randomUUID())
+ val storeIdForKeyWithIndexToValue = new StateStoreId(stateCheckpointLocation,
+ operatorId, partitionId, storeNames(1))
+ val providerIdForKeyWithIndexToValue = new StateStoreProviderId(
+ storeIdForKeyWithIndexToValue, UUID.randomUUID())
- val newHadoopConf = session.sessionState.newHadoopConf()
+ // read the key schema from the keyToNumValues store for the join keys
+ val manager = new StateSchemaCompatibilityChecker(
+ providerIdForKeyToNumValues, newHadoopConf, oldSchemaFilePaths)
+ val kSchema = manager.readSchemaFile().head.keySchema
+
+ // read the value schema from the keyWithIndexToValue store for the values
+ val manager2 = new StateSchemaCompatibilityChecker(providerIdForKeyWithIndexToValue,
+ newHadoopConf, oldSchemaFilePaths)
+ val vSchema = manager2.readSchemaFile().head.valueSchema
+
+ (kSchema, vSchema)
+ } else {
+ val storeId = new StateStoreId(stateCheckpointLocation, operatorId,
+ partitionId, StateStoreId.DEFAULT_STORE_NAME)
+ val providerId = new StateStoreProviderId(storeId, UUID.randomUUID())
+
+ val manager = new StateSchemaCompatibilityChecker(
+ providerId, newHadoopConf, oldSchemaFilePaths)
+ val kSchema = manager.readSchemaFile().find { schema =>
+ schema.colFamilyName == storeNames(0)
+ }.map(_.keySchema).get
- // read the key schema from the keyToNumValues store for the join keys
- val manager = new StateSchemaCompatibilityChecker(providerIdForKeyToNumValues, newHadoopConf)
- val keySchema = manager.readSchemaFile().head.keySchema
+ val vSchema = manager.readSchemaFile().find { schema =>
+ schema.colFamilyName == storeNames(1)
+ }.map(_.valueSchema).get
- // read the value schema from the keyWithIndexToValue store for the values
- val manager2 = new StateSchemaCompatibilityChecker(providerIdForKeyWithIndexToValue,
- newHadoopConf)
- val valueSchema = manager2.readSchemaFile().head.valueSchema
+ (kSchema, vSchema)
+ }
val maybeMatchedColumn = valueSchema.last
if (excludeAuxColumns
- && maybeMatchedColumn.name == "matched"
- && maybeMatchedColumn.dataType == BooleanType) {
+ && maybeMatchedColumn.name == "matched"
+ && maybeMatchedColumn.dataType == BooleanType) {
// remove internal column `matched` for format version 2
(keySchema, StructType(valueSchema.dropRight(1)))
} else {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStatePartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStatePartitionReader.scala
index e1d61de77380f..0f8a3b3b609f8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStatePartitionReader.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStatePartitionReader.scala
@@ -23,9 +23,10 @@ import org.apache.spark.sql.catalyst.types.DataTypeUtils
import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory}
import org.apache.spark.sql.execution.datasources.v2.state.StateSourceOptions.JoinSideValues
import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
-import org.apache.spark.sql.execution.streaming.StatefulOperatorStateInfo
-import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{JoinSide, LeftSide, RightSide}
-import org.apache.spark.sql.execution.streaming.state.{JoinStateManagerStoreGenerator, StateStoreConf, SymmetricHashJoinStateManager}
+import org.apache.spark.sql.execution.streaming.operators.stateful.StatefulOperatorStateInfo
+import org.apache.spark.sql.execution.streaming.operators.stateful.join.{JoinStateManagerStoreGenerator, SymmetricHashJoinStateManager}
+import org.apache.spark.sql.execution.streaming.operators.stateful.join.StreamingSymmetricHashJoinHelper.{JoinSide, LeftSide, RightSide}
+import org.apache.spark.sql.execution.streaming.state.StateStoreConf
import org.apache.spark.sql.types.{BooleanType, StructType}
import org.apache.spark.util.SerializableConfiguration
@@ -70,6 +71,28 @@ class StreamStreamJoinStatePartitionReader(
throw StateDataSourceErrors.internalError("Unexpected join side for stream-stream read!")
}
+ private val usesVirtualColumnFamilies = StreamStreamJoinStateHelper.usesVirtualColumnFamilies(
+ hadoopConf.value,
+ partition.sourceOptions.stateCheckpointLocation.toString,
+ partition.sourceOptions.operatorId)
+
+ private val stateStoreCheckpointIds = SymmetricHashJoinStateManager.getStateStoreCheckpointIds(
+ partition.partition,
+ partition.sourceOptions.operatorStateUniqueIds,
+ usesVirtualColumnFamilies)
+
+ private val keyToNumValuesStateStoreCkptId = if (joinSide == LeftSide) {
+ stateStoreCheckpointIds.left.keyToNumValues
+ } else {
+ stateStoreCheckpointIds.right.keyToNumValues
+ }
+
+ private val keyWithIndexToValueStateStoreCkptId = if (joinSide == LeftSide) {
+ stateStoreCheckpointIds.left.keyWithIndexToValue
+ } else {
+ stateStoreCheckpointIds.right.keyWithIndexToValue
+ }
+
/*
* This is to handle the difference of schema across state format versions. The major difference
* is whether we have added new field(s) in addition to the fields from input schema.
@@ -80,8 +103,15 @@ class StreamStreamJoinStatePartitionReader(
private val (inputAttributes, formatVersion) = {
val maybeMatchedColumn = valueSchema.last
val (fields, version) = {
+ // If there is a matched column, version is either 2 or 3. We need to drop the matched
+ // column from the value schema to get the actual fields.
if (maybeMatchedColumn.name == "matched" && maybeMatchedColumn.dataType == BooleanType) {
- (valueSchema.dropRight(1), 2)
+ // If checkpoint is using one store and virtual column families, version is 3
+ if (usesVirtualColumnFamilies) {
+ (valueSchema.dropRight(1), 3)
+ } else {
+ (valueSchema.dropRight(1), 2)
+ }
} else {
(valueSchema, 1)
}
@@ -119,8 +149,8 @@ class StreamStreamJoinStatePartitionReader(
storeConf = storeConf,
hadoopConf = hadoopConf.value,
partitionId = partition.partition,
- keyToNumValuesStateStoreCkptId = None,
- keyWithIndexToValueStateStoreCkptId = None,
+ keyToNumValuesStateStoreCkptId = keyToNumValuesStateStoreCkptId,
+ keyWithIndexToValueStateStoreCkptId = keyWithIndexToValueStateStoreCkptId,
formatVersion,
skippedNullValueCount = None,
useStateStoreCoordinator = false,
@@ -137,7 +167,7 @@ class StreamStreamJoinStatePartitionReader(
inputAttributes)
joinStateManager.iterator.map { pair =>
- if (formatVersion == 2) {
+ if (formatVersion >= 2) {
val row = valueWithMatchedRowGenerator(pair.value)
row.setBoolean(indexOrdinalInValueWithMatchedRow, pair.matched)
unifyStateRowPair(pair.key, row)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/metadata/StateMetadataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/metadata/StateMetadataSource.scala
index c5ccff9228d6e..31e6ac30a5984 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/metadata/StateMetadataSource.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/metadata/StateMetadataSource.scala
@@ -23,7 +23,7 @@ import scala.jdk.CollectionConverters._
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{Path, PathFilter}
-import org.apache.spark.internal.{Logging, LogKeys, MDC}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.GenericInternalRow
@@ -32,7 +32,7 @@ import org.apache.spark.sql.connector.expressions.Transform
import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReader, PartitionReaderFactory, Scan, ScanBuilder}
import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceErrors
import org.apache.spark.sql.execution.datasources.v2.state.StateSourceOptions.PATH
-import org.apache.spark.sql.execution.streaming.CheckpointFileManager
+import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager
import org.apache.spark.sql.execution.streaming.state.{OperatorStateMetadata, OperatorStateMetadataReader, OperatorStateMetadataUtils, OperatorStateMetadataV1, OperatorStateMetadataV2}
import org.apache.spark.sql.sources.DataSourceRegister
import org.apache.spark.sql.types.{DataType, IntegerType, LongType, StringType, StructType}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/utils/SchemaUtil.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/utils/SchemaUtil.scala
index 84eab3356c204..52df016791d48 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/utils/SchemaUtil.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/utils/SchemaUtil.scala
@@ -25,8 +25,8 @@ import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeRow}
import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData}
import org.apache.spark.sql.execution.datasources.v2.state.{StateDataSourceErrors, StateSourceOptions}
-import org.apache.spark.sql.execution.streaming.{StateVariableType, TransformWithStateVariableInfo}
-import org.apache.spark.sql.execution.streaming.StateVariableType._
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.{StateVariableType, TransformWithStateVariableInfo}
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.StateVariableType._
import org.apache.spark.sql.execution.streaming.state.{ReadStateStore, StateStoreColFamilySchema, UnsafeRowPair}
import org.apache.spark.sql.types.{ArrayType, DataType, IntegerType, LongType, MapType, StringType, StructType}
import org.apache.spark.util.ArrayImplicits._
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlDataSource.scala
index 23bca35725397..9dbca57e2ae95 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlDataSource.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlDataSource.scala
@@ -22,7 +22,6 @@ import java.nio.charset.{Charset, StandardCharsets}
import scala.util.control.NonFatal
-import org.apache.commons.lang3.exception.ExceptionUtils
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileStatus, Path}
import org.apache.hadoop.hdfs.BlockMissingException
@@ -37,7 +36,7 @@ import org.apache.spark.rdd.{BinaryFileRDD, RDD}
import org.apache.spark.sql.{Dataset, Encoders, SparkSession}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.util.FailureSafeParser
-import org.apache.spark.sql.catalyst.xml.{StaxXmlParser, XmlInferSchema, XmlOptions}
+import org.apache.spark.sql.catalyst.xml.{StaxXmlParser, StaxXMLRecordReader, XmlInferSchema, XmlOptions}
import org.apache.spark.sql.classic.ClassicConversions.castToImpl
import org.apache.spark.sql.execution.SQLExecution
import org.apache.spark.sql.execution.datasources._
@@ -175,15 +174,26 @@ object MultiLineXmlDataSource extends XmlDataSource {
file: PartitionedFile,
parser: StaxXmlParser,
requiredSchema: StructType): Iterator[InternalRow] = {
- parser.parseStream(
- CodecStreams.createInputStreamWithCloseResource(conf, file.toPath),
- requiredSchema)
+ if (parser.options.useLegacyXMLParser) {
+ parser.parseStream(
+ CodecStreams.createInputStreamWithCloseResource(conf, file.toPath),
+ requiredSchema)
+ } else {
+ parser.parseStreamOptimized(
+ () => CodecStreams.createInputStreamWithCloseResource(conf, file.toPath),
+ requiredSchema)
+ }
}
override def infer(
sparkSession: SparkSession,
inputPaths: Seq[FileStatus],
parsedOptions: XmlOptions): StructType = {
+
+ if (!parsedOptions.useLegacyXMLParser) {
+ return inferOptimized(sparkSession, inputPaths, parsedOptions)
+ }
+
val xml = createBaseRdd(sparkSession, inputPaths, parsedOptions)
val tokenRDD: RDD[String] =
@@ -199,7 +209,7 @@ object MultiLineXmlDataSource extends XmlDataSource {
logWarning("Skipped missing file", e)
Iterator.empty[String]
case NonFatal(e) =>
- ExceptionUtils.getRootCause(e) match {
+ Utils.getRootCause(e) match {
case e @ (_ : AccessControlException | _ : BlockMissingException) => throw e
case _: RuntimeException | _: IOException if parsedOptions.ignoreCorruptFiles =>
logWarning("Skipped the rest of the content in the corrupted file", e)
@@ -216,6 +226,31 @@ object MultiLineXmlDataSource extends XmlDataSource {
}
}
+ private def inferOptimized(
+ sparkSession: SparkSession,
+ inputPaths: Seq[FileStatus],
+ parsedOptions: XmlOptions): StructType = {
+
+ val xml = createBaseRdd(sparkSession, inputPaths, parsedOptions)
+
+ val xmlParserRdd: RDD[StaxXMLRecordReader] =
+ xml.flatMap { portableDataStream =>
+ val inputStream = () =>
+ CodecStreams.createInputStreamWithCloseResource(
+ portableDataStream.getConfiguration,
+ new Path(portableDataStream.getPath())
+ )
+ StaxXmlParser.convertStream(inputStream, parsedOptions)(identity)
+ }
+
+ SQLExecution.withSQLConfPropagated(sparkSession) {
+ val schema =
+ new XmlInferSchema(parsedOptions, sparkSession.sessionState.conf.caseSensitiveAnalysis)
+ .inferFromReaders(xmlParserRdd)
+ schema
+ }
+ }
+
private def createBaseRdd(
sparkSession: SparkSession,
inputPaths: Seq[FileStatus],
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlFileFormat.scala
index 5072a87af4df7..06151c5fa4d64 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlFileFormat.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/xml/XmlFileFormat.scala
@@ -39,13 +39,11 @@ case class XmlFileFormat() extends TextBasedFileFormat with DataSourceRegister {
override def shortName(): String = "xml"
- def getXmlOptions(
+ private def getXmlOptions(
sparkSession: SparkSession,
parameters: Map[String, String]): XmlOptions = {
- new XmlOptions(parameters,
- sparkSession.sessionState.conf.sessionLocalTimeZone,
- sparkSession.sessionState.conf.columnNameOfCorruptRecord,
- true)
+ val conf = getSqlConf(sparkSession)
+ new XmlOptions(parameters, conf.sessionLocalTimeZone, conf.columnNameOfCorruptRecord, true)
}
override def isSplitable(
@@ -53,8 +51,7 @@ case class XmlFileFormat() extends TextBasedFileFormat with DataSourceRegister {
options: Map[String, String],
path: Path): Boolean = {
val xmlOptions = getXmlOptions(sparkSession, options)
- val xmlDataSource = XmlDataSource(xmlOptions)
- xmlDataSource.isSplitable && super.isSplitable(sparkSession, options, path)
+ XmlDataSource(xmlOptions).isSplitable && super.isSplitable(sparkSession, options, path)
}
override def inferSchema(
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala
index 0b97c8ebab815..b408f2823e16a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala
@@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.plans.physical.Partitioning
import org.apache.spark.sql.catalyst.trees.TreeNodeRef
import org.apache.spark.sql.catalyst.util.StringConcat
import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, QueryStageExec}
-import org.apache.spark.sql.execution.streaming.{StreamExecution, StreamingQueryWrapper}
+import org.apache.spark.sql.execution.streaming.runtime.{StreamExecution, StreamingQueryWrapper}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.streaming.StreamingQuery
import org.apache.spark.sql.vectorized.ColumnarBatch
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PlanDynamicPruningFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PlanDynamicPruningFilters.scala
index 059729d86bfaf..fbd341b6e7b87 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PlanDynamicPruningFilters.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PlanDynamicPruningFilters.scala
@@ -17,15 +17,14 @@
package org.apache.spark.sql.execution.dynamicpruning
-import org.apache.spark.sql.catalyst.expressions
-import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeSeq, BindReferences, DynamicPruningExpression, DynamicPruningSubquery, Expression, ListQuery, Literal}
+import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeSeq, BindReferences, DynamicPruningExpression, DynamicPruningSubquery, Expression, Literal}
import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight}
import org.apache.spark.sql.catalyst.plans.logical.Aggregate
import org.apache.spark.sql.catalyst.plans.physical.BroadcastMode
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.catalyst.trees.TreePattern.DYNAMIC_PRUNING_SUBQUERY
import org.apache.spark.sql.classic.SparkSession
-import org.apache.spark.sql.execution.{InSubqueryExec, QueryExecution, SparkPlan, SubqueryBroadcastExec}
+import org.apache.spark.sql.execution.{InSubqueryExec, QueryExecution, SparkPlan, SubqueryBroadcastExec, SubqueryExec}
import org.apache.spark.sql.execution.exchange.BroadcastExchangeExec
import org.apache.spark.sql.execution.joins._
import org.apache.spark.sql.internal.SQLConf
@@ -56,6 +55,7 @@ case class PlanDynamicPruningFilters(sparkSession: SparkSession) extends Rule[Sp
case DynamicPruningSubquery(
value, buildPlan, buildKeys, broadcastKeyIndices, onlyInBroadcast, exprId, _) =>
val sparkPlan = QueryExecution.createSparkPlan(sparkSession.sessionState.planner, buildPlan)
+ val name = s"dynamicpruning#${exprId.id}"
// Using `sparkPlan` is a little hacky as it is based on the assumption that this rule is
// the first to be applied (apart from `InsertAdaptiveSparkPlan`).
val canReuseExchange = conf.exchangeReuseEnabled && buildKeys.nonEmpty &&
@@ -72,7 +72,6 @@ case class PlanDynamicPruningFilters(sparkSession: SparkSession) extends Rule[Sp
val mode = broadcastMode(buildKeys, executedPlan.output)
// plan a broadcast exchange of the build side of the join
val exchange = BroadcastExchangeExec(mode, executedPlan)
- val name = s"dynamicpruning#${exprId.id}"
// place the broadcast adaptor for reusing the broadcast results on the probe side
val broadcastValues =
SubqueryBroadcastExec(name, broadcastKeyIndices, buildKeys, exchange)
@@ -85,8 +84,9 @@ case class PlanDynamicPruningFilters(sparkSession: SparkSession) extends Rule[Sp
val aliases = broadcastKeyIndices.map(idx =>
Alias(buildKeys(idx), buildKeys(idx).toString)())
val aggregate = Aggregate(aliases, aliases, buildPlan)
- DynamicPruningExpression(expressions.InSubquery(
- Seq(value), ListQuery(aggregate, numCols = aggregate.output.length)))
+ val sparkPlan = QueryExecution.prepareExecutedPlan(sparkSession, aggregate)
+ val values = SubqueryExec(name, sparkPlan)
+ DynamicPruningExpression(InSubqueryExec(value, values, exprId))
}
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala
index c70ee637a2489..8c695f4f3958d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala
@@ -26,7 +26,6 @@ import scala.util.control.NonFatal
import org.apache.spark.{broadcast, SparkException}
import org.apache.spark.internal.LogKeys._
-import org.apache.spark.internal.MDC
import org.apache.spark.rdd.{RDD, RDDOperationScope}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.UnsafeRow
@@ -145,6 +144,12 @@ case class BroadcastExchangeExec(
Statistics(dataSize, Some(rowCount))
}
+ override def resetMetrics(): Unit = {
+ // no-op
+ // BroadcastExchangeExec after materialized won't be materialized again, so we should not
+ // reset the metrics. Otherwise, we will lose the metrics collected in the broadcast job.
+ }
+
@transient
private lazy val promise = Promise[broadcast.Broadcast[Any]]()
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
index de5c3aaa4fe4d..a0fc4b65fdbf3 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.exchange
import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer
-import org.apache.spark.internal.{LogKeys, MDC}
+import org.apache.spark.internal.{LogKeys}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans._
@@ -374,6 +374,26 @@ case class EnsureRequirements(
}
}
+ /**
+ * Whether partial clustering can be applied to a given child query plan. This is true if the plan
+ * consists only of a sequence of unary nodes where each node does not use the scan's key-grouped
+ * partitioning to satisfy its required distribution. Otherwise, partially clustering could be
+ * applied to a key-grouped partitioning unrelated to this join.
+ */
+ private def canApplyPartialClusteredDistribution(plan: SparkPlan): Boolean = {
+ !plan.exists {
+ // Unary nodes are safe as long as they don't have a required distribution (for example, a
+ // project or filter). If they have a required distribution, then we should assume that this
+ // plan can't be partially clustered (since the key-grouped partitioning may be needed to
+ // satisfy this distribution unrelated to this JOIN).
+ case u if u.children.length == 1 =>
+ u.requiredChildDistribution.head != UnspecifiedDistribution
+ // Only allow a non-unary node if it's a leaf node - key-grouped partitionings other binary
+ // nodes (like another JOIN) aren't safe to partially cluster.
+ case other => other.children.nonEmpty
+ }
+ }
+
/**
* Checks whether two children, `left` and `right`, of a join operator have compatible
* `KeyGroupedPartitioning`, and can benefit from storage-partitioned join.
@@ -490,9 +510,16 @@ case class EnsureRequirements(
// whether partially clustered distribution can be applied. For instance, the
// optimization cannot be applied to a left outer join, where the left hand
// side is chosen as the side to replicate partitions according to stats.
+ // Similarly, the partially clustered distribution cannot be applied if the
+ // partially clustered side must use the scan's key-grouped partitioning to
+ // satisfy some unrelated required distribution in its plan (for example, for an aggregate
+ // or window function), as this will give incorrect results (for example, duplicate
+ // row_number() values).
// Otherwise, query result could be incorrect.
- val canReplicateLeft = canReplicateLeftSide(joinType)
- val canReplicateRight = canReplicateRightSide(joinType)
+ val canReplicateLeft = canReplicateLeftSide(joinType) &&
+ canApplyPartialClusteredDistribution(right)
+ val canReplicateRight = canReplicateRightSide(joinType) &&
+ canApplyPartialClusteredDistribution(left)
if (!canReplicateLeft && !canReplicateRight) {
logInfo(log"Skipping partially clustered distribution as it cannot be applied for " +
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala
index 8a996bce251c7..8065decb0dffe 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala
@@ -36,11 +36,13 @@ class UnsafeCartesianRDD(
left : RDD[UnsafeRow],
right : RDD[UnsafeRow],
inMemoryBufferThreshold: Int,
- spillThreshold: Int)
+ spillThreshold: Int,
+ spillSizeThreshold: Long)
extends CartesianRDD[UnsafeRow, UnsafeRow](left.sparkContext, left, right) {
override def compute(split: Partition, context: TaskContext): Iterator[(UnsafeRow, UnsafeRow)] = {
- val rowArray = new ExternalAppendOnlyUnsafeRowArray(inMemoryBufferThreshold, spillThreshold)
+ val rowArray = new ExternalAppendOnlyUnsafeRowArray(inMemoryBufferThreshold, spillThreshold,
+ spillSizeThreshold)
val partition = split.asInstanceOf[CartesianPartition]
rdd2.iterator(partition.s2, context).foreach(rowArray.add)
@@ -81,7 +83,8 @@ case class CartesianProductExec(
leftResults,
rightResults,
conf.cartesianProductExecBufferInMemoryThreshold,
- conf.cartesianProductExecBufferSpillThreshold)
+ conf.cartesianProductExecBufferSpillThreshold,
+ conf.cartesianProductExecBufferSizeSpillThreshold)
pair.mapPartitionsWithIndexInternal { (index, iter) =>
val joiner = GenerateUnsafeRowJoiner.create(left.schema, right.schema)
val filtered = if (condition.isDefined) {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala
index 974f6f9e50c2e..97ca74aee30c0 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala
@@ -65,14 +65,35 @@ case class ShuffledHashJoinExec(
case _ => super.outputOrdering
}
+ private def validCondForIgnoreDupKey(cond: Expression): Boolean = {
+ // to ignore duplicate keys on the build side, the join condition must
+ // have the following properties:
+ // 1) a subtree that is a semantic match to a build-side key, and/or
+ // 2) outside any subtree that is a semantic match to a build-side key,
+ // all attributes should be from the stream-side.
+ val buildKeysSet = ExpressionSet(buildKeys)
+ val streamedOutputAttrs = AttributeSet(streamedOutput)
+
+ def validCond(cond: Expression): Boolean = {
+ cond match {
+ // don't bother traversing any subtree that has a semantic match to a build key
+ case e: Expression if buildKeysSet.contains(e) => true
+ // all attributes (outside any subtree that matches a build key) should be
+ // from the stream side
+ case a: Attribute if !streamedOutputAttrs.contains(a) => false
+ case e: Expression =>
+ e.children.forall(validCond(_))
+ case _ => true
+ }
+ }
+
+ validCond(cond)
+ }
+
// Exposed for testing
@transient lazy val ignoreDuplicatedKey = joinType match {
case LeftExistence(_) =>
- // For building hash relation, ignore duplicated rows with same join keys if:
- // 1. Join condition is empty, or
- // 2. Join condition only references streamed attributes and build join keys.
- val streamedOutputAndBuildKeys = AttributeSet(streamedOutput ++ buildKeys)
- condition.forall(_.references.subsetOf(streamedOutputAndBuildKeys))
+ condition.forall(validCondForIgnoreDupKey(_))
case _ => false
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledJoin.scala
index 60e5a7769a503..f363156c81e54 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledJoin.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledJoin.scala
@@ -37,7 +37,7 @@ trait ShuffledJoin extends JoinCodegenSupport {
override def requiredChildDistribution: Seq[Distribution] = {
if (isSkewJoin) {
// We re-arrange the shuffle partitions to deal with skew join, and the new children
- // partitioning doesn't satisfy `HashClusteredDistribution`.
+ // partitioning doesn't satisfy `ClusteredDistribution`.
UnspecifiedDistribution :: UnspecifiedDistribution :: Nil
} else {
ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinEvaluatorFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinEvaluatorFactory.scala
index 57ca135407d4a..b4e52ba050b8d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinEvaluatorFactory.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinEvaluatorFactory.scala
@@ -34,6 +34,7 @@ class SortMergeJoinEvaluatorFactory(
output: Seq[Attribute],
inMemoryThreshold: Int,
spillThreshold: Int,
+ spillSizeThreshold: Long,
numOutputRows: SQLMetric,
spillSize: SQLMetric,
onlyBufferFirstMatchedRow: Boolean)
@@ -85,6 +86,7 @@ class SortMergeJoinEvaluatorFactory(
RowIterator.fromScala(rightIter),
inMemoryThreshold,
spillThreshold,
+ spillSizeThreshold,
spillSize,
cleanupResources)
private[this] val joinRow = new JoinedRow
@@ -130,6 +132,7 @@ class SortMergeJoinEvaluatorFactory(
bufferedIter = RowIterator.fromScala(rightIter),
inMemoryThreshold,
spillThreshold,
+ spillSizeThreshold,
spillSize,
cleanupResources)
val rightNullRow = new GenericInternalRow(right.output.length)
@@ -149,6 +152,7 @@ class SortMergeJoinEvaluatorFactory(
bufferedIter = RowIterator.fromScala(leftIter),
inMemoryThreshold,
spillThreshold,
+ spillSizeThreshold,
spillSize,
cleanupResources)
val leftNullRow = new GenericInternalRow(left.output.length)
@@ -185,6 +189,7 @@ class SortMergeJoinEvaluatorFactory(
RowIterator.fromScala(rightIter),
inMemoryThreshold,
spillThreshold,
+ spillSizeThreshold,
spillSize,
cleanupResources,
onlyBufferFirstMatchedRow)
@@ -222,6 +227,7 @@ class SortMergeJoinEvaluatorFactory(
RowIterator.fromScala(rightIter),
inMemoryThreshold,
spillThreshold,
+ spillSizeThreshold,
spillSize,
cleanupResources,
onlyBufferFirstMatchedRow)
@@ -266,6 +272,7 @@ class SortMergeJoinEvaluatorFactory(
RowIterator.fromScala(rightIter),
inMemoryThreshold,
spillThreshold,
+ spillSizeThreshold,
spillSize,
cleanupResources,
onlyBufferFirstMatchedRow)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala
index 8d49b1558d687..39387ebbb7ee3 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala
@@ -103,6 +103,10 @@ case class SortMergeJoinExec(
conf.sortMergeJoinExecBufferSpillThreshold
}
+ private def getSpillSizeThreshold: Long = {
+ conf.sortMergeJoinExecBufferSpillSizeThreshold
+ }
+
// Flag to only buffer first matched row, to avoid buffering unnecessary rows.
private val onlyBufferFirstMatchedRow = (joinType, condition) match {
case (LeftExistence(_), None) => true
@@ -121,6 +125,7 @@ case class SortMergeJoinExec(
val numOutputRows = longMetric("numOutputRows")
val spillSize = longMetric("spillSize")
val spillThreshold = getSpillThreshold
+ val spillSizeThreshold = getSpillSizeThreshold
val inMemoryThreshold = getInMemoryThreshold
val evaluatorFactory = new SortMergeJoinEvaluatorFactory(
leftKeys,
@@ -132,6 +137,7 @@ case class SortMergeJoinExec(
output,
inMemoryThreshold,
spillThreshold,
+ spillSizeThreshold,
numOutputRows,
spillSize,
onlyBufferFirstMatchedRow
@@ -222,11 +228,13 @@ case class SortMergeJoinExec(
val clsName = classOf[ExternalAppendOnlyUnsafeRowArray].getName
val spillThreshold = getSpillThreshold
+ val spillSizeThreshold = getSpillSizeThreshold
val inMemoryThreshold = getInMemoryThreshold
// Inline mutable state since not many join operations in a task
val matches = ctx.addMutableState(clsName, "matches",
- v => s"$v = new $clsName($inMemoryThreshold, $spillThreshold);", forceInline = true)
+ v => s"$v = new $clsName($inMemoryThreshold, $spillThreshold, ${spillSizeThreshold}L);",
+ forceInline = true)
// Copy the streamed keys as class members so they could be used in next function call.
val matchedKeyVars = copyKeys(ctx, streamedKeyVars)
@@ -1044,6 +1052,7 @@ case class SortMergeJoinExec(
* @param inMemoryThreshold Threshold for number of rows guaranteed to be held in memory by
* internal buffer
* @param spillThreshold Threshold for number of rows to be spilled by internal buffer
+ * @param spillSizeThreshold Threshold for size of rows to be spilled by internal buffer
* @param eagerCleanupResources the eager cleanup function to be invoked when no join row found
* @param onlyBufferFirstMatch [[bufferMatchingRows]] should buffer only the first matching row
*/
@@ -1055,6 +1064,7 @@ private[joins] class SortMergeJoinScanner(
bufferedIter: RowIterator,
inMemoryThreshold: Int,
spillThreshold: Int,
+ spillSizeThreshold: Long,
spillSize: SQLMetric,
eagerCleanupResources: () => Unit,
onlyBufferFirstMatch: Boolean = false) {
@@ -1069,7 +1079,7 @@ private[joins] class SortMergeJoinScanner(
private[this] var matchJoinKey: InternalRow = _
/** Buffered rows from the buffered side of the join. This is empty if there are no matches. */
private[this] val bufferedMatches: ExternalAppendOnlyUnsafeRowArray =
- new ExternalAppendOnlyUnsafeRowArray(inMemoryThreshold, spillThreshold)
+ new ExternalAppendOnlyUnsafeRowArray(inMemoryThreshold, spillThreshold, spillSizeThreshold)
// At the end of the task, update the task's spill size for buffered side.
TaskContext.get().addTaskCompletionListener[Unit](_ => {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/StoragePartitionJoinParams.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/StoragePartitionJoinParams.scala
new file mode 100644
index 0000000000000..a28eafc5cae5b
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/StoragePartitionJoinParams.scala
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.joins
+
+import java.util.Objects
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.connector.catalog.functions.Reducer
+
+case class StoragePartitionJoinParams(
+ keyGroupedPartitioning: Option[Seq[Expression]] = None,
+ joinKeyPositions: Option[Seq[Int]] = None,
+ commonPartitionValues: Option[Seq[(InternalRow, Int)]] = None,
+ reducers: Option[Seq[Option[Reducer[_, _]]]] = None,
+ applyPartialClustering: Boolean = false,
+ replicatePartitions: Boolean = false) {
+ override def equals(other: Any): Boolean = other match {
+ case other: StoragePartitionJoinParams =>
+ this.commonPartitionValues == other.commonPartitionValues &&
+ this.replicatePartitions == other.replicatePartitions &&
+ this.applyPartialClustering == other.applyPartialClustering &&
+ this.joinKeyPositions == other.joinKeyPositions
+ case _ =>
+ false
+ }
+
+ override def hashCode(): Int = Objects.hash(
+ joinKeyPositions: Option[Seq[Int]],
+ commonPartitionValues: Option[Seq[(InternalRow, Int)]],
+ applyPartialClustering: java.lang.Boolean,
+ replicatePartitions: java.lang.Boolean)
+}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala
index 065c8db7ac6f9..13f4d7926bea8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala
@@ -221,4 +221,19 @@ object SQLMetrics {
SparkListenerDriverAccumUpdates(executionId.toLong, metrics.map(m => m.id -> m.value)))
}
}
+
+ /**
+ * Measures the time taken by the function `f` in nanoseconds and adds it to the provided metric.
+ *
+ * @param metric SQLMetric to record the time taken.
+ * @param f Function/Codeblock to execute and measure.
+ * @return The result of the function `f`.
+ */
+ def withTimingNs[T](metric: SQLMetric)(f: => T): T = {
+ val startTime = System.nanoTime()
+ val result = f
+ val endTime = System.nanoTime()
+ metric.add(endTime - startTime)
+ result
+ }
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala
index ca7836992aacb..015872d98c372 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala
@@ -39,7 +39,7 @@ import org.apache.spark.sql.catalyst.types.DataTypeUtils
import org.apache.spark.sql.errors.QueryExecutionErrors
import org.apache.spark.sql.execution.python.BatchIterator
import org.apache.spark.sql.execution.r.ArrowRRunner
-import org.apache.spark.sql.execution.streaming.GroupStateImpl
+import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.GroupStateImpl
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.streaming.GroupStateTimeout
import org.apache.spark.sql.types._
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/AggregateInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowAggregatePythonExec.scala
similarity index 84%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/python/AggregateInPandasExec.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowAggregatePythonExec.scala
index 26871b68dde80..d6cc350e485a8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/AggregateInPandasExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowAggregatePythonExec.scala
@@ -21,7 +21,7 @@ import java.io.File
import scala.collection.mutable.ArrayBuffer
-import org.apache.spark.{JobArtifactSet, SparkEnv, TaskContext}
+import org.apache.spark.{JobArtifactSet, SparkEnv, SparkException, TaskContext}
import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
@@ -35,19 +35,28 @@ import org.apache.spark.sql.types.{DataType, StructField, StructType}
import org.apache.spark.util.Utils
/**
- * Physical node for aggregation with group aggregate Pandas UDF.
+ * Physical node for aggregation with group aggregate vectorized UDF.
+ * Following eval types are supported:
+ *
+ *
+ *
SQL_GROUPED_AGG_ARROW_UDF for Arrow UDF
+ *
SQL_GROUPED_AGG_PANDAS_UDF for Pandas UDF
+ *
*
* This plan works by sending the necessary (projected) input grouped data as Arrow record batches
* to the python worker, the python worker invokes the UDF and sends the results to the executor,
* finally the executor evaluates any post-aggregation expressions and join the result with the
* grouped key.
*/
-case class AggregateInPandasExec(
+case class ArrowAggregatePythonExec(
groupingExpressions: Seq[NamedExpression],
aggExpressions: Seq[AggregateExpression],
resultExpressions: Seq[NamedExpression],
- child: SparkPlan)
- extends UnaryExecNode with PythonSQLMetrics {
+ child: SparkPlan,
+ evalType: Int) extends UnaryExecNode with PythonSQLMetrics {
+ if (!supportedPythonEvalTypes.contains(evalType)) {
+ throw SparkException.internalError(s"Unexpected eval type $evalType")
+ }
override val output: Seq[Attribute] = resultExpressions.map(_.toAttribute)
@@ -173,7 +182,7 @@ case class AggregateInPandasExec(
val columnarBatchIter = new ArrowPythonWithNamedArgumentRunner(
pyFuncs,
- PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF,
+ evalType,
argMetas,
aggInputSchema,
sessionLocalTimeZone,
@@ -206,13 +215,33 @@ case class AggregateInPandasExec(
case Some(sessionExpression) =>
val inMemoryThreshold = conf.windowExecBufferInMemoryThreshold
val spillThreshold = conf.windowExecBufferSpillThreshold
+ val spillSizeThreshold = conf.windowExecBufferSpillSizeThreshold
new UpdatingSessionsIterator(iter, groupingWithoutSessionExpressions, sessionExpression,
- child.output, inMemoryThreshold, spillThreshold)
+ child.output, inMemoryThreshold, spillThreshold, spillSizeThreshold)
case None => iter
}
newIter
}
+
+ private def supportedPythonEvalTypes: Array[Int] =
+ Array(
+ PythonEvalType.SQL_GROUPED_AGG_ARROW_UDF,
+ PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF)
+}
+
+object ArrowAggregatePythonExec {
+ def apply(
+ groupingExpressions: Seq[NamedExpression],
+ aggExpressions: Seq[AggregateExpression],
+ resultExpressions: Seq[NamedExpression],
+ child: SparkPlan): ArrowAggregatePythonExec = {
+ val evalTypes = aggExpressions.map(_.aggregateFunction.asInstanceOf[PythonUDAF].evalType)
+ assert(evalTypes.distinct.size == 1,
+ "All aggregate functions must have the same eval type in ArrowAggregatePythonExec")
+ new ArrowAggregatePythonExec(
+ groupingExpressions, aggExpressions, resultExpressions, child, evalTypes.head)
+ }
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonExec.scala
index 9ec454731e4a6..92236ca42b2db 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonExec.scala
@@ -19,8 +19,8 @@ package org.apache.spark.sql.execution.python
import scala.jdk.CollectionConverters._
-import org.apache.spark.{JobArtifactSet, TaskContext}
-import org.apache.spark.api.python.ChainedPythonFunctions
+import org.apache.spark.{JobArtifactSet, SparkException, TaskContext}
+import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.errors.QueryExecutionErrors
@@ -28,6 +28,7 @@ import org.apache.spark.sql.execution.SparkPlan
import org.apache.spark.sql.execution.metric.SQLMetric
import org.apache.spark.sql.execution.python.EvalPythonExec.ArgumentMetadata
import org.apache.spark.sql.types.{StructType, UserDefinedType}
+import org.apache.spark.sql.types.DataType.equalsIgnoreCompatibleCollation
/**
* Grouped a iterator into batches.
@@ -59,11 +60,26 @@ private[spark] class BatchIterator[T](iter: Iterator[T], batchSize: Int)
}
/**
- * A physical plan that evaluates a [[PythonUDF]].
+ * A physical plan that evaluates a vectorized UDF.
+ * Following eval types are supported:
+ *
+ *
+ *
SQL_ARROW_BATCHED_UDF for Arrow Optimized Python UDF
+ *
SQL_SCALAR_ARROW_UDF for Scalar Arrow UDF
+ *
SQL_SCALAR_ARROW_ITER_UDF for Scalar Iterator Arrow UDF
+ *
SQL_SCALAR_PANDAS_UDF for Scalar Pandas UDF
+ *
SQL_SCALAR_PANDAS_ITER_UDF for Scalar Iterator Pandas UDF
+ *
+ *
*/
-case class ArrowEvalPythonExec(udfs: Seq[PythonUDF], resultAttrs: Seq[Attribute], child: SparkPlan,
- evalType: Int)
- extends EvalPythonExec with PythonSQLMetrics {
+case class ArrowEvalPythonExec(
+ udfs: Seq[PythonUDF],
+ resultAttrs: Seq[Attribute],
+ child: SparkPlan,
+ evalType: Int) extends EvalPythonExec with PythonSQLMetrics {
+ if (!supportedPythonEvalTypes.contains(evalType)) {
+ throw SparkException.internalError(s"Unexpected eval type $evalType")
+ }
private[this] val jobArtifactUUID = JobArtifactSet.getCurrentJobArtifactState.map(_.uuid)
@@ -84,6 +100,14 @@ case class ArrowEvalPythonExec(udfs: Seq[PythonUDF], resultAttrs: Seq[Attribute]
override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan =
copy(child = newChild)
+
+ private def supportedPythonEvalTypes: Array[Int] =
+ Array(
+ PythonEvalType.SQL_ARROW_BATCHED_UDF,
+ PythonEvalType.SQL_SCALAR_ARROW_UDF,
+ PythonEvalType.SQL_SCALAR_ARROW_ITER_UDF,
+ PythonEvalType.SQL_SCALAR_PANDAS_UDF,
+ PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF)
}
class ArrowEvalPythonEvaluatorFactory(
@@ -128,7 +152,7 @@ class ArrowEvalPythonEvaluatorFactory(
columnarBatchIter.flatMap { batch =>
val actualDataTypes = (0 until batch.numCols()).map(i => batch.column(i).dataType())
- if (outputTypes != actualDataTypes) {
+ if (!equalsIgnoreCompatibleCollation(outputTypes, actualDataTypes)) {
throw QueryExecutionErrors.arrowDataTypeMismatchError(
"pandas_udf()", outputTypes, actualDataTypes)
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonUDTFExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonUDTFExec.scala
index d7106403a3880..ae1982ecec20e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonUDTFExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonUDTFExec.scala
@@ -25,7 +25,8 @@ import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.errors.QueryExecutionErrors
import org.apache.spark.sql.execution.SparkPlan
import org.apache.spark.sql.execution.python.EvalPythonExec.ArgumentMetadata
-import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.types.{StructType, UserDefinedType}
+import org.apache.spark.sql.types.DataType.equalsIgnoreCompatibleCollation
import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch}
/**
@@ -61,7 +62,9 @@ case class ArrowEvalPythonUDTFExec(
val batchIter = if (batchSize > 0) new BatchIterator(iter, batchSize) else Iterator(iter)
- val outputTypes = resultAttrs.map(_.dataType)
+ val outputTypes = resultAttrs.map(_.dataType.transformRecursively {
+ case udt: UserDefinedType[_] => udt.sqlType
+ })
val columnarBatchIter = new ArrowPythonUDTFRunner(
udtf,
@@ -82,7 +85,7 @@ case class ArrowEvalPythonUDTFExec(
val actualDataTypes = (0 until flattenedBatch.numCols()).map(
i => flattenedBatch.column(i).dataType())
- if (outputTypes != actualDataTypes) {
+ if (!equalsIgnoreCompatibleCollation(outputTypes, actualDataTypes)) {
throw QueryExecutionErrors.arrowDataTypeMismatchError(
"Python UDTF", outputTypes, actualDataTypes)
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala
index 9a9fb574b87fb..70d2b17de2e72 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala
@@ -49,6 +49,8 @@ abstract class BaseArrowPythonRunner(
override val faultHandlerEnabled: Boolean = SQLConf.get.pythonUDFWorkerFaulthandlerEnabled
override val idleTimeoutSeconds: Long = SQLConf.get.pythonUDFWorkerIdleTimeoutSeconds
override val killOnIdleTimeout: Boolean = SQLConf.get.pythonUDFWorkerKillOnIdleTimeout
+ override val tracebackDumpIntervalSeconds: Long =
+ SQLConf.get.pythonUDFWorkerTracebackDumpIntervalSeconds
override val errorOnDuplicatedFieldNames: Boolean = true
@@ -131,7 +133,15 @@ object ArrowPythonRunner {
val legacyPandasConversion = Seq(
SQLConf.PYTHON_TABLE_UDF_LEGACY_PANDAS_CONVERSION_ENABLED.key ->
conf.legacyPandasConversion.toString)
+ val legacyPandasConversionUDF = Seq(
+ SQLConf.PYTHON_UDF_LEGACY_PANDAS_CONVERSION_ENABLED.key ->
+ conf.legacyPandasConversionUDF.toString)
+ val intToDecimalCoercion = Seq(
+ SQLConf.PYTHON_UDF_PANDAS_INT_TO_DECIMAL_COERCION_ENABLED.key ->
+ conf.getConf(SQLConf.PYTHON_UDF_PANDAS_INT_TO_DECIMAL_COERCION_ENABLED, false).toString)
Map(timeZoneConf ++ pandasColsByName ++ arrowSafeTypeCheck ++
- arrowAyncParallelism ++ useLargeVarTypes ++ legacyPandasConversion: _*)
+ arrowAyncParallelism ++ useLargeVarTypes ++
+ intToDecimalCoercion ++
+ legacyPandasConversion ++ legacyPandasConversionUDF: _*)
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonUDTFRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonUDTFRunner.scala
index ae875c777b434..c081787b52095 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonUDTFRunner.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonUDTFRunner.scala
@@ -48,6 +48,20 @@ class ArrowPythonUDTFRunner(
with BasicPythonArrowOutput {
override protected def writeUDF(dataOut: DataOutputStream): Unit = {
+ // For arrow-optimized Python UDTFs (@udtf(useArrow=True)), we need to write
+ // the schema to the worker to support UDT (user-defined type).
+ // Currently, UDT is not supported in PyArrow native UDTFs (arrow_udf)
+ if (evalType == PythonEvalType.SQL_ARROW_TABLE_UDF) {
+ PythonWorkerUtils.writeUTF(schema.json, dataOut)
+ }
+ // Write the table argument offsets for Arrow UDTFs.
+ else if (evalType == PythonEvalType.SQL_ARROW_UDTF) {
+ val tableArgOffsets = argMetas.collect {
+ case ArgumentMetadata(offset, _, isTableArg) if isTableArg => offset
+ }
+ dataOut.writeInt(tableArgOffsets.length)
+ tableArgOffsets.foreach(dataOut.writeInt(_))
+ }
PythonUDTFRunner.writeUDTF(dataOut, udtf, argMetas)
}
@@ -58,6 +72,8 @@ class ArrowPythonUDTFRunner(
override val faultHandlerEnabled: Boolean = SQLConf.get.pythonUDFWorkerFaulthandlerEnabled
override val idleTimeoutSeconds: Long = SQLConf.get.pythonUDFWorkerIdleTimeoutSeconds
override val killOnIdleTimeout: Boolean = SQLConf.get.pythonUDFWorkerKillOnIdleTimeout
+ override val tracebackDumpIntervalSeconds: Long =
+ SQLConf.get.pythonUDFWorkerTracebackDumpIntervalSeconds
override val errorOnDuplicatedFieldNames: Boolean = true
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasEvaluatorFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowWindowPythonEvaluatorFactory.scala
similarity index 97%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasEvaluatorFactory.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowWindowPythonEvaluatorFactory.scala
index e7fc9c7391af4..92ed9ff9de456 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasEvaluatorFactory.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowWindowPythonEvaluatorFactory.scala
@@ -23,7 +23,7 @@ import scala.collection.mutable.ArrayBuffer
import scala.jdk.CollectionConverters._
import org.apache.spark.{JobArtifactSet, PartitionEvaluator, PartitionEvaluatorFactory, SparkEnv, TaskContext}
-import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType}
+import org.apache.spark.api.python.ChainedPythonFunctions
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, BoundReference, EmptyRow, Expression, JoinedRow, NamedArgumentExpression, NamedExpression, PythonFuncExpression, PythonUDAF, SortOrder, SpecificInternalRow, UnsafeProjection, UnsafeRow, WindowExpression}
import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
@@ -37,11 +37,12 @@ import org.apache.spark.sql.types.{DataType, IntegerType, StructField, StructTyp
import org.apache.spark.util.ArrayImplicits._
import org.apache.spark.util.Utils
-class WindowInPandasEvaluatorFactory(
+class ArrowWindowPythonEvaluatorFactory(
val windowExpression: Seq[NamedExpression],
val partitionSpec: Seq[Expression],
val orderSpec: Seq[SortOrder],
val childOutput: Seq[Attribute],
+ val evalType: Int,
val spillSize: SQLMetric,
pythonMetrics: Map[String, SQLMetric],
profiler: Option[String])
@@ -67,7 +68,7 @@ class WindowInPandasEvaluatorFactory(
private object BoundedWindow extends WindowBoundType("bounded")
- private val windowBoundTypeConf = "pandas_window_bound_types"
+ private val windowBoundTypeConf = "window_bound_types"
private def collectFunctions(
udf: PythonFuncExpression): ((ChainedPythonFunctions, Long), Seq[Expression]) = {
@@ -148,6 +149,7 @@ class WindowInPandasEvaluatorFactory(
private val inMemoryThreshold = conf.windowExecBufferInMemoryThreshold
private val spillThreshold = conf.windowExecBufferSpillThreshold
+ private val spillSizeThreshold = conf.windowExecBufferSpillSizeThreshold
private val sessionLocalTimeZone = conf.sessionLocalTimeZone
private val largeVarTypes = conf.arrowUseLargeVarTypes
@@ -286,7 +288,8 @@ class WindowInPandasEvaluatorFactory(
// Manage the current partition.
val buffer: ExternalAppendOnlyUnsafeRowArray =
- new ExternalAppendOnlyUnsafeRowArray(inMemoryThreshold, spillThreshold)
+ new ExternalAppendOnlyUnsafeRowArray(inMemoryThreshold, spillThreshold,
+ spillSizeThreshold)
var bufferIterator: Iterator[UnsafeRow] = _
val indexRow =
@@ -362,7 +365,7 @@ class WindowInPandasEvaluatorFactory(
val windowFunctionResult = new ArrowPythonWithNamedArgumentRunner(
pyFuncs,
- PythonEvalType.SQL_WINDOW_AGG_PANDAS_UDF,
+ evalType,
argMetas,
pythonInputSchema,
sessionLocalTimeZone,
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowWindowPythonExec.scala
similarity index 74%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasExec.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowWindowPythonExec.scala
index 294bcdadc2b20..c8259c10dbd93 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowWindowPythonExec.scala
@@ -17,6 +17,8 @@
package org.apache.spark.sql.execution.python
+import org.apache.spark.SparkException
+import org.apache.spark.api.python.PythonEvalType
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
@@ -26,6 +28,12 @@ import org.apache.spark.sql.execution.window._
/**
* This class calculates and outputs windowed aggregates over the rows in a single partition.
+ * Following eval types are supported:
+ *
+ *
+ *
SQL_WINDOW_AGG_ARROW_UDF for Arrow UDF
+ *
SQL_WINDOW_AGG_PANDAS_UDF for Pandas UDF
+ *
*
* This is similar to [[WindowExec]]. The main difference is that this node does not compute
* any window aggregation values. Instead, it computes the lower and upper bound for each window
@@ -61,7 +69,7 @@ import org.apache.spark.sql.execution.window._
* Unbounded window takes only input columns.
* (2) Bounded window evaluates the udf once per input row.
* Unbounded window evaluates the udf once per window partition.
- * This is controlled by Python runner conf "pandas_window_bound_types"
+ * This is controlled by Python runner conf "window_bound_types"
*
* The logic to compute window bounds is delegated to [[WindowFunctionFrame]] and shared with
* [[WindowExec]]
@@ -69,23 +77,28 @@ import org.apache.spark.sql.execution.window._
* Note this doesn't support partial aggregation and all aggregation is computed from the entire
* window.
*/
-case class WindowInPandasExec(
+case class ArrowWindowPythonExec(
windowExpression: Seq[NamedExpression],
partitionSpec: Seq[Expression],
orderSpec: Seq[SortOrder],
- child: SparkPlan)
- extends WindowExecBase with PythonSQLMetrics {
+ child: SparkPlan,
+ evalType: Int) extends WindowExecBase with PythonSQLMetrics {
+ if (!supportedPythonEvalTypes.contains(evalType)) {
+ throw SparkException.internalError(s"Unexpected eval type $evalType")
+ }
+
override lazy val metrics: Map[String, SQLMetric] = pythonMetrics ++ Map(
"spillSize" -> SQLMetrics.createSizeMetric(sparkContext, "spill size")
)
protected override def doExecute(): RDD[InternalRow] = {
val evaluatorFactory =
- new WindowInPandasEvaluatorFactory(
+ new ArrowWindowPythonEvaluatorFactory(
windowExpression,
partitionSpec,
orderSpec,
child.output,
+ evalType,
longMetric("spillSize"),
pythonMetrics,
conf.pythonUDFProfiler)
@@ -101,6 +114,26 @@ case class WindowInPandasExec(
}
}
- override protected def withNewChildInternal(newChild: SparkPlan): WindowInPandasExec =
+ override protected def withNewChildInternal(newChild: SparkPlan): ArrowWindowPythonExec =
copy(child = newChild)
+
+ private def supportedPythonEvalTypes: Array[Int] =
+ Array(
+ PythonEvalType.SQL_WINDOW_AGG_ARROW_UDF,
+ PythonEvalType.SQL_WINDOW_AGG_PANDAS_UDF)
+}
+
+object ArrowWindowPythonExec {
+ def apply(
+ windowExpression: Seq[NamedExpression],
+ partitionSpec: Seq[Expression],
+ orderSpec: Seq[SortOrder],
+ child: SparkPlan): ArrowWindowPythonExec = {
+ val evalTypes = windowExpression.flatMap(w => WindowFunctionType.pythonEvalType(w))
+ assert(evalTypes.nonEmpty,
+ "Cannot extract eval type from PythonUDAFs in ArrowWindowPythonExec")
+ assert(evalTypes.distinct.size == 1,
+ "All window functions must have the same eval type in ArrowWindowPythonExec")
+ ArrowWindowPythonExec(windowExpression, partitionSpec, orderSpec, child, evalTypes.head)
+ }
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/AttachDistributedSequenceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/AttachDistributedSequenceExec.scala
index a8a6fa97c52a9..e27bde38a6f5f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/AttachDistributedSequenceExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/AttachDistributedSequenceExec.scala
@@ -18,7 +18,6 @@
package org.apache.spark.sql.execution.python
import org.apache.spark.internal.LogKeys.{RDD_ID, SPARK_PLAN_ID}
-import org.apache.spark.internal.MDC
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonUDTFExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonUDTFExec.scala
index c0dcb77817420..a1358c9cd7746 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonUDTFExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonUDTFExec.scala
@@ -120,7 +120,7 @@ object PythonUDTFRunner {
// Write the argument types of the UDTF.
dataOut.writeInt(argMetas.length)
argMetas.foreach {
- case ArgumentMetadata(offset, name) =>
+ case ArgumentMetadata(offset, name, _) =>
dataOut.writeInt(offset)
name match {
case Some(name) =>
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CoGroupedArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CoGroupedArrowPythonRunner.scala
index 27d6f7dc1c66b..8b160accd7a4d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CoGroupedArrowPythonRunner.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CoGroupedArrowPythonRunner.scala
@@ -62,6 +62,8 @@ class CoGroupedArrowPythonRunner(
override val faultHandlerEnabled: Boolean = SQLConf.get.pythonUDFWorkerFaulthandlerEnabled
override val idleTimeoutSeconds: Long = SQLConf.get.pythonUDFWorkerIdleTimeoutSeconds
override val killOnIdleTimeout: Boolean = SQLConf.get.pythonUDFWorkerKillOnIdleTimeout
+ override val tracebackDumpIntervalSeconds: Long =
+ SQLConf.get.pythonUDFWorkerTracebackDumpIntervalSeconds
override val hideTraceback: Boolean = SQLConf.get.pysparkHideTraceback
override val simplifiedTraceback: Boolean = SQLConf.get.pysparkSimplifiedTraceback
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExec.scala
index af6769cfbb9d7..0c366b1280b41 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExec.scala
@@ -28,8 +28,9 @@ object EvalPythonExec {
*
* @param offset the offset of the argument
* @param name the name of the argument if it's a `NamedArgumentExpression`
+ * @param isTableArg whether this argument is a table argument
*/
- case class ArgumentMetadata(offset: Int, name: Option[String])
+ case class ArgumentMetadata(offset: Int, name: Option[String], isTableArg: Boolean = false)
}
/**
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonUDTFExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonUDTFExec.scala
index 41a99693443e5..3cb9431fed6f6 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonUDTFExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonUDTFExec.scala
@@ -68,7 +68,9 @@ trait EvalPythonUDTFExec extends UnaryExecNode {
// flatten all the arguments
val allInputs = new ArrayBuffer[Expression]
val dataTypes = new ArrayBuffer[DataType]
- val argMetas = udtf.children.map { e =>
+ val argMetas = udtf.children.zip(
+ udtf.tableArguments.getOrElse(Seq.fill(udtf.children.length)(false))
+ ).map { case (e: Expression, isTableArg: Boolean) =>
val (key, value) = e match {
case NamedArgumentExpression(key, value) =>
(Some(key), value)
@@ -76,11 +78,11 @@ trait EvalPythonUDTFExec extends UnaryExecNode {
(None, e)
}
if (allInputs.exists(_.semanticEquals(value))) {
- ArgumentMetadata(allInputs.indexWhere(_.semanticEquals(value)), key)
+ ArgumentMetadata(allInputs.indexWhere(_.semanticEquals(value)), key, isTableArg)
} else {
allInputs += value
dataTypes += value.dataType
- ArgumentMetadata(allInputs.length - 1, key)
+ ArgumentMetadata(allInputs.length - 1, key, isTableArg)
}
}.toArray
val projection = MutableProjection.create(allInputs.toSeq, child.output)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala
index fd7ccb2189bff..5d117a67e6bee 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala
@@ -36,7 +36,8 @@ import org.apache.spark.unsafe.types.{UTF8String, VariantVal}
object EvaluatePython {
def needConversionInPython(dt: DataType): Boolean = dt match {
- case DateType | TimestampType | TimestampNTZType | VariantType | _: DayTimeIntervalType => true
+ case DateType | TimestampType | TimestampNTZType | VariantType | _: DayTimeIntervalType
+ | _: TimeType => true
case _: StructType => true
case _: UserDefinedType[_] => true
case ArrayType(elementType, _) => needConversionInPython(elementType)
@@ -78,7 +79,7 @@ object EvaluatePython {
case (d: Decimal, _) => d.toJavaBigDecimal
- case (s: UTF8String, StringType) => s.toString
+ case (s: UTF8String, _: StringType) => s.toString
case (other, _) => other
}
@@ -138,7 +139,7 @@ object EvaluatePython {
case c: Int => c
}
- case TimestampType | TimestampNTZType | _: DayTimeIntervalType => (obj: Any) =>
+ case TimestampType | TimestampNTZType | _: DayTimeIntervalType | _: TimeType => (obj: Any) =>
nullSafeConvert(obj) {
case c: Long => c
// Py4J serializes values between MIN_INT and MAX_INT as Ints, not Longs
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala
index a42e5d3e6c972..1407e020353f0 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala
@@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer
import org.apache.spark.SparkException
import org.apache.spark.api.python.PythonEvalType
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.REASON
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
@@ -169,16 +169,63 @@ object ExtractPythonUDFs extends Rule[LogicalPlan] with Logging {
e.exists(PythonUDF.isScalarPythonUDF)
}
+ /**
+ * Return true if we should extract the current expression, including all of its current
+ * children (including UDF expression, and all others), to a logical node.
+ * The children of the expression can be UDF expressions, this would be nested chaining.
+ * If child UDF expressions were already extracted before, then this will just extract
+ * the current UDF expression, so they will end up in separate logical nodes. The child
+ * expressions will have been transformed to Attribute expressions referencing the child plan
+ * node's output.
+ *
+ * Return false if there is no single continuous chain of UDFs that can be extracted:
+ * - if there are other expression in-between, return false. In
+ * below example, the caller will have to extract bar(baz()) separately first:
+ * Query: foo(1 + bar(baz()))
+ * Plan:
+ * - PythonUDF (foo)
+ * - Project
+ * - PythonUDF (bar)
+ * - PythonUDF (baz)
+ * - if the eval types of the UDF expressions in the chain differ, return false.
+ * - if a UDF has more than one child, e.g. foo(bar(), baz()), return false
+ * If we return false here, the expectation is that the recursive calls of
+ * collectEvaluableUDFsFromExpressions will then visit the children and extract them first to
+ * separate nodes.
+ */
@scala.annotation.tailrec
- private def canEvaluateInPython(e: PythonUDF): Boolean = {
+ private def shouldExtractUDFExpressionTree(e: PythonUDF): Boolean = {
e.children match {
- // single PythonUDF child could be chained and evaluated in Python
- case Seq(u: PythonUDF) => correctEvalType(e) == correctEvalType(u) && canEvaluateInPython(u)
+ case Seq(child: PythonUDF) => correctEvalType(e) == correctEvalType(child) &&
+ shouldExtractUDFExpressionTree(child)
// Python UDF can't be evaluated directly in JVM
case children => !children.exists(hasScalarPythonUDF)
}
}
+ /**
+ * We use the following terminology:
+ * - chaining is the act of combining multiple UDFs into a single logical node. This can be
+ * accomplished in different cases, for example:
+ * - parallel chaining: if the UDFs are siblings, e.g., foo(x), bar(x),
+ * where multiple independent UDFs are evaluated together over the same input
+ * - nested chaining: if the UDFs are nested, e.g., foo(bar(...)),
+ * where the output of one UDF feeds into the next in a sequential pipeline
+ *
+ * collectEvaluableUDFsFromExpressions returns a list of UDF expressions that can be planned
+ * together into one plan node. collectEvaluableUDFsFromExpressions will be called multiple times
+ * by recursive calls of extract(plan), until no more evaluable UDFs are found.
+ *
+ * As an example, consider the following expression tree:
+ * udf1(udf2(udf3(x)), udf4(x))), where all UDFs are PythonUDFs of the same evaltype.
+ * We can only fuse UDFs of the same eval type, and never UDFs of SQL_SCALAR_PANDAS_ITER_UDF.
+ * The following udf expressions will be returned:
+ * - First, we will return Seq(udf3, udf4), as these two UDFs must be evaluated first.
+ * We return both in one Seq, as it is possible to do parallel fusing for udf3 an udf4.
+ * - As we can only chain UDFs with exactly one child, we will not fuse udf2 with its children.
+ * But we can chain udf1 and udf2, so a later call to collectEvaluableUDFsFromExpressions will
+ * return Seq(udf1, udf2).
+ */
private def collectEvaluableUDFsFromExpressions(expressions: Seq[Expression]): Seq[PythonUDF] = {
// If first UDF is SQL_SCALAR_PANDAS_ITER_UDF or SQL_SCALAR_ARROW_ITER_UDF,
// then only return this UDF,
@@ -187,7 +234,7 @@ object ExtractPythonUDFs extends Rule[LogicalPlan] with Logging {
var firstVisitedScalarUDFEvalType: Option[Int] = None
- def canChainUDF(evalType: Int): Boolean = {
+ def canChainWithParallelUDFs(evalType: Int): Boolean = {
if (evalType == PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF ||
evalType == PythonEvalType.SQL_SCALAR_ARROW_ITER_UDF) {
false
@@ -197,12 +244,14 @@ object ExtractPythonUDFs extends Rule[LogicalPlan] with Logging {
}
def collectEvaluableUDFs(expr: Expression): Seq[PythonUDF] = expr match {
- case udf: PythonUDF if PythonUDF.isScalarPythonUDF(udf) && canEvaluateInPython(udf)
+ case udf: PythonUDF if PythonUDF.isScalarPythonUDF(udf)
+ && shouldExtractUDFExpressionTree(udf)
&& firstVisitedScalarUDFEvalType.isEmpty =>
firstVisitedScalarUDFEvalType = Some(correctEvalType(udf))
Seq(udf)
- case udf: PythonUDF if PythonUDF.isScalarPythonUDF(udf) && canEvaluateInPython(udf)
- && canChainUDF(correctEvalType(udf)) =>
+ case udf: PythonUDF if PythonUDF.isScalarPythonUDF(udf)
+ && shouldExtractUDFExpressionTree(udf)
+ && canChainWithParallelUDFs(correctEvalType(udf)) =>
Seq(udf)
case e => e.children.flatMap(collectEvaluableUDFs)
}
@@ -357,6 +406,10 @@ object ExtractPythonUDTFs extends Rule[LogicalPlan] {
BatchEvalPythonUDTF(func, g.requiredChildOutput, g.generatorOutput, child)
case PythonEvalType.SQL_ARROW_TABLE_UDF =>
ArrowEvalPythonUDTF(func, g.requiredChildOutput, g.generatorOutput, child, func.evalType)
+ case PythonEvalType.SQL_ARROW_UDTF =>
+ ArrowEvalPythonUDTF(func, g.requiredChildOutput, g.generatorOutput, child, func.evalType)
+ case _ =>
+ throw SparkException.internalError(s"Unsupported UDTF eval type: ${func.evalType}")
}
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonPlannerRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonPlannerRunner.scala
index 897b520f0b1ab..200a92f3dc3c6 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonPlannerRunner.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonPlannerRunner.scala
@@ -30,7 +30,7 @@ import net.razorvine.pickle.Pickler
import org.apache.spark.{JobArtifactSet, SparkEnv, SparkException}
import org.apache.spark.api.python.{BasePythonRunner, PythonFunction, PythonWorker, PythonWorkerUtils, SpecialLengths}
-import org.apache.spark.internal.{Logging, LogKeys, MDC}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.internal.config.BUFFER_SIZE
import org.apache.spark.internal.config.Python._
import org.apache.spark.sql.internal.SQLConf
@@ -57,6 +57,7 @@ abstract class PythonPlannerRunner[T](func: PythonFunction) extends Logging {
val faultHandlerEnabled: Boolean = SQLConf.get.pythonUDFWorkerFaulthandlerEnabled
val idleTimeoutSeconds: Long = SQLConf.get.pythonUDFWorkerIdleTimeoutSeconds
val killOnIdleTimeout: Boolean = SQLConf.get.pythonUDFWorkerKillOnIdleTimeout
+ val tracebackDumpIntervalSeconds: Long = SQLConf.get.pythonUDFWorkerTracebackDumpIntervalSeconds
val hideTraceback: Boolean = SQLConf.get.pysparkHideTraceback
val simplifiedTraceback: Boolean = SQLConf.get.pysparkSimplifiedTraceback
val workerMemoryMb = SQLConf.get.pythonPlannerExecMemory
@@ -88,6 +89,9 @@ abstract class PythonPlannerRunner[T](func: PythonFunction) extends Logging {
if (faultHandlerEnabled) {
envVars.put("PYTHON_FAULTHANDLER_DIR", faultHandlerLogDir.toString)
}
+ if (tracebackDumpIntervalSeconds > 0L) {
+ envVars.put("PYTHON_TRACEBACK_DUMP_INTERVAL_SECONDS", tracebackDumpIntervalSeconds.toString)
+ }
envVars.put("SPARK_JOB_ARTIFACT_UUID", jobArtifactUUID.getOrElse("default"))
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDFRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDFRunner.scala
index 4baddcd4d9e77..8ff7e57d9421e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDFRunner.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDFRunner.scala
@@ -48,6 +48,8 @@ abstract class BasePythonUDFRunner(
override val faultHandlerEnabled: Boolean = SQLConf.get.pythonUDFWorkerFaulthandlerEnabled
override val idleTimeoutSeconds: Long = SQLConf.get.pythonUDFWorkerIdleTimeoutSeconds
override val killOnIdleTimeout: Boolean = SQLConf.get.pythonUDFWorkerKillOnIdleTimeout
+ override val tracebackDumpIntervalSeconds: Long =
+ SQLConf.get.pythonUDFWorkerTracebackDumpIntervalSeconds
override val bufferSize: Int = SQLConf.get.getConf(SQLConf.PYTHON_UDF_BUFFER_SIZE)
override val batchSizeForPythonUDF: Int =
@@ -191,7 +193,7 @@ object PythonUDFRunner {
funcs.zip(argMetas).foreach { case ((chained, resultId), metas) =>
dataOut.writeInt(metas.length)
metas.foreach {
- case ArgumentMetadata(offset, name) =>
+ case ArgumentMetadata(offset, name, _) =>
dataOut.writeInt(offset)
name match {
case Some(name) =>
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala
index c43cbad7c395f..5ab0d259d83c4 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala
@@ -25,7 +25,7 @@ import net.razorvine.pickle.Pickler
import org.apache.spark.api.python.{PythonEvalType, PythonFunction, PythonWorkerUtils, SpecialLengths}
import org.apache.spark.sql.{Column, TableArg}
-import org.apache.spark.sql.catalyst.expressions.{Alias, Ascending, Descending, Expression, FunctionTableSubqueryArgumentExpression, NamedArgumentExpression, NullsFirst, NullsLast, PythonUDAF, PythonUDF, PythonUDTF, PythonUDTFAnalyzeResult, PythonUDTFSelectedExpression, SortOrder, UnresolvedPolymorphicPythonUDTF}
+import org.apache.spark.sql.catalyst.expressions.{Alias, Ascending, Descending, Expression, FunctionTableSubqueryArgumentExpression, NamedArgumentExpression, NullsFirst, NullsLast, PythonUDAF, PythonUDF, PythonUDTF, PythonUDTFAnalyzeResult, PythonUDTFSelectedExpression, SortOrder, UnresolvedPolymorphicPythonUDTF, UnresolvedTableArgPlanId}
import org.apache.spark.sql.catalyst.parser.ParserInterface
import org.apache.spark.sql.catalyst.plans.logical.{Generate, LogicalPlan, NamedParametersSupport, OneRowRelation}
import org.apache.spark.sql.classic.{DataFrame, Dataset, SparkSession}
@@ -50,7 +50,8 @@ case class UserDefinedPythonFunction(
|| pythonEvalType ==PythonEvalType.SQL_ARROW_BATCHED_UDF
|| pythonEvalType == PythonEvalType.SQL_SCALAR_PANDAS_UDF
|| pythonEvalType == PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF
- || pythonEvalType == PythonEvalType.SQL_SCALAR_ARROW_UDF) {
+ || pythonEvalType == PythonEvalType.SQL_SCALAR_ARROW_UDF
+ || pythonEvalType == PythonEvalType.SQL_GROUPED_AGG_ARROW_UDF) {
/*
* Check if the named arguments:
* - don't have duplicated names
@@ -61,8 +62,9 @@ case class UserDefinedPythonFunction(
throw QueryCompilationErrors.namedArgumentsNotSupported(name)
}
- if (pythonEvalType == PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF) {
- PythonUDAF(name, func, dataType, e, udfDeterministic)
+ if (pythonEvalType == PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF
+ || pythonEvalType == PythonEvalType.SQL_GROUPED_AGG_ARROW_UDF) {
+ PythonUDAF(name, func, dataType, e, udfDeterministic, pythonEvalType)
} else {
PythonUDF(name, func, dataType, e, pythonEvalType, udfDeterministic)
}
@@ -121,6 +123,16 @@ case class UserDefinedPythonTableFunction(
*/
NamedParametersSupport.splitAndCheckNamedArguments(exprs, name)
+ // Check which argument is a table argument here since it will be replaced with
+ // `UnresolvedAttribute` to construct lateral join.
+ val tableArgs = exprs.map {
+ case _: FunctionTableSubqueryArgumentExpression => true
+ case _: UnresolvedTableArgPlanId => true
+ case NamedArgumentExpression(_, _: FunctionTableSubqueryArgumentExpression) => true
+ case NamedArgumentExpression(_, _: UnresolvedTableArgPlanId) => true
+ case _ => false
+ }
+
val udtf = returnType match {
case Some(rt) =>
PythonUDTF(
@@ -130,15 +142,9 @@ case class UserDefinedPythonTableFunction(
pickledAnalyzeResult = None,
children = exprs,
evalType = pythonEvalType,
- udfDeterministic = udfDeterministic)
+ udfDeterministic = udfDeterministic,
+ tableArguments = Some(tableArgs))
case _ =>
- // Check which argument is a table argument here since it will be replaced with
- // `UnresolvedAttribute` to construct lateral join.
- val tableArgs = exprs.map {
- case _: FunctionTableSubqueryArgumentExpression => true
- case NamedArgumentExpression(_, _: FunctionTableSubqueryArgumentExpression) => true
- case _ => false
- }
val runAnalyzeInPython = (func: PythonFunction, exprs: Seq[Expression]) => {
val runner =
new UserDefinedPythonTableFunctionAnalyzeRunner(name, func, exprs, tableArgs, parser)
@@ -150,7 +156,8 @@ case class UserDefinedPythonTableFunction(
children = exprs,
evalType = pythonEvalType,
udfDeterministic = udfDeterministic,
- resolveElementMetadata = runAnalyzeInPython)
+ resolveElementMetadata = runAnalyzeInPython,
+ tableArguments = Some(tableArgs))
}
Generate(
udtf,
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/ApplyInPandasWithStatePythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/ApplyInPandasWithStatePythonRunner.scala
index 0de937df05f4a..b6f6a4cbc30b6 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/ApplyInPandasWithStatePythonRunner.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/ApplyInPandasWithStatePythonRunner.scala
@@ -28,7 +28,6 @@ import org.json4s.jackson.JsonMethods._
import org.apache.spark.api.python._
import org.apache.spark.internal.LogKeys.CONFIG
-import org.apache.spark.internal.MDC
import org.apache.spark.sql.Row
import org.apache.spark.sql.api.python.PythonSQLUtils
import org.apache.spark.sql.catalyst.InternalRow
@@ -39,7 +38,7 @@ import org.apache.spark.sql.execution.metric.SQLMetric
import org.apache.spark.sql.execution.python.{PythonArrowInput, PythonArrowOutput, PythonUDFRunner}
import org.apache.spark.sql.execution.python.streaming.ApplyInPandasWithStatePythonRunner.{COUNT_COLUMN_SCHEMA_FROM_PYTHON_WORKER, InType, OutType, OutTypeForState, STATE_METADATA_SCHEMA_FROM_PYTHON_WORKER}
import org.apache.spark.sql.execution.python.streaming.ApplyInPandasWithStateWriter.STATE_METADATA_SCHEMA
-import org.apache.spark.sql.execution.streaming.GroupStateImpl
+import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.GroupStateImpl
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch}
@@ -78,6 +77,8 @@ class ApplyInPandasWithStatePythonRunner(
override val faultHandlerEnabled: Boolean = SQLConf.get.pythonUDFWorkerFaulthandlerEnabled
override val idleTimeoutSeconds: Long = SQLConf.get.pythonUDFWorkerIdleTimeoutSeconds
override val killOnIdleTimeout: Boolean = SQLConf.get.pythonUDFWorkerKillOnIdleTimeout
+ override val tracebackDumpIntervalSeconds: Long =
+ SQLConf.get.pythonUDFWorkerTracebackDumpIntervalSeconds
private val sqlConf = SQLConf.get
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/ApplyInPandasWithStateWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/ApplyInPandasWithStateWriter.scala
index 6bfa5440db37f..f55ca749112fb 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/ApplyInPandasWithStateWriter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/ApplyInPandasWithStateWriter.scala
@@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeRow}
import org.apache.spark.sql.execution.arrow.ArrowWriter
import org.apache.spark.sql.execution.arrow.ArrowWriter.createFieldWriter
-import org.apache.spark.sql.execution.streaming.GroupStateImpl
+import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.GroupStateImpl
import org.apache.spark.sql.types.{BinaryType, BooleanType, IntegerType, StringType, StructField, StructType}
import org.apache.spark.unsafe.types.UTF8String
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/FlatMapGroupsInPandasWithStateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/FlatMapGroupsInPandasWithStateExec.scala
index e53bdae813d26..5b9b95ef413aa 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/FlatMapGroupsInPandasWithStateExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/FlatMapGroupsInPandasWithStateExec.scala
@@ -28,9 +28,10 @@ import org.apache.spark.sql.catalyst.types.DataTypeUtils
import org.apache.spark.sql.execution.{GroupedIterator, SparkPlan, UnaryExecNode}
import org.apache.spark.sql.execution.python.ArrowPythonRunner
import org.apache.spark.sql.execution.python.PandasGroupUtils.resolveArgOffsets
-import org.apache.spark.sql.execution.streaming._
-import org.apache.spark.sql.execution.streaming.GroupStateImpl.NO_TIMESTAMP
-import org.apache.spark.sql.execution.streaming.state.FlatMapGroupsWithStateExecHelper.StateData
+import org.apache.spark.sql.execution.streaming.operators.stateful._
+import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.{FlatMapGroupsWithStateExecBase, GroupStateImpl}
+import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.FlatMapGroupsWithStateExecHelper.StateData
+import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.GroupStateImpl.NO_TIMESTAMP
import org.apache.spark.sql.execution.streaming.state.StateStore
import org.apache.spark.sql.streaming.{GroupStateTimeout, OutputMode}
import org.apache.spark.sql.types.StructType
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/PythonForeachWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/PythonForeachWriter.scala
index 04c51c859baca..01643af9cf30d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/PythonForeachWriter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/PythonForeachWriter.scala
@@ -102,6 +102,8 @@ class PythonForeachWriter(func: PythonFunction, schema: StructType)
override val faultHandlerEnabled: Boolean = SQLConf.get.pythonUDFWorkerFaulthandlerEnabled
override val idleTimeoutSeconds: Long = SQLConf.get.pythonUDFWorkerIdleTimeoutSeconds
override val killOnIdleTimeout: Boolean = SQLConf.get.pythonUDFWorkerKillOnIdleTimeout
+ override val tracebackDumpIntervalSeconds: Long =
+ SQLConf.get.pythonUDFWorkerTracebackDumpIntervalSeconds
override val hideTraceback: Boolean = SQLConf.get.pysparkHideTraceback
override val simplifiedTraceback: Boolean = SQLConf.get.pysparkSimplifiedTraceback
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/PythonStreamingSourceRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/PythonStreamingSourceRunner.scala
index 3979220618baa..5945b91084278 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/PythonStreamingSourceRunner.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/PythonStreamingSourceRunner.scala
@@ -28,7 +28,7 @@ import org.apache.arrow.vector.ipc.ArrowStreamReader
import org.apache.spark.SparkEnv
import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths}
-import org.apache.spark.internal.{Logging, LogKeys, MDC}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.internal.LogKeys.PYTHON_EXEC
import org.apache.spark.internal.config.BUFFER_SIZE
import org.apache.spark.internal.config.Python.PYTHON_AUTH_SOCKET_TIMEOUT
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkExec.scala
index b65d46fb16325..1b967af38b6db 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkExec.scala
@@ -35,8 +35,10 @@ import org.apache.spark.sql.execution.{CoGroupedIterator, SparkPlan}
import org.apache.spark.sql.execution.metric.SQLMetric
import org.apache.spark.sql.execution.python.ArrowPythonRunner
import org.apache.spark.sql.execution.python.PandasGroupUtils.{executePython, groupAndProject, resolveArgOffsets}
-import org.apache.spark.sql.execution.streaming.{DriverStatefulProcessorHandleImpl, StatefulOperatorStateInfo, StatefulProcessorHandleImpl, TransformWithStateExecBase, TransformWithStateVariableInfo}
-import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.StateStoreAwareZipPartitionsHelper
+import org.apache.spark.sql.execution.streaming.operators.stateful.{StatefulOperatorStateInfo, StatefulOperatorsUtils}
+import org.apache.spark.sql.execution.streaming.operators.stateful.join.StreamingSymmetricHashJoinHelper.StateStoreAwareZipPartitionsHelper
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.{TransformWithStateExecBase, TransformWithStateVariableInfo}
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.{DriverStatefulProcessorHandleImpl, StatefulProcessorHandleImpl}
import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, RocksDBStateStoreProvider, StateSchemaValidationResult, StateStore, StateStoreColFamilySchema, StateStoreConf, StateStoreId, StateStoreOps, StateStoreProvider, StateStoreProviderId}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.streaming.{OutputMode, TimeMode}
@@ -95,9 +97,9 @@ case class TransformWithStateInPySparkExec(
override def shortName: String = if (
userFacingDataType == TransformWithStateInPySpark.UserFacingDataType.PANDAS
) {
- "transformWithStateInPandasExec"
+ StatefulOperatorsUtils.TRANSFORM_WITH_STATE_IN_PANDAS_EXEC_OP_NAME
} else {
- "transformWithStateInPySparkExec"
+ StatefulOperatorsUtils.TRANSFORM_WITH_STATE_IN_PYSPARK_EXEC_OP_NAME
}
private val pythonUDF = functionExpr.asInstanceOf[PythonUDF]
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkPythonRunner.scala
index dffdaca1b835e..51dc179c901ab 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkPythonRunner.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkPythonRunner.scala
@@ -35,7 +35,7 @@ import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.execution.metric.SQLMetric
import org.apache.spark.sql.execution.python.{BasicPythonArrowOutput, PythonArrowInput, PythonUDFRunner}
import org.apache.spark.sql.execution.python.streaming.TransformWithStateInPySparkPythonRunner.{GroupedInType, InType}
-import org.apache.spark.sql.execution.streaming.{DriverStatefulProcessorHandleImpl, StatefulProcessorHandleImpl}
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.{DriverStatefulProcessorHandleImpl, StatefulProcessorHandleImpl}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.vectorized.ColumnarBatch
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkStateServer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkStateServer.scala
index 5c8a55edaffc1..4edeae132b47a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkStateServer.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkStateServer.scala
@@ -29,14 +29,15 @@ import org.apache.arrow.vector.VectorSchemaRoot
import org.apache.arrow.vector.ipc.ArrowStreamWriter
import org.apache.spark.SparkEnv
-import org.apache.spark.internal.{Logging, LogKeys, MDC}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.internal.config.Python.PYTHON_UNIX_DOMAIN_SOCKET_ENABLED
import org.apache.spark.sql.{Encoders, Row}
import org.apache.spark.sql.api.python.PythonSQLUtils
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
-import org.apache.spark.sql.execution.streaming.{ImplicitGroupingKeyTracker, StatefulProcessorHandleImpl, StatefulProcessorHandleImplBase, StatefulProcessorHandleState, StateVariableType}
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.StateVariableType
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.{ImplicitGroupingKeyTracker, StatefulProcessorHandleImpl, StatefulProcessorHandleImplBase, StatefulProcessorHandleState}
import org.apache.spark.sql.execution.streaming.state.StateMessage.{HandleState, ImplicitGroupingKeyRequest, ListStateCall, MapStateCall, StatefulProcessorCall, StateRequest, StateResponse, StateResponseWithLongTypeVal, StateResponseWithMapIterator, StateResponseWithMapKeysOrValues, StateResponseWithStringTypeVal, StateResponseWithTimer, StateVariableRequest, TimerInfo, TimerRequest, TimerStateCallCommand, TimerValueRequest, UtilsRequest, ValueStateCall}
import org.apache.spark.sql.execution.streaming.state.StateMessage.KeyAndValuePair
import org.apache.spark.sql.execution.streaming.state.StateMessage.StateResponseWithListGet
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/benchmark/BenchmarkTransformWithStateInPySparkStateServer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/benchmark/BenchmarkTransformWithStateInPySparkStateServer.scala
index 3cb9bfa24a536..5dc7d9733dcdc 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/benchmark/BenchmarkTransformWithStateInPySparkStateServer.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/benchmark/BenchmarkTransformWithStateInPySparkStateServer.scala
@@ -33,9 +33,7 @@ import org.apache.spark.sql.Encoder
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.execution.python.streaming.TransformWithStateInPySparkStateServer
-import org.apache.spark.sql.execution.streaming.ImplicitGroupingKeyTracker
-import org.apache.spark.sql.execution.streaming.QueryInfoImpl
-import org.apache.spark.sql.execution.streaming.StatefulProcessorHandleImplBase
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.{ImplicitGroupingKeyTracker, QueryInfoImpl, StatefulProcessorHandleImplBase}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.streaming.ListState
import org.apache.spark.sql.streaming.MapState
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/ArrowRRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/ArrowRRunner.scala
index aaf2f256273d3..fafe1fbe313ab 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/ArrowRRunner.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/ArrowRRunner.scala
@@ -30,7 +30,7 @@ import org.apache.spark.TaskContext
import org.apache.spark.api.r._
import org.apache.spark.api.r.SpecialLengths
import org.apache.spark.broadcast.Broadcast
-import org.apache.spark.internal.{LogKeys, MDC}
+import org.apache.spark.internal.{LogKeys}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.execution.arrow.ArrowWriter
import org.apache.spark.sql.types.StructType
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala
index b382642eb6bf6..66e90ec689131 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala
@@ -25,11 +25,12 @@ import scala.collection.mutable.ArrayBuffer
import org.apache.hadoop.fs.Path
import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext}
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{BATCH_ID, PATH}
-import org.apache.spark.internal.io.FileCommitProtocol
+import org.apache.spark.internal.io.{FileCommitProtocol, FileNameSpec}
import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage
import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.execution.streaming.sinks.{FileStreamSinkLog, SinkFileStatus}
/**
* A [[FileCommitProtocol]] that tracks the list of valid files in a manifest file, used in
@@ -114,13 +115,13 @@ class ManifestFileCommitProtocol(jobId: String, path: String)
}
override def newTaskTempFile(
- taskContext: TaskAttemptContext, dir: Option[String], ext: String): String = {
+ taskContext: TaskAttemptContext, dir: Option[String], spec: FileNameSpec): String = {
// The file name looks like part-r-00000-2dd664f9-d2c4-4ffe-878f-c6c70c1fb0cb_00003.gz.parquet
// Note that %05d does not truncate the split number, so if we have more than 100000 tasks,
// the file name is fine and won't overflow.
val split = taskContext.getTaskAttemptID.getTaskID.getId
val uuid = UUID.randomUUID.toString
- val filename = f"part-$split%05d-$uuid$ext"
+ val filename = f"part-$split%05d-$uuid${spec.suffix}"
val file = dir.map { d =>
new Path(new Path(path, d), filename).toString
@@ -133,7 +134,7 @@ class ManifestFileCommitProtocol(jobId: String, path: String)
}
override def newTaskTempFileAbsPath(
- taskContext: TaskAttemptContext, absoluteDir: String, ext: String): String = {
+ taskContext: TaskAttemptContext, absoluteDir: String, spec: FileNameSpec): String = {
throw QueryExecutionErrors.addFilesWithAbsolutePathUnsupportedError(this.toString)
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingErrors.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingErrors.scala
new file mode 100644
index 0000000000000..98b8832ee2a82
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingErrors.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.execution.streaming
+
+import org.apache.spark.SparkException
+
+/**
+ * Object for grouping error messages from streaming query exceptions
+ */
+object StreamingErrors {
+ def cannotLoadCheckpointFileManagerClass(path: String, className: String, err: Throwable):
+ Throwable = {
+ new SparkException(
+ errorClass = "CANNOT_LOAD_CHECKPOINT_FILE_MANAGER.ERROR_LOADING_CLASS",
+ messageParameters = Map("path" -> path, "className" -> className, "msg" -> err.toString),
+ cause = err
+ )
+ }
+
+ def cannotLoadCheckpointFileManager(path: String, err: Throwable):
+ Throwable = {
+ new SparkException(
+ errorClass = "CANNOT_LOAD_CHECKPOINT_FILE_MANAGER.UNCATEGORIZED",
+ messageParameters = Map("path" -> path),
+ cause = err
+ )
+ }
+}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncCommitLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/AsyncCommitLog.scala
similarity index 97%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncCommitLog.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/AsyncCommitLog.scala
index 6db01624fd26b..116ea18326ef0 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncCommitLog.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/AsyncCommitLog.scala
@@ -15,14 +15,14 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.checkpointing
import java.io.OutputStream
import java.util.concurrent.{CompletableFuture, ConcurrentLinkedDeque, ThreadPoolExecutor}
import scala.jdk.CollectionConverters._
-import org.apache.spark.internal.{LogKeys, MDC}
+import org.apache.spark.internal.LogKeys
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.errors.QueryExecutionErrors
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncOffsetSeqLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/AsyncOffsetSeqLog.scala
similarity index 98%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncOffsetSeqLog.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/AsyncOffsetSeqLog.scala
index 54a8855b77cdb..18d18e61da475 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncOffsetSeqLog.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/AsyncOffsetSeqLog.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.checkpointing
import java.io.OutputStream
import java.util.concurrent._
@@ -23,7 +23,7 @@ import java.util.concurrent.atomic.AtomicLong
import scala.jdk.CollectionConverters._
-import org.apache.spark.internal.{LogKeys, MDC}
+import org.apache.spark.internal.{LogKeys}
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.errors.QueryExecutionErrors
import org.apache.spark.util.{Clock, SystemClock}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/CheckpointFileManager.scala
similarity index 92%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/CheckpointFileManager.scala
index 982cc13c40868..3d07483ba4dd1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/CheckpointFileManager.scala
@@ -14,9 +14,10 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.checkpointing
import java.io.{FileNotFoundException, OutputStream}
+import java.lang.reflect.InvocationTargetException
import java.util.{EnumSet, UUID}
import scala.util.control.NonFatal
@@ -26,10 +27,11 @@ import org.apache.hadoop.fs._
import org.apache.hadoop.fs.local.{LocalFs, RawLocalFs}
import org.apache.hadoop.fs.permission.FsPermission
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{FINAL_PATH, PATH, TEMP_PATH}
import org.apache.spark.sql.errors.QueryExecutionErrors
-import org.apache.spark.sql.execution.streaming.CheckpointFileManager.RenameHelperMethods
+import org.apache.spark.sql.execution.streaming.StreamingErrors
+import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager.RenameHelperMethods
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.util.Utils
@@ -49,7 +51,7 @@ import org.apache.spark.util.Utils
*/
trait CheckpointFileManager {
- import org.apache.spark.sql.execution.streaming.CheckpointFileManager._
+ import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager._
/**
* Create a file and make its contents available atomically after the output stream is closed.
@@ -200,10 +202,19 @@ object CheckpointFileManager extends Logging {
val fileManagerClass = hadoopConf.get(
SQLConf.STREAMING_CHECKPOINT_FILE_MANAGER_CLASS.parent.key)
if (fileManagerClass != null) {
- return Utils.classForName(fileManagerClass)
- .getConstructor(classOf[Path], classOf[Configuration])
- .newInstance(path, hadoopConf)
- .asInstanceOf[CheckpointFileManager]
+ try {
+ return Utils.classForName(fileManagerClass)
+ .getConstructor(classOf[Path], classOf[Configuration])
+ .newInstance(path, hadoopConf)
+ .asInstanceOf[CheckpointFileManager]
+ } catch {
+ case e: InvocationTargetException if e.getCause != null =>
+ throw StreamingErrors.cannotLoadCheckpointFileManagerClass(path.toString,
+ fileManagerClass, e.getCause)
+ case NonFatal(e) =>
+ throw StreamingErrors.cannotLoadCheckpointFileManagerClass(path.toString,
+ fileManagerClass, e)
+ }
}
try {
// Try to create a manager based on `FileContext` because HDFS's `FileContext.rename()
@@ -218,6 +229,8 @@ object CheckpointFileManager extends Logging {
log"the implementation of FileSystem.rename() is not atomic, then the correctness " +
log"and fault-tolerance of your Structured Streaming is not guaranteed.")
new FileSystemBasedCheckpointFileManager(path, hadoopConf)
+ case NonFatal(e) =>
+ throw StreamingErrors.cannotLoadCheckpointFileManager(path.toString, e)
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CommitLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/CommitLog.scala
similarity index 98%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CommitLog.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/CommitLog.scala
index f501b62b8a718..6892b6b535cf9 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CommitLog.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/CommitLog.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.checkpointing
import java.io.{InputStream, OutputStream}
import java.nio.charset.StandardCharsets._
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/HDFSMetadataLog.scala
similarity index 98%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/HDFSMetadataLog.scala
index 423648f445904..6d35b1a8f8c00 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/HDFSMetadataLog.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.checkpointing
import java.io._
import java.nio.charset.StandardCharsets
@@ -24,17 +24,17 @@ import java.util.{Collections, LinkedHashMap => JLinkedHashMap}
import scala.jdk.CollectionConverters._
import scala.reflect.ClassTag
-import org.apache.commons.io.IOUtils
import org.apache.hadoop.fs._
import org.json4s.{Formats, NoTypeHints}
import org.json4s.jackson.Serialization
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys._
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.errors.QueryExecutionErrors
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.util.ArrayImplicits._
+import org.apache.spark.util.Utils
/**
@@ -191,7 +191,7 @@ class HDFSMetadataLog[T <: AnyRef : ClassTag](sparkSession: SparkSession, path:
throw new IllegalStateException(
s"Failed to read log file $batchMetadataFile. ${ise.getMessage}", ise)
} finally {
- IOUtils.closeQuietly(input)
+ Utils.closeQuietly(input)
}
} else {
throw QueryExecutionErrors.batchMetadataFileNotFoundError(batchMetadataFile)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/MetadataLog.scala
similarity index 97%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLog.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/MetadataLog.scala
index 4083d5d897afc..18c04e0b9b5db 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLog.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/MetadataLog.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.checkpointing
/**
* A general MetadataLog that supports the following features:
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataVersionUtil.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/MetadataVersionUtil.scala
similarity index 97%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataVersionUtil.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/MetadataVersionUtil.scala
index 854e36b5304a9..8d832c21287a5 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataVersionUtil.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/MetadataVersionUtil.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.checkpointing
import org.apache.spark.sql.errors.QueryExecutionErrors
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/OffsetSeq.scala
similarity index 93%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/OffsetSeq.scala
index a599f3bc66118..c1c3c379719a1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/OffsetSeq.scala
@@ -15,17 +15,20 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.checkpointing
import org.json4s.{Formats, NoTypeHints}
import org.json4s.jackson.Serialization
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{CONFIG, DEFAULT_VALUE, NEW_VALUE, OLD_VALUE, TIP}
import org.apache.spark.io.CompressionCodec
import org.apache.spark.sql.RuntimeConfig
import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2, SparkDataStream}
-import org.apache.spark.sql.execution.streaming.state.{FlatMapGroupsWithStateExecHelper, StreamingAggregationStateManager, SymmetricHashJoinStateManager}
+import org.apache.spark.sql.execution.streaming.operators.stateful.StreamingAggregationStateManager
+import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.FlatMapGroupsWithStateExecHelper
+import org.apache.spark.sql.execution.streaming.operators.stateful.join.SymmetricHashJoinStateManager
+import org.apache.spark.sql.execution.streaming.runtime.{MultipleWatermarkPolicy, StreamProgress}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.SQLConf._
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/OffsetSeqLog.scala
similarity index 96%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLog.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/OffsetSeqLog.scala
index 7e490ef4cd53d..816563b3f09fd 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLog.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/OffsetSeqLog.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.checkpointing
import java.io.{InputStream, OutputStream}
@@ -25,6 +25,7 @@ import scala.io.{Source => IOSource}
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2}
+import org.apache.spark.sql.execution.streaming.runtime.SerializedOffset
/**
* This class is used to log offsets to persistent files in HDFS.
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
index 67585bdf20ce0..d8666e6740046 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
@@ -19,8 +19,8 @@ package org.apache.spark.sql.execution.streaming
import java.util
-import org.apache.spark.sql._
-import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability}
+import org.apache.spark.sql.{Column => _, _}
+import org.apache.spark.sql.connector.catalog.{Column, SupportsWrite, Table, TableCapability}
import org.apache.spark.sql.connector.write.{LogicalWriteInfo, SupportsTruncate, Write, WriteBuilder}
import org.apache.spark.sql.connector.write.streaming.StreamingWrite
import org.apache.spark.sql.execution.streaming.sources.ConsoleWrite
@@ -64,7 +64,7 @@ object ConsoleTable extends Table with SupportsWrite {
override def name(): String = "console"
- override def schema(): StructType = StructType(Nil)
+ override def columns(): Array[Column] = Array.empty
override def capabilities(): util.Set[TableCapability] = {
util.EnumSet.of(TableCapability.STREAMING_WRITE)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
index a13c00ee20576..51cd457fbc856 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
@@ -26,7 +26,6 @@ import scala.collection.mutable.{Map => MutableMap}
import org.apache.spark.SparkEnv
import org.apache.spark.internal.LogKeys._
-import org.apache.spark.internal.MDC
import org.apache.spark.sql.catalyst.expressions.{CurrentDate, CurrentTimestampLike, LocalTimestamp}
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.streaming.{StreamingRelationV2, WriteToStream}
@@ -40,6 +39,8 @@ import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors
import org.apache.spark.sql.execution.SQLExecution
import org.apache.spark.sql.execution.datasources.v2.{StreamingDataSourceV2Relation, StreamingDataSourceV2ScanRelation}
import org.apache.spark.sql.execution.streaming._
+import org.apache.spark.sql.execution.streaming.checkpointing.{CommitMetadata, OffsetSeq}
+import org.apache.spark.sql.execution.streaming.runtime.{AcceptsLatestSeenOffsetHandler, ACTIVE, ContinuousExecutionContext, IncrementalExecution, ProcessingTimeExecutor, RECONFIGURING, State, StreamExecution, StreamExecutionContext, TERMINATED, WatermarkPropagator}
import org.apache.spark.sql.streaming.Trigger
import org.apache.spark.util.ArrayImplicits._
import org.apache.spark.util.Clock
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousQueuedDataReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousQueuedDataReader.scala
index 398df496d15ff..d9eff62bcc986 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousQueuedDataReader.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousQueuedDataReader.scala
@@ -23,7 +23,7 @@ import java.util.concurrent.{ArrayBlockingQueue, TimeUnit}
import scala.util.control.NonFatal
import org.apache.spark.{SparkEnv, TaskContext}
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys._
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala
index 132d9a9d61e57..ef5eab951b605 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala
@@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.connector.read.InputPartition
import org.apache.spark.sql.connector.read.streaming.{ContinuousPartitionReader, ContinuousPartitionReaderFactory, ContinuousStream, Offset, PartitionOffset}
-import org.apache.spark.sql.execution.streaming.{RateStreamOffset, ValueRunTimeMsPair}
+import org.apache.spark.sql.execution.streaming.runtime.{RateStreamOffset, ValueRunTimeMsPair}
case class RateStreamPartitionOffset(
partition: Int, currentValue: Long, currentTimeMs: Long) extends PartitionOffset
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala
index 273ffa6aefb7b..c23e38a13efee 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala
@@ -29,7 +29,7 @@ import org.json4s.{DefaultFormats, Formats, NoTypeHints}
import org.json4s.jackson.Serialization
import org.apache.spark.SparkEnv
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{HOST, PORT}
import org.apache.spark.rpc.RpcEndpointRef
import org.apache.spark.sql.Encoders
@@ -38,7 +38,8 @@ import org.apache.spark.sql.catalyst.encoders.encoderFor
import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow}
import org.apache.spark.sql.connector.read.InputPartition
import org.apache.spark.sql.connector.read.streaming.{ContinuousPartitionReader, ContinuousPartitionReaderFactory, ContinuousStream, Offset, PartitionOffset}
-import org.apache.spark.sql.execution.streaming.{Offset => _, _}
+import org.apache.spark.sql.execution.streaming.{Offset => _}
+import org.apache.spark.sql.execution.streaming.runtime.{ContinuousRecordEndpoint, ContinuousRecordPartitionOffset, GetRecord}
import org.apache.spark.sql.execution.streaming.sources.TextSocketReader
import org.apache.spark.sql.util.CaseInsensitiveStringMap
import org.apache.spark.util.RpcUtils
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousWriteRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousWriteRDD.scala
index d5daa9a875f83..51399969df1b9 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousWriteRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousWriteRDD.scala
@@ -18,7 +18,7 @@
package org.apache.spark.sql.execution.streaming.continuous
import org.apache.spark.{Partition, SparkEnv, TaskContext}
-import org.apache.spark.internal.{LogKeys, MDC}
+import org.apache.spark.internal.{LogKeys}
import org.apache.spark.internal.LogKeys._
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala
index 42ce32e1bc674..8b5d6c580f105 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala
@@ -17,7 +17,7 @@
package org.apache.spark.sql.execution.streaming.continuous
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys._
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/EventTimeWatermarkExec.scala
similarity index 99%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/EventTimeWatermarkExec.scala
index d25c4be0fb84a..03c8d70ec87f9 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/EventTimeWatermarkExec.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.operators.stateful
import java.util.UUID
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MergingSortWithSessionWindowStateIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/MergingSortWithSessionWindowStateIterator.scala
similarity index 97%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MergingSortWithSessionWindowStateIterator.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/MergingSortWithSessionWindowStateIterator.scala
index a923ebd79889c..69577a1f74339 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MergingSortWithSessionWindowStateIterator.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/MergingSortWithSessionWindowStateIterator.scala
@@ -15,13 +15,13 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.operators.stateful
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeProjection, UnsafeRow}
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
-import org.apache.spark.sql.execution.streaming.state.{ReadStateStore, StreamingSessionWindowStateManager}
+import org.apache.spark.sql.execution.streaming.state.ReadStateStore
/**
* This class technically does the merge sort between input rows and existing sessions in state,
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulOperatorPartitioning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/StatefulOperatorPartitioning.scala
similarity index 97%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulOperatorPartitioning.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/StatefulOperatorPartitioning.scala
index 527349201574e..ed2941e3f2f4c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulOperatorPartitioning.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/StatefulOperatorPartitioning.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.operators.stateful
import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Distribution, StatefulOpClusteredDistribution}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StreamingAggregationStateManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/StreamingAggregationStateManager.scala
similarity index 97%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StreamingAggregationStateManager.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/StreamingAggregationStateManager.scala
index 97feb9b579af9..c7f7f388010da 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StreamingAggregationStateManager.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/StreamingAggregationStateManager.scala
@@ -15,12 +15,13 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming.state
+package org.apache.spark.sql.execution.streaming.operators.stateful
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeRow}
import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateUnsafeProjection, GenerateUnsafeRowJoiner}
import org.apache.spark.sql.catalyst.types.DataTypeUtils
+import org.apache.spark.sql.execution.streaming.state.{ReadStateStore, StateStore, UnsafeRowPair}
import org.apache.spark.sql.types.StructType
/**
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StreamingSessionWindowStateManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/StreamingSessionWindowStateManager.scala
similarity index 97%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StreamingSessionWindowStateManager.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/StreamingSessionWindowStateManager.scala
index 71df9dc65b419..a74b4aaf0da12 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StreamingSessionWindowStateManager.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/StreamingSessionWindowStateManager.scala
@@ -15,13 +15,14 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming.state
+package org.apache.spark.sql.execution.streaming.operators.stateful
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys._
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Attribute, Literal, UnsafeProjection, UnsafeRow}
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.execution.streaming.state.{ReadStateStore, StateStore, UnsafeRowPair}
import org.apache.spark.sql.types.{StructType, TimestampType}
import org.apache.spark.util.NextIterator
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/flatmapgroupswithstate/FlatMapGroupsWithStateExec.scala
similarity index 98%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/flatmapgroupswithstate/FlatMapGroupsWithStateExec.scala
index ebf08358c07bc..a0778fbfb6149 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/flatmapgroupswithstate/FlatMapGroupsWithStateExec.scala
@@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate
import java.util.concurrent.TimeUnit.NANOSECONDS
@@ -31,7 +31,8 @@ import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.plans.physical.Distribution
import org.apache.spark.sql.execution._
import org.apache.spark.sql.execution.metric.SQLMetric
-import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper._
+import org.apache.spark.sql.execution.streaming.operators.stateful.{StatefulOperatorPartitioning, StatefulOperatorStateInfo, StateStoreWriter, WatermarkSupport}
+import org.apache.spark.sql.execution.streaming.operators.stateful.join.StreamingSymmetricHashJoinHelper._
import org.apache.spark.sql.execution.streaming.state._
import org.apache.spark.sql.streaming.{GroupStateTimeout, OutputMode}
import org.apache.spark.sql.streaming.GroupStateTimeout.NoTimeout
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/FlatMapGroupsWithStateExecHelper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/flatmapgroupswithstate/FlatMapGroupsWithStateExecHelper.scala
similarity index 97%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/FlatMapGroupsWithStateExecHelper.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/flatmapgroupswithstate/FlatMapGroupsWithStateExecHelper.scala
index b68c08b3ea5c6..280fcfc0ca1ca 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/FlatMapGroupsWithStateExecHelper.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/flatmapgroupswithstate/FlatMapGroupsWithStateExecHelper.scala
@@ -15,13 +15,14 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming.state
+package org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes
import org.apache.spark.sql.execution.ObjectOperator
-import org.apache.spark.sql.execution.streaming.GroupStateImpl.NO_TIMESTAMP
+import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.GroupStateImpl.NO_TIMESTAMP
+import org.apache.spark.sql.execution.streaming.state.StateStore
import org.apache.spark.sql.types._
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/flatmapgroupswithstate/GroupStateImpl.scala
similarity index 97%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/flatmapgroupswithstate/GroupStateImpl.scala
index d4e93642b8164..1cab963802ca2 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/flatmapgroupswithstate/GroupStateImpl.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate
import java.sql.Date
import java.util.concurrent.TimeUnit
@@ -28,7 +28,7 @@ import org.apache.spark.api.java.Optional
import org.apache.spark.sql.catalyst.plans.logical.{EventTimeTimeout, NoTimeout, ProcessingTimeTimeout}
import org.apache.spark.sql.catalyst.util.IntervalUtils
import org.apache.spark.sql.errors.QueryExecutionErrors
-import org.apache.spark.sql.execution.streaming.GroupStateImpl._
+import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.GroupStateImpl._
import org.apache.spark.sql.streaming.{GroupStateTimeout, TestGroupState}
import org.apache.spark.unsafe.types.UTF8String
import org.apache.spark.util.Utils
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/join/StreamingSymmetricHashJoinExec.scala
similarity index 98%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/join/StreamingSymmetricHashJoinExec.scala
index 7d71db8d8e4b9..ef37185ce4166 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/join/StreamingSymmetricHashJoinExec.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.operators.stateful.join
import java.util.concurrent.TimeUnit.NANOSECONDS
@@ -29,9 +29,10 @@ import org.apache.spark.sql.catalyst.plans.physical._
import org.apache.spark.sql.catalyst.types.DataTypeUtils
import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan}
import org.apache.spark.sql.execution.metric.SQLMetric
-import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper._
+import org.apache.spark.sql.execution.streaming.operators.stateful.{SchemaValidationUtils, StatefulOperatorCustomMetric, StatefulOperatorCustomSumMetric, StatefulOperatorStateInfo, StatefulOperatorsUtils, StatefulOpStateStoreCheckpointInfo, StateStoreWriter, WatermarkSupport}
+import org.apache.spark.sql.execution.streaming.operators.stateful.join.StreamingSymmetricHashJoinHelper._
+import org.apache.spark.sql.execution.streaming.operators.stateful.join.SymmetricHashJoinStateManager.KeyToValuePair
import org.apache.spark.sql.execution.streaming.state._
-import org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager.KeyToValuePair
import org.apache.spark.sql.internal.{SessionState, SQLConf}
import org.apache.spark.sql.types.StructType
import org.apache.spark.util.{CompletionIterator, SerializableConfiguration}
@@ -236,7 +237,7 @@ case class StreamingSymmetricHashJoinExec(
case _ => throwBadJoinTypeException()
}
- override def shortName: String = "symmetricHashJoin"
+ override def shortName: String = StatefulOperatorsUtils.SYMMETRIC_HASH_JOIN_EXEC_OP_NAME
override val stateStoreNames: Seq[String] = _stateStoreNames
@@ -350,7 +351,7 @@ case class StreamingSymmetricHashJoinExec(
assert(stateInfo.isDefined, "State info not defined")
val checkpointIds = SymmetricHashJoinStateManager.getStateStoreCheckpointIds(
- partitionId, stateInfo.get, useVirtualColumnFamilies)
+ partitionId, stateInfo.get.stateStoreCkptIds, useVirtualColumnFamilies)
val inputSchema = left.output ++ right.output
val postJoinFilter =
@@ -362,12 +363,12 @@ case class StreamingSymmetricHashJoinExec(
new OneSideHashJoiner(
LeftSide, left.output, leftKeys, leftInputIter,
condition.leftSideOnly, postJoinFilter, stateWatermarkPredicates.left, partitionId,
- checkpointIds.left.keyToNumValues, checkpointIds.left.valueToNumKeys,
+ checkpointIds.left.keyToNumValues, checkpointIds.left.keyWithIndexToValue,
skippedNullValueCount, joinStateManagerStoreGenerator),
new OneSideHashJoiner(
RightSide, right.output, rightKeys, rightInputIter,
condition.rightSideOnly, postJoinFilter, stateWatermarkPredicates.right, partitionId,
- checkpointIds.right.keyToNumValues, checkpointIds.right.valueToNumKeys,
+ checkpointIds.right.keyToNumValues, checkpointIds.right.keyWithIndexToValue,
skippedNullValueCount, joinStateManagerStoreGenerator))
// Join one side input using the other side's buffered/state rows. Here is how it is done.
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinHelper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/join/StreamingSymmetricHashJoinHelper.scala
similarity index 97%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinHelper.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/join/StreamingSymmetricHashJoinHelper.scala
index 497e71070a09a..7b02a43cd5a9f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinHelper.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/join/StreamingSymmetricHashJoinHelper.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.operators.stateful.join
import scala.reflect.ClassTag
@@ -26,7 +26,8 @@ import org.apache.spark.sql.catalyst.analysis.StreamingJoinHelper
import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeSet, BoundReference, Expression, NamedExpression, PredicateHelper}
import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark._
import org.apache.spark.sql.execution.SparkPlan
-import org.apache.spark.sql.execution.streaming.WatermarkSupport.watermarkExpression
+import org.apache.spark.sql.execution.streaming.operators.stateful.{StatefulOperatorStateInfo, WatermarkSupport}
+import org.apache.spark.sql.execution.streaming.operators.stateful.WatermarkSupport.watermarkExpression
import org.apache.spark.sql.execution.streaming.state.{StateStoreCheckpointInfo, StateStoreCoordinatorRef, StateStoreProviderId}
@@ -323,15 +324,15 @@ object StreamingSymmetricHashJoinHelper extends Logging {
case class JoinerStateStoreCkptInfo(
keyToNumValues: StateStoreCheckpointInfo,
- valueToNumKeys: StateStoreCheckpointInfo)
+ keyWithIndexToValue: StateStoreCheckpointInfo)
case class JoinStateStoreCkptInfo(
left: JoinerStateStoreCkptInfo,
right: JoinerStateStoreCkptInfo)
case class JoinerStateStoreCheckpointId(
- keyToNumValues: Option[String],
- valueToNumKeys: Option[String])
+ keyToNumValues: Option[String],
+ keyWithIndexToValue: Option[String])
case class JoinStateStoreCheckpointId(
left: JoinerStateStoreCheckpointId,
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/join/SymmetricHashJoinStateManager.scala
similarity index 91%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/join/SymmetricHashJoinStateManager.scala
index 6ec197d7cc7b1..c0965747722e5 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/join/SymmetricHashJoinStateManager.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming.state
+package org.apache.spark.sql.execution.streaming.operators.stateful.join
import java.util.Locale
@@ -24,15 +24,16 @@ import scala.annotation.tailrec
import org.apache.hadoop.conf.Configuration
import org.apache.spark.TaskContext
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{END_INDEX, START_INDEX, STATE_STORE_ID}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, JoinedRow, Literal, SafeProjection, SpecificInternalRow, UnsafeProjection, UnsafeRow}
import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes
import org.apache.spark.sql.execution.metric.SQLMetric
-import org.apache.spark.sql.execution.streaming.StatefulOperatorStateInfo
-import org.apache.spark.sql.execution.streaming.StatefulOpStateStoreCheckpointInfo
-import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper._
+import org.apache.spark.sql.execution.streaming.operators.stateful.StatefulOperatorStateInfo
+import org.apache.spark.sql.execution.streaming.operators.stateful.StatefulOpStateStoreCheckpointInfo
+import org.apache.spark.sql.execution.streaming.operators.stateful.join.StreamingSymmetricHashJoinHelper._
+import org.apache.spark.sql.execution.streaming.state.{KeyStateEncoderSpec, NoPrefixKeyStateEncoderSpec, StateSchemaBroadcast, StateStore, StateStoreCheckpointInfo, StateStoreColFamilySchema, StateStoreConf, StateStoreErrors, StateStoreId, StateStoreMetrics, StateStoreProvider, StateStoreProviderId, SupportsFineGrainedReplay}
import org.apache.spark.sql.types.{BooleanType, LongType, StructField, StructType}
import org.apache.spark.util.NextIterator
@@ -439,7 +440,7 @@ abstract class SymmetricHashJoinStateManager(
* NOTE: this function is only intended for use in unit tests
* to simulate null values.
*/
- private[state] def updateNumValuesTestOnly(key: UnsafeRow, numValues: Long): Unit = {
+ private[streaming] def updateNumValuesTestOnly(key: UnsafeRow, numValues: Long): Unit = {
keyToNumValues.put(key, numValues)
}
@@ -528,7 +529,7 @@ abstract class SymmetricHashJoinStateManager(
* Helper class for representing data returned by [[KeyWithIndexToValueStore]].
* Designed for object reuse.
*/
- private[state] class KeyAndNumValues(var key: UnsafeRow = null, var numValue: Long = 0) {
+ private[join] class KeyAndNumValues(var key: UnsafeRow = null, var numValue: Long = 0) {
def withNew(newKey: UnsafeRow, newNumValues: Long): this.type = {
this.key = newKey
this.numValue = newNumValues
@@ -595,7 +596,7 @@ abstract class SymmetricHashJoinStateManager(
* Helper class for representing data returned by [[KeyWithIndexToValueStore]].
* Designed for object reuse.
*/
- private[state] class KeyWithIndexAndValue(
+ private[join] class KeyWithIndexAndValue(
var key: UnsafeRow = null,
var valueIndex: Long = -1,
var value: UnsafeRow = null,
@@ -737,9 +738,9 @@ abstract class SymmetricHashJoinStateManager(
if (useVirtualColumnFamilies) {
stateStore.createColFamilyIfAbsent(
colFamilyName,
- keySchema,
+ keyWithIndexSchema,
valueRowConverter.valueAttributes.toStructType,
- NoPrefixKeyStateEncoderSpec(keySchema)
+ NoPrefixKeyStateEncoderSpec(keyWithIndexSchema)
)
}
@@ -1134,17 +1135,17 @@ object SymmetricHashJoinStateManager {
val ckptIds = joinCkptInfo.left.keyToNumValues.stateStoreCkptId.map(
Array(
_,
- joinCkptInfo.left.valueToNumKeys.stateStoreCkptId.get,
+ joinCkptInfo.left.keyWithIndexToValue.stateStoreCkptId.get,
joinCkptInfo.right.keyToNumValues.stateStoreCkptId.get,
- joinCkptInfo.right.valueToNumKeys.stateStoreCkptId.get
+ joinCkptInfo.right.keyWithIndexToValue.stateStoreCkptId.get
)
)
val baseCkptIds = joinCkptInfo.left.keyToNumValues.baseStateStoreCkptId.map(
Array(
_,
- joinCkptInfo.left.valueToNumKeys.baseStateStoreCkptId.get,
+ joinCkptInfo.left.keyWithIndexToValue.baseStateStoreCkptId.get,
joinCkptInfo.right.keyToNumValues.baseStateStoreCkptId.get,
- joinCkptInfo.right.valueToNumKeys.baseStateStoreCkptId.get
+ joinCkptInfo.right.keyWithIndexToValue.baseStateStoreCkptId.get
)
)
@@ -1157,49 +1158,94 @@ object SymmetricHashJoinStateManager {
/**
* Stream-stream join has 4 state stores instead of one. So it will generate 4 different
- * checkpoint IDs. They are translated from each joiners' state store into an array through
- * mergeStateStoreCheckpointInfo(). This function is used to read it back into individual state
- * store checkpoint IDs.
- * @param partitionId
- * @param stateInfo
- * @return
+ * checkpoint IDs using stateStoreCkptIds. They are translated from each joiners' state
+ * store into an array through mergeStateStoreCheckpointInfo(). This function is used to read
+ * it back into individual state store checkpoint IDs for each store.
+ * If useColumnFamiliesForJoins is true, then it will always return the first checkpoint ID.
+ *
+ * @param partitionId the partition ID of the state store
+ * @param stateStoreCkptIds the array of checkpoint IDs for all the state stores
+ * @param useColumnFamiliesForJoins whether virtual column families are used for the join
+ *
+ * @return the checkpoint IDs for all state stores used by this joiner
*/
def getStateStoreCheckpointIds(
partitionId: Int,
- stateInfo: StatefulOperatorStateInfo,
+ stateStoreCkptIds: Option[Array[Array[String]]],
useColumnFamiliesForJoins: Boolean): JoinStateStoreCheckpointId = {
if (useColumnFamiliesForJoins) {
- val ckpt = stateInfo.stateStoreCkptIds.map(_(partitionId)).map(_.head)
+ val ckpt = stateStoreCkptIds.map(_(partitionId)).map(_.head)
JoinStateStoreCheckpointId(
- left = JoinerStateStoreCheckpointId(keyToNumValues = ckpt, valueToNumKeys = ckpt),
- right = JoinerStateStoreCheckpointId(keyToNumValues = ckpt, valueToNumKeys = ckpt)
+ left = JoinerStateStoreCheckpointId(keyToNumValues = ckpt, keyWithIndexToValue = ckpt),
+ right = JoinerStateStoreCheckpointId(keyToNumValues = ckpt, keyWithIndexToValue = ckpt)
)
} else {
- val stateStoreCkptIds = stateInfo.stateStoreCkptIds
+ val stateStoreCkptIdsOpt = stateStoreCkptIds
.map(_(partitionId))
.map(_.map(Option(_)))
.getOrElse(Array.fill[Option[String]](4)(None))
JoinStateStoreCheckpointId(
left = JoinerStateStoreCheckpointId(
- keyToNumValues = stateStoreCkptIds(0),
- valueToNumKeys = stateStoreCkptIds(1)),
+ keyToNumValues = stateStoreCkptIdsOpt(0),
+ keyWithIndexToValue = stateStoreCkptIdsOpt(1)),
right = JoinerStateStoreCheckpointId(
- keyToNumValues = stateStoreCkptIds(2),
- valueToNumKeys = stateStoreCkptIds(3)))
+ keyToNumValues = stateStoreCkptIdsOpt(2),
+ keyWithIndexToValue = stateStoreCkptIdsOpt(3)))
+ }
+ }
+
+ /**
+ * Stream-stream join has 4 state stores instead of one. So it will generate 4 different
+ * checkpoint IDs when not using virtual column families.
+ * This function is used to get the checkpoint ID for a specific state store
+ * by the name of the store, partition ID and the stateStoreCkptIds array. The expected names
+ * for the stores are generated by getStateStoreName().
+ * If useColumnFamiliesForJoins is true, then it will always return the first checkpoint ID.
+ *
+ * @param storeName the name of the state store
+ * @param partitionId the partition ID of the state store
+ * @param stateStoreCkptIds the array of checkpoint IDs for all the state stores
+ * @param useColumnFamiliesForJoins whether virtual column families are used for the join
+ *
+ * @return the checkpoint ID for the specific state store, or None if not found
+ */
+ def getStateStoreCheckpointId(
+ storeName: String,
+ partitionId: Int,
+ stateStoreCkptIds: Option[Array[Array[String]]],
+ useColumnFamiliesForJoins: Boolean = false) : Option[String] = {
+ if (useColumnFamiliesForJoins || storeName == StateStoreId.DEFAULT_STORE_NAME) {
+ stateStoreCkptIds.map(_(partitionId)).map(_.head)
+ } else {
+ val joinStateStoreCkptIds = getStateStoreCheckpointIds(
+ partitionId, stateStoreCkptIds, useColumnFamiliesForJoins)
+
+ if (storeName == getStateStoreName(LeftSide, KeyToNumValuesType)) {
+ joinStateStoreCkptIds.left.keyToNumValues
+ } else if (storeName == getStateStoreName(RightSide, KeyToNumValuesType)) {
+ joinStateStoreCkptIds.right.keyToNumValues
+ } else if (storeName == getStateStoreName(LeftSide, KeyWithIndexToValueType)) {
+ joinStateStoreCkptIds.left.keyWithIndexToValue
+ } else if (storeName == getStateStoreName(RightSide, KeyWithIndexToValueType)) {
+ joinStateStoreCkptIds.right.keyWithIndexToValue
+ } else {
+ None
+ }
}
}
- private[state] sealed trait StateStoreType
+ private[join] sealed trait StateStoreType
- private[state] case object KeyToNumValuesType extends StateStoreType {
+ private[join] case object KeyToNumValuesType extends StateStoreType {
override def toString(): String = "keyToNumValues"
}
- private[state] case object KeyWithIndexToValueType extends StateStoreType {
+ private[join] case object KeyWithIndexToValueType extends StateStoreType {
override def toString(): String = "keyWithIndexToValue"
}
- private[state] def getStateStoreName(joinSide: JoinSide, storeType: StateStoreType): String = {
+ private[join] def getStateStoreName(
+ joinSide: JoinSide, storeType: StateStoreType): String = {
s"$joinSide-$storeType"
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/statefulOperators.scala
similarity index 98%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/statefulOperators.scala
index d92e5dbae1aa1..cc8d354a03938 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/statefulOperators.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.operators.stateful
import java.util.UUID
import java.util.concurrent.TimeUnit._
@@ -324,8 +324,11 @@ trait StateStoreWriter
.map(_._2)
.toArray
assert(
- ret.length == getStateInfo.numPartitions,
- s"ChekpointInfo length: ${ret.length}, numPartitions: ${getStateInfo.numPartitions}")
+ // Normally, we should have checkpoint info for all partitions.
+ // However, for globalLimit operator, there is only one partition (0) that has state.
+ ret.length == getStateInfo.numPartitions
+ || (outputPartitioning.numPartitions == 1 && ret.length == 1),
+ s"CheckpointInfo length: ${ret.length}, numPartitions: ${getStateInfo.numPartitions}")
ret
}
@@ -1546,3 +1549,16 @@ trait SchemaValidationUtils extends Logging {
schemaEvolutionEnabled = usingAvro && schemaEvolutionEnabledForOperator))
}
}
+
+object StatefulOperatorsUtils {
+ val TRANSFORM_WITH_STATE_EXEC_OP_NAME = "transformWithStateExec"
+ val TRANSFORM_WITH_STATE_IN_PANDAS_EXEC_OP_NAME = "transformWithStateInPandasExec"
+ val TRANSFORM_WITH_STATE_IN_PYSPARK_EXEC_OP_NAME = "transformWithStateInPySparkExec"
+ // Seq of operator names who uses state schema v3 and TWS related options.
+ val TRANSFORM_WITH_STATE_OP_NAMES: Seq[String] = Seq(
+ TRANSFORM_WITH_STATE_EXEC_OP_NAME,
+ TRANSFORM_WITH_STATE_IN_PANDAS_EXEC_OP_NAME,
+ TRANSFORM_WITH_STATE_IN_PYSPARK_EXEC_OP_NAME
+ )
+ val SYMMETRIC_HASH_JOIN_EXEC_OP_NAME = "symmetricHashJoin"
+}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/streamingLimits.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/streamingLimits.scala
similarity index 98%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/streamingLimits.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/streamingLimits.scala
index 07fe0ae4357d0..6816be103f6e2 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/streamingLimits.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/streamingLimits.scala
@@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.operators.stateful
import java.util.concurrent.TimeUnit.NANOSECONDS
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateStoreColumnFamilySchemaUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/StateStoreColumnFamilySchemaUtils.scala
similarity index 95%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateStoreColumnFamilySchemaUtils.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/StateStoreColumnFamilySchemaUtils.scala
index 4401f8cedff6b..7e25960daf33d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateStoreColumnFamilySchemaUtils.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/StateStoreColumnFamilySchemaUtils.scala
@@ -14,14 +14,14 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate
import scala.collection.mutable
import org.apache.spark.sql.Encoder
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
-import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchemaUtils._
-import org.apache.spark.sql.execution.streaming.TransformWithStateVariableUtils.getRowCounterCFName
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateKeyValueRowSchemaUtils._
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateVariableUtils.getRowCounterCFName
import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, PrefixKeyScanStateEncoderSpec, RangeKeyScanStateEncoderSpec, StateStoreColFamilySchema}
import org.apache.spark.sql.types._
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/StateTypesEncoderUtils.scala
similarity index 95%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/StateTypesEncoderUtils.scala
index a2b7ee4ba7916..8ce300a40b43c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/StateTypesEncoderUtils.scala
@@ -15,13 +15,15 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate
import org.apache.spark.sql.Encoder
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder}
import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow}
-import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchemaUtils._
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateKeyValueRowSchemaUtils._
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.ttl.StateTTL
import org.apache.spark.sql.execution.streaming.state.StateStoreErrors
import org.apache.spark.sql.types._
@@ -180,7 +182,7 @@ class CompositeKeyStateEncoder[K, V](
stateName: String,
hasTtl: Boolean = false)
extends StateTypesEncoder[V](keyEncoder, valEncoder, stateName, hasTtl) {
- import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchemaUtils._
+ import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateKeyValueRowSchemaUtils._
/** Encoders */
private val userKeyExpressionEnc = encoderFor(userKeyEnc)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/TransformWithStateExec.scala
similarity index 96%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/TransformWithStateExec.scala
index 80fdaa1e71e2f..20e2c32015d84 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/TransformWithStateExec.scala
@@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate
import java.util.UUID
import java.util.concurrent.TimeUnit.NANOSECONDS
@@ -28,7 +28,10 @@ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, UnsafeRow}
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.execution._
-import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.StateStoreAwareZipPartitionsHelper
+import org.apache.spark.sql.execution.streaming.operators.stateful.{StatefulOperatorStateInfo, StatefulOperatorsUtils}
+import org.apache.spark.sql.execution.streaming.operators.stateful.join.StreamingSymmetricHashJoinHelper.StateStoreAwareZipPartitionsHelper
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.{DriverStatefulProcessorHandleImpl, ImplicitGroupingKeyTracker, StatefulProcessorHandleImpl, StatefulProcessorHandleState}
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.timers.{ExpiredTimerInfoImpl, TimerValuesImpl}
import org.apache.spark.sql.execution.streaming.state._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.streaming._
@@ -84,7 +87,7 @@ case class TransformWithStateExec(
initialState)
with ObjectProducerExec {
- override def shortName: String = "transformWithStateExec"
+ override def shortName: String = StatefulOperatorsUtils.TRANSFORM_WITH_STATE_EXEC_OP_NAME
// We need to just initialize key and value deserializer once per partition.
// The deserializers need to be lazily created on the executor since they
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExecBase.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/TransformWithStateExecBase.scala
similarity index 95%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExecBase.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/TransformWithStateExecBase.scala
index df68b21e0bb9f..c2d24e735ab2d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExecBase.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/TransformWithStateExecBase.scala
@@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate
import org.apache.spark.SparkThrowable
import org.apache.spark.sql.catalyst.InternalRow
@@ -22,6 +22,8 @@ import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, SortOrde
import org.apache.spark.sql.catalyst.plans.logical.{EventTime, ProcessingTime}
import org.apache.spark.sql.catalyst.plans.physical.Distribution
import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan}
+import org.apache.spark.sql.execution.streaming.operators.stateful.{StatefulOperatorCustomMetric, StatefulOperatorCustomSumMetric, StatefulOperatorPartitioning, StateStoreWriter, WatermarkSupport}
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker
import org.apache.spark.sql.execution.streaming.state.{OperatorStateMetadata, TransformWithStateUserFunctionException}
import org.apache.spark.sql.streaming.{OutputMode, TimeMode}
import org.apache.spark.sql.types.{BinaryType, StructType}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateVariableUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/TransformWithStateVariableUtils.scala
similarity index 96%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateVariableUtils.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/TransformWithStateVariableUtils.scala
index 7b9a478b8be19..068303e25e2e8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateVariableUtils.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/TransformWithStateVariableUtils.scala
@@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate
import org.json4s.DefaultFormats
import org.json4s.JsonAST._
@@ -23,7 +23,8 @@ import org.json4s.jackson.JsonMethods
import org.json4s.jackson.JsonMethods.{compact, render}
import org.apache.spark.internal.Logging
-import org.apache.spark.sql.execution.streaming.StateVariableType.StateVariableType
+import org.apache.spark.sql.execution.streaming.operators.stateful.{SchemaValidationUtils, StatefulOperatorStateInfo}
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.StateVariableType.StateVariableType
import org.apache.spark.sql.execution.streaming.state.{OperatorInfoV1, OperatorStateMetadata, OperatorStateMetadataV2, StateStoreErrors, StateStoreId, StateStoreMetadataV2}
import org.apache.spark.sql.streaming.{OutputMode, TimeMode}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statefulprocessor/StatefulProcessorHandleImpl.scala
similarity index 95%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statefulprocessor/StatefulProcessorHandleImpl.scala
index 3cf3286fafb80..dfba0e1f12146 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statefulprocessor/StatefulProcessorHandleImpl.scala
@@ -14,7 +14,8 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+// scalastyle:off line.size.limit
+package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor
import java.util
import java.util.UUID
@@ -26,13 +27,19 @@ import org.apache.spark.internal.Logging
import org.apache.spark.sql.Encoder
import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder}
import org.apache.spark.sql.execution.metric.SQLMetric
-import org.apache.spark.sql.execution.streaming.StatefulProcessorHandleState.PRE_INIT
-import org.apache.spark.sql.execution.streaming.StateVariableType._
-import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchemaUtils.{getExpirationMsRowSchema, getTTLRowKeySchema}
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.{StateStoreColumnFamilySchemaUtils, TimerKeyEncoder, TransformWithStateVariableInfo, TransformWithStateVariableUtils}
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.StateVariableType._
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateKeyValueRowSchemaUtils.{getExpirationMsRowSchema, getTTLRowKeySchema}
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.StatefulProcessorHandleState.PRE_INIT
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statevariables.{ListStateImpl, MapStateImpl, ValueStateImpl}
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.timers.{TimerStateImpl, TimerStateUtils}
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.ttl.{ListStateImplWithTTL, MapStateImplWithTTL, TTLState, ValueStateImplWithTTL}
+import org.apache.spark.sql.execution.streaming.runtime.{MicroBatchExecution, StreamExecution}
import org.apache.spark.sql.execution.streaming.state._
import org.apache.spark.sql.streaming.{ListState, MapState, QueryInfo, TimeMode, TTLConfig, ValueState}
import org.apache.spark.sql.types._
import org.apache.spark.util.Utils
+// scalastyle:on line.size.limit
/**
* Object used to assign/retrieve/remove grouping key passed implicitly for various state
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImplBase.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statefulprocessor/StatefulProcessorHandleImplBase.scala
similarity index 85%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImplBase.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statefulprocessor/StatefulProcessorHandleImplBase.scala
index 64d87073ccf9f..4b593fe309472 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImplBase.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statefulprocessor/StatefulProcessorHandleImplBase.scala
@@ -14,13 +14,15 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+// scalastyle:off line.size.limit
+package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.plans.logical.NoTime
-import org.apache.spark.sql.execution.streaming.StatefulProcessorHandleState.{INITIALIZED, PRE_INIT, StatefulProcessorHandleState, TIMER_PROCESSED}
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.StatefulProcessorHandleState.{INITIALIZED, PRE_INIT, StatefulProcessorHandleState, TIMER_PROCESSED}
import org.apache.spark.sql.execution.streaming.state.StateStoreErrors
import org.apache.spark.sql.streaming.{StatefulProcessorHandle, TimeMode}
+// scalastyle:on line.size.limit
abstract class StatefulProcessorHandleImplBase(
timeMode: TimeMode, keyExprEnc: ExpressionEncoder[Any]) extends StatefulProcessorHandle {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statevariables/ListStateImpl.scala
similarity index 93%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImpl.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statevariables/ListStateImpl.scala
index 32683aebd8c18..c6fe03da49ddf 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImpl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statevariables/ListStateImpl.scala
@@ -14,14 +14,18 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+// scalastyle:off line.size.limit
+package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statevariables
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.StateTypesEncoder
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.TWSMetricsUtils
import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, StateStore, StateStoreErrors}
import org.apache.spark.sql.streaming.ListState
import org.apache.spark.sql.types.StructType
+// scalastyle:on line.size.limit
/**
* Provides concrete implementation for list of values associated with a state variable
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateMetricsImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statevariables/ListStateMetricsImpl.scala
similarity index 91%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateMetricsImpl.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statevariables/ListStateMetricsImpl.scala
index 66b6bd063ae6b..2275aba1a5887 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateMetricsImpl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statevariables/ListStateMetricsImpl.scala
@@ -14,13 +14,15 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+// scalastyle:off line.size.limit
+package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statevariables
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection, UnsafeRow}
-import org.apache.spark.sql.execution.streaming.TransformWithStateVariableUtils.getRowCounterCFName
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateVariableUtils.getRowCounterCFName
import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, StateStore}
import org.apache.spark.sql.types._
+// scalastyle:on line.size.limit
/**
* Trait that provides helper methods to maintain metrics for a list state.
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MapStateImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statevariables/MapStateImpl.scala
similarity index 88%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MapStateImpl.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statevariables/MapStateImpl.scala
index 4e608a5d5dbbe..007791ea55477 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MapStateImpl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statevariables/MapStateImpl.scala
@@ -14,15 +14,19 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+// scalastyle:off line.size.limit
+package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statevariables
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.execution.metric.SQLMetric
-import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchemaUtils._
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.CompositeKeyStateEncoder
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateKeyValueRowSchemaUtils._
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.TWSMetricsUtils
import org.apache.spark.sql.execution.streaming.state.{PrefixKeyScanStateEncoderSpec, StateStore, StateStoreErrors, UnsafeRowPair}
import org.apache.spark.sql.streaming.MapState
import org.apache.spark.sql.types.StructType
+// scalastyle:on line.size.limit
/**
* Class that provides a concrete implementation for map state associated with state
@@ -56,7 +60,10 @@ class MapStateImpl[K, V](
/** Whether state exists or not. */
override def exists(): Boolean = {
- store.prefixScan(stateTypesEncoder.encodeGroupingKey(), stateName).nonEmpty
+ val iter = store.prefixScan(stateTypesEncoder.encodeGroupingKey(), stateName)
+ val result = iter.nonEmpty
+ iter.close()
+ result
}
/** Get the state value if it exists */
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statevariables/ValueStateImpl.scala
similarity index 89%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statevariables/ValueStateImpl.scala
index d7e7a3d83f4df..2a4a79bca1e0a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statevariables/ValueStateImpl.scala
@@ -14,13 +14,17 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+// scalastyle:off line.size.limit
+package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statevariables
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.StateTypesEncoder
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.TWSMetricsUtils
import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, StateStore}
import org.apache.spark.sql.streaming.ValueState
+// scalastyle:on line.size.limit
/**
* Class that provides a concrete implementation for a single value state associated with state
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ExpiredTimerInfoImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/timers/ExpiredTimerInfoImpl.scala
similarity index 93%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ExpiredTimerInfoImpl.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/timers/ExpiredTimerInfoImpl.scala
index 984d650a27ccc..ec602988dc3e1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ExpiredTimerInfoImpl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/timers/ExpiredTimerInfoImpl.scala
@@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.timers
import org.apache.spark.sql.streaming.ExpiredTimerInfo
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TimerStateImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/timers/TimerStateImpl.scala
similarity index 94%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TimerStateImpl.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/timers/TimerStateImpl.scala
index ede45fc924a98..647c0b3036a2a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TimerStateImpl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/timers/TimerStateImpl.scala
@@ -14,13 +14,15 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.timers
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{EXPIRY_TIMESTAMP, KEY}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TimerKeyEncoder
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker
import org.apache.spark.sql.execution.streaming.state._
import org.apache.spark.sql.streaming.TimeMode
import org.apache.spark.sql.types._
@@ -199,7 +201,9 @@ class TimerStateImpl(
}
}
- override protected def close(): Unit = { }
+ override protected def close(): Unit = {
+ iter.close()
+ }
}
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TimerValuesImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/timers/TimerValuesImpl.scala
similarity index 94%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TimerValuesImpl.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/timers/TimerValuesImpl.scala
index eb65aade67e36..234b80a84e8ed 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TimerValuesImpl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/timers/TimerValuesImpl.scala
@@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.timers
import org.apache.spark.sql.streaming.TimerValues
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/ttl/ListStateImplWithTTL.scala
similarity index 96%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/ttl/ListStateImplWithTTL.scala
index 4e32b80578155..08f97e38bd086 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/ttl/ListStateImplWithTTL.scala
@@ -14,12 +14,13 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.ttl
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.expressions.UnsafeRow
import org.apache.spark.sql.execution.metric.SQLMetric
-import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchemaUtils._
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.StateTypesEncoder
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateKeyValueRowSchemaUtils._
import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, StateStore, StateStoreErrors}
import org.apache.spark.sql.streaming.{ListState, TTLConfig}
import org.apache.spark.util.NextIterator
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MapStateImplWithTTL.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/ttl/MapStateImplWithTTL.scala
similarity index 94%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MapStateImplWithTTL.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/ttl/MapStateImplWithTTL.scala
index 64581006555e7..f063354bc8c8c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MapStateImplWithTTL.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/ttl/MapStateImplWithTTL.scala
@@ -14,12 +14,14 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.ttl
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.execution.metric.SQLMetric
-import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchemaUtils._
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.CompositeKeyStateEncoder
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateKeyValueRowSchemaUtils._
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.TWSMetricsUtils
import org.apache.spark.sql.execution.streaming.state.{PrefixKeyScanStateEncoderSpec, StateStore, StateStoreErrors}
import org.apache.spark.sql.streaming.{MapState, TTLConfig}
import org.apache.spark.util.NextIterator
@@ -128,7 +130,9 @@ metrics: Map[String, SQLMetric])
}
}
- override protected def close(): Unit = {}
+ override protected def close(): Unit = {
+ unsafeRowPairIterator.close()
+ }
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/ttl/TTLState.scala
similarity index 98%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/ttl/TTLState.scala
index 398088ab16978..548a47ea75e13 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/ttl/TTLState.scala
@@ -14,14 +14,16 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.ttl
import java.time.Duration
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection, UnsafeRow}
import org.apache.spark.sql.execution.metric.SQLMetric
-import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchemaUtils._
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateKeyValueRowSchemaUtils._
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TTLEncoder
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.TWSMetricsUtils
import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, RangeKeyScanStateEncoderSpec, StateStore}
import org.apache.spark.sql.streaming.TTLConfig
import org.apache.spark.sql.types._
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/ttl/ValueStateImplWithTTL.scala
similarity index 94%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/ttl/ValueStateImplWithTTL.scala
index 836aac4cdb5a0..587da75993610 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/ttl/ValueStateImplWithTTL.scala
@@ -14,11 +14,12 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.ttl
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.execution.metric.SQLMetric
-import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchemaUtils._
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.StateTypesEncoder
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateKeyValueRowSchemaUtils._
import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, StateStore}
import org.apache.spark.sql.streaming.{TTLConfig, ValueState}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AcceptsLatestSeenOffsetHandler.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AcceptsLatestSeenOffsetHandler.scala
similarity index 91%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AcceptsLatestSeenOffsetHandler.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AcceptsLatestSeenOffsetHandler.scala
index bf4333d8e9dd6..b15b93b47ada4 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AcceptsLatestSeenOffsetHandler.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AcceptsLatestSeenOffsetHandler.scala
@@ -15,10 +15,12 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import org.apache.spark.SparkUnsupportedOperationException
import org.apache.spark.sql.connector.read.streaming.{AcceptsLatestSeenOffset, SparkDataStream}
+import org.apache.spark.sql.execution.streaming.Source
+import org.apache.spark.sql.execution.streaming.checkpointing.OffsetSeq
/**
* This feeds "latest seen offset" to the sources that implement AcceptsLatestSeenOffset.
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncLogPurge.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AsyncLogPurge.scala
similarity index 98%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncLogPurge.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AsyncLogPurge.scala
index cb7e71bda84dd..43d5a50b2ccda 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncLogPurge.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AsyncLogPurge.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import java.util.concurrent.atomic.AtomicBoolean
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AsyncProgressTrackingMicroBatchExecution.scala
similarity index 97%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecution.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AsyncProgressTrackingMicroBatchExecution.scala
index bbfedd5454deb..2a87ba3380883 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecution.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AsyncProgressTrackingMicroBatchExecution.scala
@@ -15,16 +15,18 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import java.util.concurrent._
import java.util.concurrent.atomic.AtomicLong
import org.apache.spark.internal.LogKeys.{BATCH_ID, PRETTY_ID_STRING}
-import org.apache.spark.internal.MDC
import org.apache.spark.sql.catalyst.streaming.WriteToStream
import org.apache.spark.sql.classic.SparkSession
import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.execution.streaming.{AvailableNowTrigger, OneTimeTrigger, ProcessingTimeTrigger}
+import org.apache.spark.sql.execution.streaming.checkpointing.{AsyncCommitLog, AsyncOffsetSeqLog, CommitMetadata, OffsetSeq}
+import org.apache.spark.sql.execution.streaming.operators.stateful.StateStoreWriter
import org.apache.spark.sql.streaming.Trigger
import org.apache.spark.util.{Clock, ThreadUtils}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncStreamingQueryCheckpointMetadata.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AsyncStreamingQueryCheckpointMetadata.scala
similarity index 93%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncStreamingQueryCheckpointMetadata.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AsyncStreamingQueryCheckpointMetadata.scala
index e3a5ded83eb17..65113ce15c397 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncStreamingQueryCheckpointMetadata.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AsyncStreamingQueryCheckpointMetadata.scala
@@ -14,11 +14,12 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import java.util.concurrent.ThreadPoolExecutor
import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.execution.streaming.checkpointing.{AsyncCommitLog, AsyncOffsetSeqLog}
import org.apache.spark.util.Clock
/**
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AvailableNowDataStreamWrapper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AvailableNowDataStreamWrapper.scala
similarity index 96%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AvailableNowDataStreamWrapper.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AvailableNowDataStreamWrapper.scala
index f42250c3c702d..9a5cf89d03fab 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AvailableNowDataStreamWrapper.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AvailableNowDataStreamWrapper.scala
@@ -15,12 +15,13 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{DELEGATE, READ_LIMIT}
import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, ReadLimit, SparkDataStream, SupportsAdmissionControl, SupportsTriggerAvailableNow}
import org.apache.spark.sql.connector.read.streaming
+import org.apache.spark.sql.execution.streaming.Source
/**
* This class wraps a [[SparkDataStream]] and makes it support Trigger.AvailableNow, by overriding
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AvailableNowMicroBatchStreamWrapper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AvailableNowMicroBatchStreamWrapper.scala
similarity index 96%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AvailableNowMicroBatchStreamWrapper.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AvailableNowMicroBatchStreamWrapper.scala
index 3df358ae9bf80..ec63033edd40b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AvailableNowMicroBatchStreamWrapper.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AvailableNowMicroBatchStreamWrapper.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import org.apache.spark.SparkUnsupportedOperationException
import org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFactory}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AvailableNowSourceWrapper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AvailableNowSourceWrapper.scala
similarity index 92%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AvailableNowSourceWrapper.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AvailableNowSourceWrapper.scala
index af5713144ebb2..bca4324447718 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AvailableNowSourceWrapper.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AvailableNowSourceWrapper.scala
@@ -15,10 +15,11 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import org.apache.spark.SparkUnsupportedOperationException
import org.apache.spark.sql.DataFrame
+import org.apache.spark.sql.execution.streaming.{Offset, Source}
import org.apache.spark.sql.types.StructType
/**
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/CompactibleFileStreamLog.scala
similarity index 98%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/CompactibleFileStreamLog.scala
index d6770452e71f3..8a90982b7c0c6 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/CompactibleFileStreamLog.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import java.io.{FileNotFoundException, InputStream, IOException, OutputStream}
import java.nio.charset.StandardCharsets.UTF_8
@@ -27,9 +27,10 @@ import org.apache.hadoop.fs.Path
import org.json4s.{Formats, NoTypeHints}
import org.json4s.jackson.Serialization
-import org.apache.spark.internal.{LogKeys, MDC}
+import org.apache.spark.internal.LogKeys
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.execution.streaming.checkpointing.HDFSMetadataLog
import org.apache.spark.util.Utils
/**
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ContinuousRecordEndpoint.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ContinuousRecordEndpoint.scala
similarity index 97%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ContinuousRecordEndpoint.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ContinuousRecordEndpoint.scala
index 11bdfee460e66..d948325c9ae3b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ContinuousRecordEndpoint.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ContinuousRecordEndpoint.scala
@@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import scala.collection.mutable
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ErrorNotifier.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ErrorNotifier.scala
similarity index 96%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ErrorNotifier.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ErrorNotifier.scala
index 0f25d0667a0ef..19867c5168373 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ErrorNotifier.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ErrorNotifier.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import java.util.concurrent.atomic.AtomicReference
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamOptions.scala
similarity index 99%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamOptions.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamOptions.scala
index b259f9dbcdcb2..9ceee21a871c5 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamOptions.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamOptions.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import java.util.Locale
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamSource.scala
similarity index 99%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamSource.scala
index 465973cabe587..d5503f1c247da 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamSource.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import java.util.concurrent.ThreadPoolExecutor
import java.util.concurrent.TimeUnit._
@@ -27,7 +27,7 @@ import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileStatus, FileSystem, GlobFilter, Path}
import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.spark.internal.{Logging, LogKeys, MDC}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.paths.SparkPath
import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
import org.apache.spark.sql.classic.{DataFrame, Dataset, SparkSession}
@@ -35,6 +35,8 @@ import org.apache.spark.sql.connector.read.streaming
import org.apache.spark.sql.connector.read.streaming.{ReadAllAvailable, ReadLimit, ReadMaxBytes, ReadMaxFiles, SupportsAdmissionControl, SupportsTriggerAvailableNow}
import org.apache.spark.sql.errors.QueryExecutionErrors
import org.apache.spark.sql.execution.datasources.{DataSource, InMemoryFileIndex, LogicalRelation}
+import org.apache.spark.sql.execution.streaming.{Offset, Source}
+import org.apache.spark.sql.execution.streaming.sinks.FileStreamSink
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.StructType
import org.apache.spark.util.ArrayImplicits._
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamSourceLog.scala
similarity index 96%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamSourceLog.scala
index 14653864a2922..451cf41152421 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamSourceLog.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import java.util.{LinkedHashMap => JLinkedHashMap}
import java.util.Map.Entry
@@ -23,7 +23,8 @@ import java.util.Map.Entry
import scala.collection.mutable
import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.execution.streaming.FileStreamSource.FileEntry
+import org.apache.spark.sql.execution.streaming.checkpointing.HDFSMetadataLog
+import org.apache.spark.sql.execution.streaming.runtime.FileStreamSource.FileEntry
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.util.ArrayImplicits._
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceOffset.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamSourceOffset.scala
similarity index 94%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceOffset.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamSourceOffset.scala
index ba79c77f38677..2cfece1942f0c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceOffset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamSourceOffset.scala
@@ -15,13 +15,15 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import scala.util.control.Exception._
import org.json4s.{Formats, NoTypeHints}
import org.json4s.jackson.Serialization
+import org.apache.spark.sql.execution.streaming.Offset
+
/**
* Offset for the [[FileStreamSource]].
*
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/IncrementalExecution.scala
similarity index 96%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/IncrementalExecution.scala
index b6701182d7e06..0d4b0f0941d9d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/IncrementalExecution.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import java.util.UUID
import java.util.concurrent.atomic.AtomicInteger
@@ -24,7 +24,7 @@ import scala.collection.mutable.{Map => MutableMap}
import org.apache.hadoop.fs.Path
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{BATCH_TIMESTAMP, ERROR}
import org.apache.spark.sql.catalyst.QueryPlanningTracker
import org.apache.spark.sql.catalyst.expressions.{CurrentBatchTimestamp, ExpressionWithRandomSeed}
@@ -38,6 +38,11 @@ import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, MergingSessi
import org.apache.spark.sql.execution.datasources.v2.state.metadata.StateMetadataPartitionReader
import org.apache.spark.sql.execution.exchange.ShuffleExchangeLike
import org.apache.spark.sql.execution.python.streaming.{FlatMapGroupsInPandasWithStateExec, TransformWithStateInPySparkExec}
+import org.apache.spark.sql.execution.streaming.checkpointing.{CheckpointFileManager, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.operators.stateful.{SessionWindowStateStoreRestoreExec, SessionWindowStateStoreSaveExec, StatefulOperator, StatefulOperatorStateInfo, StateStoreRestoreExec, StateStoreSaveExec, StateStoreWriter, StreamingDeduplicateExec, StreamingDeduplicateWithinWatermarkExec, StreamingGlobalLimitExec, StreamingLocalLimitExec, UpdateEventTimeColumnExec}
+import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.FlatMapGroupsWithStateExec
+import org.apache.spark.sql.execution.streaming.operators.stateful.join.{StreamingSymmetricHashJoinExec, StreamingSymmetricHashJoinHelper}
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateExec
import org.apache.spark.sql.execution.streaming.sources.WriteToMicroBatchDataSourceV1
import org.apache.spark.sql.execution.streaming.state.{OperatorStateMetadataReader, OperatorStateMetadataV1, OperatorStateMetadataV2, OperatorStateMetadataWriter, StateSchemaBroadcast, StateSchemaMetadata}
import org.apache.spark.sql.internal.SQLConf
@@ -73,7 +78,9 @@ class IncrementalExecution(
MutableMap[Long, StateSchemaBroadcast](),
mode: CommandExecutionMode.Value = CommandExecutionMode.ALL,
val isTerminatingTrigger: Boolean = false)
- extends QueryExecution(sparkSession, logicalPlan, mode = mode) with Logging {
+ extends QueryExecution(sparkSession, logicalPlan, mode = mode,
+ shuffleCleanupMode =
+ QueryExecution.determineShuffleCleanupMode(sparkSession.sessionState.conf)) with Logging {
// Modified planner with stateful operations.
override val planner: SparkPlanner = new SparkPlanner(
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/LongOffset.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/LongOffset.scala
similarity index 92%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/LongOffset.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/LongOffset.scala
index a27898cb0c9fc..19588082b3c54 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/LongOffset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/LongOffset.scala
@@ -15,7 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
+
+import org.apache.spark.sql.execution.streaming.Offset
/**
* A simple offset for sources that produce a single linear stream of data.
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MetadataLogFileIndex.scala
similarity index 95%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileIndex.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MetadataLogFileIndex.scala
index 45bb69a9c056b..32e2a0e53dd46 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileIndex.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MetadataLogFileIndex.scala
@@ -15,16 +15,16 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import scala.collection.mutable
import org.apache.hadoop.fs.{FileStatus, Path}
import org.apache.spark.internal.LogKeys._
-import org.apache.spark.internal.MDC
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.execution.datasources._
+import org.apache.spark.sql.execution.streaming.sinks.{FileStreamSink, FileStreamSinkLog}
import org.apache.spark.sql.types.StructType
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetricsReporter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MetricsReporter.scala
similarity index 98%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetricsReporter.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MetricsReporter.scala
index 3919f8a2eb21f..0789ecee8533a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetricsReporter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MetricsReporter.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import java.time.ZonedDateTime
import java.time.format.DateTimeFormatter
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MicroBatchExecution.scala
similarity index 90%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MicroBatchExecution.scala
index 1dd70ad985cc2..461936b40218c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MicroBatchExecution.scala
@@ -15,15 +15,19 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import scala.collection.mutable.{Map => MutableMap}
import scala.collection.mutable
+import scala.util.control.NonFatal
-import org.apache.spark.internal.{LogKeys, MDC}
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.internal.LogKeys
+import org.apache.spark.internal.LogKeys.BATCH_ID
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, CurrentBatchTimestamp, CurrentDate, CurrentTimestamp, FileSourceMetadataAttribute, LocalTimestamp}
-import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LocalRelation, LogicalPlan, Project, StreamSourceAwareLogicalPlan}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, GlobalLimit, LeafNode, LocalRelation, LogicalPlan, Project, StreamSourceAwareLogicalPlan}
import org.apache.spark.sql.catalyst.streaming.{StreamingRelationV2, WriteToStream}
import org.apache.spark.sql.catalyst.trees.TreePattern.CURRENT_LIKE
import org.apache.spark.sql.catalyst.util.truncatedString
@@ -35,8 +39,13 @@ import org.apache.spark.sql.errors.QueryExecutionErrors
import org.apache.spark.sql.execution.{SparkPlan, SQLExecution}
import org.apache.spark.sql.execution.datasources.LogicalRelation
import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, StreamingDataSourceV2Relation, StreamingDataSourceV2ScanRelation, StreamWriterCommitProgress, WriteToDataSourceV2Exec}
-import org.apache.spark.sql.execution.streaming.sources.{WriteToMicroBatchDataSource, WriteToMicroBatchDataSourceV1}
-import org.apache.spark.sql.execution.streaming.state.StateSchemaBroadcast
+import org.apache.spark.sql.execution.streaming.{AvailableNowTrigger, Offset, OneTimeTrigger, ProcessingTimeTrigger, Sink, Source}
+import org.apache.spark.sql.execution.streaming.checkpointing.{CheckpointFileManager, CommitMetadata, OffsetSeq, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.operators.stateful.{StatefulOperatorStateInfo, StatefulOpStateStoreCheckpointInfo, StateStoreWriter}
+import org.apache.spark.sql.execution.streaming.runtime.AcceptsLatestSeenOffsetHandler
+import org.apache.spark.sql.execution.streaming.runtime.StreamingCheckpointConstants.{DIR_NAME_COMMITS, DIR_NAME_OFFSETS, DIR_NAME_STATE}
+import org.apache.spark.sql.execution.streaming.sources.{ForeachBatchSink, WriteToMicroBatchDataSource, WriteToMicroBatchDataSourceV1}
+import org.apache.spark.sql.execution.streaming.state.{StateSchemaBroadcast, StateStoreErrors}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.streaming.Trigger
import org.apache.spark.util.{Clock, Utils}
@@ -303,6 +312,15 @@ class MicroBatchExecution(
}
private val watermarkPropagator = WatermarkPropagator(sparkSession.sessionState.conf)
+ private lazy val hasGlobalAggregateOrLimit = containsGlobalAggregateOrLimit(logicalPlan)
+
+ private def containsGlobalAggregateOrLimit(logicalPlan: LogicalPlan): Boolean = {
+ logicalPlan.collect {
+ case agg: Aggregate if agg.groupingExpressions.isEmpty => agg
+ case limit: GlobalLimit => limit
+ }.nonEmpty
+ }
+
override def cleanup(): Unit = {
super.cleanup()
@@ -547,12 +565,40 @@ class MicroBatchExecution(
log"offsets ${MDC(LogKeys.STREAMING_OFFSETS_START, execCtx.startOffsets)} and " +
log"available offsets ${MDC(LogKeys.STREAMING_OFFSETS_END, execCtx.endOffsets)}")
case None => // We are starting this stream for the first time.
+ val shouldVerifyNewCheckpointDirectory =
+ sparkSession.conf.get(SQLConf.STREAMING_VERIFY_CHECKPOINT_DIRECTORY_EMPTY_ON_START)
+ if (shouldVerifyNewCheckpointDirectory) {
+ verifyNewCheckpointDirectory()
+ }
logInfo(s"Starting new streaming query.")
execCtx.batchId = 0
watermarkTracker = WatermarkTracker(sparkSessionToRunBatches.conf, logicalPlan)
}
}
+ /**
+ * Verify that the checkpoint directory is in a good state to start a new
+ * streaming query. This checks that the offsets, state, commits directories are
+ * either non-existent or empty.
+ *
+ * If this check fails, an exception is thrown.
+ */
+ private def verifyNewCheckpointDirectory(): Unit = {
+ val fileManager = CheckpointFileManager.create(new Path(resolvedCheckpointRoot),
+ sparkSession.sessionState.newHadoopConf())
+ val dirNamesThatShouldNotHaveFiles = Array[String](
+ DIR_NAME_OFFSETS, DIR_NAME_STATE, DIR_NAME_COMMITS)
+
+ dirNamesThatShouldNotHaveFiles.foreach { dirName =>
+ val path = new Path(resolvedCheckpointRoot, dirName)
+
+ if (fileManager.exists(path) && !fileManager.list(path).isEmpty) {
+ val loc = path.toString
+ throw StateStoreErrors.streamingStateCheckpointLocationNotEmpty(loc)
+ }
+ }
+ }
+
/**
* Returns true if there is any new data available to be processed.
*/
@@ -862,6 +908,8 @@ class MicroBatchExecution(
isTerminatingTrigger = trigger.isInstanceOf[AvailableNowTrigger.type])
execCtx.executionPlan.executedPlan // Force the lazy generation of execution plan
}
+ // Set up StateStore commit tracking before execution begins
+ setupStateStoreCommitTracking(execCtx)
markMicroBatchExecutionStart(execCtx)
@@ -965,6 +1013,50 @@ class MicroBatchExecution(
}
}
+
+ /**
+ * Set up tracking for StateStore commits before batch execution begins.
+ * This collects information about expected stateful operators and initializes
+ * commit tracking, but only for ForeachBatchSink without global aggregates or limits.
+ */
+ private def setupStateStoreCommitTracking(execCtx: MicroBatchExecutionContext): Unit = {
+ try {
+ // Collect stateful operators from the executed plan
+ val statefulOps = execCtx.executionPlan.executedPlan.collect {
+ case s: StateStoreWriter => s
+ }
+
+ if (statefulOps.nonEmpty &&
+ sparkSession.sessionState.conf.stateStoreCommitValidationEnabled) {
+
+ // Start tracking before execution begins
+ // We only validate commits for ForeachBatchSink because it's the only sink where
+ // user-defined functions can cause partial processing (e.g., using show() or limit()).
+ // We exclude queries with global aggregates or limits because they naturally don't
+ // process all partitions, making commit validation unnecessary and potentially noisy.
+ if (sink.isInstanceOf[ForeachBatchSink[_]] && !hasGlobalAggregateOrLimit) {
+ progressReporter.shouldValidateStateStoreCommit.set(true)
+ // Build expected stores map: operatorId -> (storeName -> numPartitions)
+ val expectedStores = statefulOps.map { op =>
+ val operatorId = op.getStateInfo.operatorId
+ val numPartitions = op.getStateInfo.numPartitions
+ val storeNames = op.stateStoreNames.map(_ -> numPartitions).toMap
+ operatorId -> storeNames
+ }.toMap
+ sparkSession.streams.stateStoreCoordinator
+ .startStateStoreCommitTrackingForBatch(runId, execCtx.batchId, expectedStores)
+ }
+ // TODO: Find out how to dynamically set the SQLConf at this point to disable
+ // the commit tracking
+ }
+ } catch {
+ case NonFatal(e) =>
+ // Log but don't fail the query for tracking setup errors
+ logWarning(log"Error during StateStore commit tracking setup for batch " +
+ log"${MDC(BATCH_ID, execCtx.batchId)}", e)
+ }
+ }
+
/**
* Called after the microbatch has completed execution. It takes care of committing the offset
* to commit log and other bookkeeping.
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ProgressReporter.scala
similarity index 97%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ProgressReporter.scala
index dc04ba3331e71..19aa068869dc2 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ProgressReporter.scala
@@ -15,17 +15,18 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import java.time.Instant
import java.time.ZoneId
import java.time.format.DateTimeFormatter
import java.util.{Optional, UUID}
+import java.util.concurrent.atomic.AtomicBoolean
import scala.collection.mutable
import scala.jdk.CollectionConverters._
-import org.apache.spark.internal.{Logging, LogKeys, MDC}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.optimizer.InlineCTE
import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, LogicalPlan, WithCTE}
@@ -36,6 +37,8 @@ import org.apache.spark.sql.connector.catalog.Table
import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, ReportsSinkMetrics, ReportsSourceMetrics, SparkDataStream}
import org.apache.spark.sql.execution.{QueryExecution, StreamSourceAwareSparkPlan}
import org.apache.spark.sql.execution.datasources.v2.{MicroBatchScanExec, StreamingDataSourceV2ScanRelation, StreamWriterCommitProgress}
+import org.apache.spark.sql.execution.streaming.checkpointing.OffsetSeqMetadata
+import org.apache.spark.sql.execution.streaming.operators.stateful.{EventTimeWatermarkExec, StateStoreWriter}
import org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorRef
import org.apache.spark.sql.streaming._
import org.apache.spark.sql.streaming.StreamingQueryListener.{QueryIdleEvent, QueryProgressEvent}
@@ -56,6 +59,8 @@ class ProgressReporter(
// The timestamp we report an event that has not executed anything
var lastNoExecutionProgressEventTime = Long.MinValue
+ val shouldValidateStateStoreCommit = new AtomicBoolean(false)
+
/** Holds the most recent query progress updates. Accesses must lock on the queue itself. */
private val progressBuffer = new mutable.Queue[StreamingQueryProgress]()
@@ -277,6 +282,15 @@ abstract class ProgressContext(
currentTriggerStartOffsets != null && currentTriggerEndOffsets != null &&
currentTriggerLatestOffsets != null
)
+
+ // Only validate commits if enabled and the query has stateful operators
+ if (progressReporter.shouldValidateStateStoreCommit.get()) {
+ progressReporter.stateStoreCoordinator.validateStateStoreCommitForBatch(
+ lastExecution.runId,
+ lastExecution.currentBatchId
+ )
+ }
+
currentTriggerEndTimestamp = triggerClock.getTimeMillis()
val processingTimeMills = currentTriggerEndTimestamp - currentTriggerStartTimestamp
assert(lastExecution != null, "executed batch should provide the information for execution.")
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamOffset.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/RateStreamOffset.scala
similarity index 95%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamOffset.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/RateStreamOffset.scala
index 84f0961e4af12..2ec50b604ac8f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamOffset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/RateStreamOffset.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import org.json4s.DefaultFormats
import org.json4s.jackson.Serialization
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ResolveWriteToStream.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ResolveWriteToStream.scala
similarity index 97%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ResolveWriteToStream.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ResolveWriteToStream.scala
index 6c0a228ec2fca..ee7bf67eb9121 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ResolveWriteToStream.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ResolveWriteToStream.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import java.util.UUID
@@ -24,13 +24,13 @@ import scala.util.control.NonFatal
import org.apache.hadoop.fs.Path
import org.apache.spark.internal.LogKeys.{CHECKPOINT_LOCATION, CHECKPOINT_ROOT, CONFIG, PATH}
-import org.apache.spark.internal.MDC
import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.catalyst.streaming.{WriteToStream, WriteToStreamStatement}
import org.apache.spark.sql.connector.catalog.SupportsWrite
import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.util.Utils
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/SerializedOffset.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/SerializedOffset.scala
similarity index 91%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/SerializedOffset.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/SerializedOffset.scala
index 129cfed860eb6..e84ac4c1a1ed8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/SerializedOffset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/SerializedOffset.scala
@@ -15,7 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
+
+import org.apache.spark.sql.execution.streaming.Offset
/**
* Used when loading a JSON serialized offset from external storage.
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamExecution.scala
similarity index 98%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamExecution.scala
index 8dd055949289a..17095bcabf32a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamExecution.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import java.io.{InterruptedIOException, UncheckedIOException}
import java.nio.channels.ClosedByInterruptException
@@ -33,7 +33,7 @@ import org.apache.hadoop.fs.Path
import org.apache.logging.log4j.CloseableThreadContext
import org.apache.spark.{JobArtifactSet, SparkContext, SparkException, SparkThrowable}
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{CHECKPOINT_PATH, CHECKPOINT_ROOT, LOGICAL_PLAN, PATH, PRETTY_ID_STRING, QUERY_ID, RUN_ID, SPARK_DATA_STREAM}
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._
@@ -43,6 +43,8 @@ import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2, ReadLi
import org.apache.spark.sql.connector.write.{LogicalWriteInfoImpl, SupportsTruncate, Write}
import org.apache.spark.sql.execution.SparkPlan
import org.apache.spark.sql.execution.command.StreamingExplainCommand
+import org.apache.spark.sql.execution.streaming.checkpointing.{CheckpointFileManager, CommitLog, OffsetSeqLog, OffsetSeqMetadata}
+import org.apache.spark.sql.execution.streaming.operators.stateful.{StatefulOperator, StateStoreWriter}
import org.apache.spark.sql.execution.streaming.sources.{ForeachBatchUserFuncException, ForeachUserFuncException}
import org.apache.spark.sql.execution.streaming.state.OperatorStateMetadataV2FileManager
import org.apache.spark.sql.internal.SQLConf
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecutionContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamExecutionContext.scala
similarity index 98%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecutionContext.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamExecutionContext.scala
index c5e14df3e20e1..b177bbdede740 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecutionContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamExecutionContext.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import java.util.UUID
@@ -24,6 +24,7 @@ import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.connector.catalog.Table
import org.apache.spark.sql.connector.read.streaming.SparkDataStream
+import org.apache.spark.sql.execution.streaming.checkpointing.OffsetSeqMetadata
import org.apache.spark.util.Clock
/**
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetadata.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamMetadata.scala
similarity index 90%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetadata.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamMetadata.scala
index 84519150ca42b..04e2dfb7833e6 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetadata.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamMetadata.scala
@@ -15,22 +15,23 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import java.io.{InputStreamReader, OutputStreamWriter}
import java.nio.charset.StandardCharsets
import scala.util.control.NonFatal
-import org.apache.commons.io.IOUtils
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileAlreadyExistsException, FSDataInputStream, Path}
import org.json4s.{Formats, NoTypeHints}
import org.json4s.jackson.Serialization
-import org.apache.spark.internal.{Logging, LogKeys, MDC}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.sql.errors.QueryExecutionErrors
-import org.apache.spark.sql.execution.streaming.CheckpointFileManager.CancellableFSDataOutputStream
+import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager
+import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager.CancellableFSDataOutputStream
+import org.apache.spark.util.Utils
/**
* Contains metadata associated with a [[org.apache.spark.sql.streaming.StreamingQuery]].
@@ -63,7 +64,7 @@ object StreamMetadata extends Logging {
logError(log"Error reading stream metadata from ${MDC(LogKeys.PATH, metadataFile)}", e)
throw e
} finally {
- IOUtils.closeQuietly(input)
+ Utils.closeQuietly(input)
}
} else None
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamProgress.scala
similarity index 92%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamProgress.scala
index 02f52bb30e1f3..a6fd103e8d6a3 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamProgress.scala
@@ -15,11 +15,12 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import scala.collection.immutable
import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2, SparkDataStream}
+import org.apache.spark.sql.execution.streaming.checkpointing.{OffsetSeq, OffsetSeqMetadata}
/**
* A helper class that looks like a Map[Source, Offset].
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingCheckpointConstants.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingCheckpointConstants.scala
similarity index 94%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingCheckpointConstants.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingCheckpointConstants.scala
index 27c76ba6bd67b..b92d862e17889 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingCheckpointConstants.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingCheckpointConstants.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
object StreamingCheckpointConstants {
val DIR_NAME_COMMITS = "commits"
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryCheckpointMetadata.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingQueryCheckpointMetadata.scala
similarity index 95%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryCheckpointMetadata.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingQueryCheckpointMetadata.scala
index 072ccb21e514a..cc7e92d9d7d1c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryCheckpointMetadata.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingQueryCheckpointMetadata.scala
@@ -14,13 +14,14 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import java.util.UUID
import org.apache.hadoop.fs.Path
import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.execution.streaming.checkpointing.{CommitLog, OffsetSeqLog}
/**
* An interface for accessing the checkpoint metadata associated with a streaming query.
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingQueryListenerBus.scala
similarity index 99%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingQueryListenerBus.scala
index b3014ebc818da..d3762eada387e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingQueryListenerBus.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import java.util.UUID
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryWrapper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingQueryWrapper.scala
similarity index 98%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryWrapper.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingQueryWrapper.scala
index c59b9584f5383..5585962ae16a0 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryWrapper.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingQueryWrapper.scala
@@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import java.util.UUID
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingRelation.scala
similarity index 98%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingRelation.scala
index c5d5a79d34545..96f6340cb958d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingRelation.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.SparkSession
@@ -29,6 +29,7 @@ import org.apache.spark.sql.connector.read.streaming.SparkDataStream
import org.apache.spark.sql.errors.QueryExecutionErrors
import org.apache.spark.sql.execution.LeafExecNode
import org.apache.spark.sql.execution.datasources.{DataSource, FileFormat}
+import org.apache.spark.sql.execution.streaming.Source
import org.apache.spark.sql.sources.SupportsStreamSourceMetadataColumns
object StreamingRelation {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/TriggerExecutor.scala
similarity index 95%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/TriggerExecutor.scala
index bfa838e43e288..8a175e633d49d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/TriggerExecutor.scala
@@ -15,10 +15,11 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{ELAPSED_TIME, TRIGGER_INTERVAL}
+import org.apache.spark.sql.execution.streaming.ProcessingTimeTrigger
import org.apache.spark.util.{Clock, SystemClock}
trait TriggerExecutor {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/WatermarkPropagator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/WatermarkPropagator.scala
similarity index 98%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/WatermarkPropagator.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/WatermarkPropagator.scala
index aaf8cbd69ea20..b3d5baf0b5af0 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/WatermarkPropagator.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/WatermarkPropagator.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import java.{util => jutil}
@@ -24,6 +24,7 @@ import scala.collection.mutable
import org.apache.spark.internal.Logging
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.streaming.operators.stateful.{EventTimeWatermarkExec, StateStoreWriter}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.util.Utils
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/WatermarkTracker.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/WatermarkTracker.scala
similarity index 97%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/WatermarkTracker.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/WatermarkTracker.scala
index 7228767c4d18a..6d94630d8c3bd 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/WatermarkTracker.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/WatermarkTracker.scala
@@ -15,17 +15,18 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import java.util.{Locale, UUID}
import scala.collection.mutable
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys._
import org.apache.spark.sql.RuntimeConfig
import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, LogicalPlan}
import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.streaming.operators.stateful.EventTimeWatermarkExec
import org.apache.spark.sql.internal.SQLConf
/**
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/memory.scala
similarity index 99%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/memory.scala
index 48af1972e581c..68eb3cc7688d2 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/memory.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.runtime
import java.util
import java.util.concurrent.atomic.AtomicInteger
@@ -37,6 +37,7 @@ import org.apache.spark.sql.classic.ClassicConversions.castToImpl
import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability}
import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory, Scan, ScanBuilder}
import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream, Offset => OffsetV2, ReadLimit, SparkDataStream, SupportsTriggerAvailableNow}
+import org.apache.spark.sql.execution.streaming.Offset
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.connector.SimpleTableProvider
import org.apache.spark.sql.types.StructType
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sinks/FileStreamSink.scala
similarity index 96%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sinks/FileStreamSink.scala
index 11e764821c71a..978ec99ae9b39 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sinks/FileStreamSink.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.sinks
import scala.util.control.NonFatal
@@ -23,7 +23,7 @@ import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.spark.SparkException
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{BATCH_ID, ERROR, PATH}
import org.apache.spark.internal.io.FileCommitProtocol
import org.apache.spark.sql.{DataFrame, SparkSession}
@@ -31,6 +31,8 @@ import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.classic.ClassicConversions.castToImpl
import org.apache.spark.sql.errors.QueryExecutionErrors
import org.apache.spark.sql.execution.datasources.{BasicWriteJobStatsTracker, FileFormat, FileFormatWriter}
+import org.apache.spark.sql.execution.streaming.{ManifestFileCommitProtocol, Sink}
+import org.apache.spark.sql.execution.streaming.runtime.StreamExecution
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.util.{SerializableConfiguration, Utils}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sinks/FileStreamSinkLog.scala
similarity index 97%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLog.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sinks/FileStreamSinkLog.scala
index 556438811c44d..71679e284f9d9 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLog.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sinks/FileStreamSinkLog.scala
@@ -15,14 +15,14 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution.streaming
+package org.apache.spark.sql.execution.streaming.sinks
import org.apache.hadoop.fs.FileStatus
import org.apache.spark.internal.LogKeys._
-import org.apache.spark.internal.MDC
import org.apache.spark.paths.SparkPath
import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.execution.streaming.runtime.CompactibleFileStreamLog
import org.apache.spark.sql.internal.SQLConf
/**
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ContinuousMemoryStream.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ContinuousMemoryStream.scala
index d0ba95ffd8a23..03884d02faeb7 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ContinuousMemoryStream.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ContinuousMemoryStream.scala
@@ -32,7 +32,8 @@ import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.UnsafeRow
import org.apache.spark.sql.connector.read.InputPartition
import org.apache.spark.sql.connector.read.streaming.{ContinuousPartitionReader, ContinuousPartitionReaderFactory, ContinuousStream, Offset, PartitionOffset}
-import org.apache.spark.sql.execution.streaming.{Offset => _, _}
+import org.apache.spark.sql.execution.streaming.{Offset => _}
+import org.apache.spark.sql.execution.streaming.runtime.{ContinuousRecordEndpoint, ContinuousRecordPartitionOffset, GetRecord, MemoryStreamBase}
import org.apache.spark.util.RpcUtils
/**
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchStream.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchStream.scala
index 6705201c67316..80d3d97b12ed7 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchStream.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchStream.scala
@@ -21,9 +21,7 @@ import java.io._
import java.nio.charset.StandardCharsets
import java.util.concurrent.TimeUnit
-import org.apache.commons.io.IOUtils
-
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys._
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
@@ -31,9 +29,10 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory}
import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset}
import org.apache.spark.sql.errors.QueryExecutionErrors
-import org.apache.spark.sql.execution.streaming._
+import org.apache.spark.sql.execution.streaming.checkpointing.HDFSMetadataLog
+import org.apache.spark.sql.execution.streaming.runtime._
import org.apache.spark.sql.util.CaseInsensitiveStringMap
-import org.apache.spark.util.{ManualClock, SystemClock}
+import org.apache.spark.util.{ManualClock, SystemClock, Utils}
class RateStreamMicroBatchStream(
rowsPerSecond: Long,
@@ -71,7 +70,7 @@ class RateStreamMicroBatchStream(
}
override def deserialize(in: InputStream): LongOffset = {
- val content = IOUtils.toString(new InputStreamReader(in, StandardCharsets.UTF_8))
+ val content = Utils.toString(in)
// HDFSMetadataLog guarantees that it never creates a partial file.
assert(content.length != 0)
if (content(0) == 'v') {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketMicroBatchStream.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketMicroBatchStream.scala
index 597b981ebe556..243d6c784f5ae 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketMicroBatchStream.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketMicroBatchStream.scala
@@ -25,13 +25,13 @@ import javax.annotation.concurrent.GuardedBy
import scala.collection.mutable.ListBuffer
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{HOST, PORT}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory}
import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset}
-import org.apache.spark.sql.execution.streaming.LongOffset
+import org.apache.spark.sql.execution.streaming.runtime.LongOffset
import org.apache.spark.unsafe.types.UTF8String
/**
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala
index 25a1ca249c223..ac7f1a021960d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala
@@ -26,18 +26,16 @@ import scala.collection.mutable
import scala.jdk.CollectionConverters._
import scala.util.control.NonFatal
-import com.google.common.io.ByteStreams
-import org.apache.commons.io.IOUtils
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs._
-import org.apache.spark.{SparkConf, SparkEnv, SparkException}
-import org.apache.spark.internal.{Logging, LogKeys, MDC, MessageWithContext}
+import org.apache.spark.{SparkConf, SparkEnv, SparkException, TaskContext}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.io.CompressionCodec
import org.apache.spark.sql.catalyst.expressions.UnsafeRow
import org.apache.spark.sql.errors.QueryExecutionErrors
-import org.apache.spark.sql.execution.streaming.CheckpointFileManager
-import org.apache.spark.sql.execution.streaming.CheckpointFileManager.CancellableFSDataOutputStream
+import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager
+import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager.CancellableFSDataOutputStream
import org.apache.spark.sql.types.StructType
import org.apache.spark.util.{SizeEstimator, Utils}
import org.apache.spark.util.ArrayImplicits._
@@ -83,19 +81,24 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
override def get(key: UnsafeRow, colFamilyName: String): UnsafeRow = map.get(key)
- override def iterator(colFamilyName: String): Iterator[UnsafeRowPair] = {
- map.iterator()
+ override def iterator(colFamilyName: String): StateStoreIterator[UnsafeRowPair] = {
+ val iter = map.iterator()
+ new StateStoreIterator(iter)
}
override def abort(): Unit = {}
+ override def release(): Unit = {}
+
override def toString(): String = {
- s"HDFSReadStateStore[id=(op=${id.operatorId},part=${id.partitionId}),dir=$baseDir]"
+ s"HDFSReadStateStore[stateStoreId=$stateStoreId_, version=$version]"
}
- override def prefixScan(prefixKey: UnsafeRow, colFamilyName: String):
- Iterator[UnsafeRowPair] = {
- map.prefixScan(prefixKey)
+ override def prefixScan(
+ prefixKey: UnsafeRow,
+ colFamilyName: String): StateStoreIterator[UnsafeRowPair] = {
+ val iter = map.prefixScan(prefixKey)
+ new StateStoreIterator(iter)
}
override def valuesIterator(key: UnsafeRow, colFamilyName: String): Iterator[UnsafeRow] = {
@@ -112,6 +115,15 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
case object UPDATING extends STATE
case object COMMITTED extends STATE
case object ABORTED extends STATE
+ case object RELEASED extends STATE
+
+ Option(TaskContext.get()).foreach { ctxt =>
+ ctxt.addTaskCompletionListener[Unit](ctx => {
+ if (state == UPDATING) {
+ abort()
+ }
+ })
+ }
private val newVersion = version + 1
@volatile private var state: STATE = UPDATING
@@ -173,6 +185,12 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
logInfo(log"Committed version ${MDC(LogKeys.COMMITTED_VERSION, newVersion)} " +
log"for ${MDC(LogKeys.STATE_STORE_PROVIDER, this)} to file " +
log"${MDC(LogKeys.FILE_NAME, finalDeltaFile)}")
+
+ // Report the commit to StateStoreCoordinator for tracking
+ if (storeConf.commitValidationEnabled) {
+ StateStore.reportCommitToCoordinator(newVersion, stateStoreId, hadoopConf)
+ }
+
newVersion
} catch {
case e: Throwable =>
@@ -198,15 +216,18 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
* Get an iterator of all the store data.
* This can be called only after committing all the updates made in the current thread.
*/
- override def iterator(colFamilyName: String): Iterator[UnsafeRowPair] = {
+ override def iterator(colFamilyName: String): StateStoreIterator[UnsafeRowPair] = {
assertUseOfDefaultColFamily(colFamilyName)
- mapToUpdate.iterator()
+ val iter = mapToUpdate.iterator()
+ new StateStoreIterator(iter)
}
- override def prefixScan(prefixKey: UnsafeRow, colFamilyName: String):
- Iterator[UnsafeRowPair] = {
+ override def prefixScan(
+ prefixKey: UnsafeRow,
+ colFamilyName: String): StateStoreIterator[UnsafeRowPair] = {
assertUseOfDefaultColFamily(colFamilyName)
- mapToUpdate.prefixScan(prefixKey)
+ val iter = mapToUpdate.prefixScan(prefixKey)
+ new StateStoreIterator(iter)
}
override def metrics: StateStoreMetrics = {
@@ -244,7 +265,7 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
}
override def toString(): String = {
- s"HDFSStateStore[id=(op=${id.operatorId},part=${id.partitionId}),dir=$baseDir]"
+ s"HDFSStateStore[stateStoreId=$stateStoreId_, version=$version]"
}
override def removeColFamilyIfExists(colFamilyName: String): Boolean = {
@@ -351,7 +372,7 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
assert(
!storeConf.enableStateStoreCheckpointIds,
"HDFS State Store Provider doesn't support checkpointFormatVersion >= 2 " +
- s"checkpointFormatVersion ${storeConf.sqlConf.stateStoreCheckpointFormatVersion}")
+ s"checkpointFormatVersion ${storeConf.stateStoreCheckpointFormatVersion}")
this.stateStoreId_ = stateStoreId
this.keySchema = keySchema
@@ -360,6 +381,9 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
this.hadoopConf = hadoopConf
this.numberOfVersionsToRetainInMemory = storeConf.maxVersionsToRetainInMemory
+ val queryRunId = UUID.fromString(StateStoreProvider.getRunId(hadoopConf))
+ this.stateStoreProviderId = StateStoreProviderId(stateStoreId, queryRunId)
+
// run a bunch of validation checks for this state store provider
runValidation(useColumnFamilies, useMultipleValuesPerKey, keyStateEncoderSpec)
@@ -370,6 +394,8 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
override def stateStoreId: StateStoreId = stateStoreId_
+ override protected def logName: String = s"${super.logName} ${stateStoreProviderId}"
+
/** Do maintenance backing data files, including creating snapshots and cleaning up old files */
override def doMaintenance(): Unit = {
try {
@@ -377,7 +403,7 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
cleanup()
} catch {
case NonFatal(e) =>
- logWarning(log"Error performing snapshot and cleaning up " + toMessageWithContext)
+ logWarning(log"Error performing snapshot and cleaning up")
}
}
@@ -396,14 +422,8 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
override def supportedInstanceMetrics: Seq[StateStoreInstanceMetric] =
Seq(instanceMetricSnapshotLastUpload)
- private def toMessageWithContext: MessageWithContext = {
- log"HDFSStateStoreProvider[id = (op=${MDC(LogKeys.OP_ID, stateStoreId.operatorId)}," +
- log"part=${MDC(LogKeys.PARTITION_ID, stateStoreId.partitionId)})," +
- log"dir = ${MDC(LogKeys.PATH, baseDir)}]"
- }
-
override def toString(): String = {
- toMessageWithContext.message
+ s"HDFSStateStoreProvider[stateStoreProviderId=$stateStoreProviderId]"
}
/* Internal fields and methods */
@@ -415,6 +435,7 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
@volatile private var hadoopConf: Configuration = _
@volatile private var numberOfVersionsToRetainInMemory: Int = _
@volatile private var numColsPrefixKey: Int = 0
+ @volatile private var stateStoreProviderId: StateStoreProviderId = _
// TODO: The validation should be moved to a higher level so that it works for all state store
// implementations
@@ -640,7 +661,7 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
fileToRead, toString(), keySize)
} else {
val keyRowBuffer = new Array[Byte](keySize)
- ByteStreams.readFully(input, keyRowBuffer, 0, keySize)
+ Utils.readFully(input, keyRowBuffer, 0, keySize)
val keyRow = new UnsafeRow(keySchema.fields.length)
keyRow.pointTo(keyRowBuffer, keySize)
@@ -650,7 +671,7 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
map.remove(keyRow)
} else {
val valueRowBuffer = new Array[Byte](valueSize)
- ByteStreams.readFully(input, valueRowBuffer, 0, valueSize)
+ Utils.readFully(input, valueRowBuffer, 0, valueSize)
val valueRow = new UnsafeRow(valueSchema.fields.length)
// If valueSize in existing file is not multiple of 8, floor it to multiple of 8.
// This is a workaround for the following:
@@ -720,7 +741,7 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
rawStream: CancellableFSDataOutputStream): Unit = {
try {
if (rawStream != null) rawStream.cancel()
- IOUtils.closeQuietly(compressedStream)
+ Utils.closeQuietly(compressedStream)
} catch {
// Closing the compressedStream causes the stream to write/flush flush data into the
// rawStream. Since the rawStream is already closed, there may be errors.
@@ -760,7 +781,7 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
fileToRead, toString(), keySize)
} else {
val keyRowBuffer = new Array[Byte](keySize)
- ByteStreams.readFully(input, keyRowBuffer, 0, keySize)
+ Utils.readFully(input, keyRowBuffer, 0, keySize)
val keyRow = new UnsafeRow(keySchema.fields.length)
keyRow.pointTo(keyRowBuffer, keySize)
@@ -771,7 +792,7 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
fileToRead, toString(), valueSize)
} else {
val valueRowBuffer = new Array[Byte](valueSize)
- ByteStreams.readFully(input, valueRowBuffer, 0, valueSize)
+ Utils.readFully(input, valueRowBuffer, 0, valueSize)
val valueRow = new UnsafeRow(valueSchema.fields.length)
// If valueSize in existing file is not multiple of 8, floor it to multiple of 8.
// This is a workaround for the following:
@@ -821,7 +842,7 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
}
} catch {
case NonFatal(e) =>
- logWarning(log"Error doing snapshots for " + toMessageWithContext, e)
+ logWarning(log"Error doing snapshots", e)
}
}
@@ -854,7 +875,7 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
}
} catch {
case NonFatal(e) =>
- logWarning(log"Error cleaning up files for " + toMessageWithContext, e)
+ logWarning(log"Error cleaning up files", e)
}
}
@@ -908,7 +929,7 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
case "snapshot" =>
versionToFiles.put(version, StoreFile(version, path, isSnapshot = true))
case _ => logWarning(
- log"Could not identify file ${MDC(LogKeys.PATH, path)} for " + toMessageWithContext)
+ log"Could not identify file ${MDC(LogKeys.PATH, path)}")
}
}
}
@@ -953,7 +974,8 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
* @param endVersion checkpoint version to end with
* @return [[HDFSBackedStateStore]]
*/
- override def replayStateFromSnapshot(snapshotVersion: Long, endVersion: Long): StateStore = {
+ override def replayStateFromSnapshot(
+ snapshotVersion: Long, endVersion: Long, readOnly: Boolean): StateStore = {
val newMap = replayLoadedMapFromSnapshot(snapshotVersion, endVersion)
logInfo(log"Retrieved snapshot at version " +
log"${MDC(LogKeys.STATE_STORE_VERSION, snapshotVersion)} and apply delta files to version " +
@@ -1057,11 +1079,9 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
/** Reports to the coordinator the store's latest snapshot version */
private def reportSnapshotUploadToCoordinator(version: Long): Unit = {
if (storeConf.reportSnapshotUploadLag) {
- // Attach the query run ID and current timestamp to the RPC message
- val runId = UUID.fromString(StateStoreProvider.getRunId(hadoopConf))
val currentTimestamp = System.currentTimeMillis()
StateStoreProvider.coordinatorRef.foreach(
- _.snapshotUploaded(StateStoreProviderId(stateStoreId, runId), version, currentTimestamp)
+ _.snapshotUploaded(stateStoreProviderId, version, currentTimestamp)
)
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadata.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadata.scala
index 1c97e95847908..c34545216fdaf 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadata.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadata.scala
@@ -28,12 +28,14 @@ import org.json4s.{Formats, JBool, JObject, NoTypeHints}
import org.json4s.jackson.JsonMethods.{compact, render}
import org.json4s.jackson.Serialization
-import org.apache.spark.internal.{Logging, LogKeys, MDC}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceErrors
-import org.apache.spark.sql.execution.streaming.{CheckpointFileManager, CommitLog, MetadataVersionUtil, StateStoreWriter, StreamingQueryCheckpointMetadata}
-import org.apache.spark.sql.execution.streaming.CheckpointFileManager.CancellableFSDataOutputStream
-import org.apache.spark.sql.execution.streaming.StreamingCheckpointConstants.DIR_NAME_OFFSETS
+import org.apache.spark.sql.execution.streaming.checkpointing.{CheckpointFileManager, CommitLog, MetadataVersionUtil}
+import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager.CancellableFSDataOutputStream
+import org.apache.spark.sql.execution.streaming.operators.stateful.StateStoreWriter
+import org.apache.spark.sql.execution.streaming.runtime.StreamingCheckpointConstants.DIR_NAME_OFFSETS
+import org.apache.spark.sql.execution.streaming.runtime.StreamingQueryCheckpointMetadata
import org.apache.spark.sql.execution.streaming.state.OperatorStateMetadataUtils.{OperatorStateMetadataReader, OperatorStateMetadataWriter}
/**
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala
index 6b3bec2077037..85e2d72ec1630 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala
@@ -21,14 +21,13 @@ import java.io.File
import java.util.Locale
import java.util.Set
import java.util.UUID
-import java.util.concurrent.{ConcurrentHashMap, ConcurrentLinkedQueue, TimeUnit}
+import java.util.concurrent.{ConcurrentHashMap, ConcurrentLinkedQueue}
import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger, AtomicLong}
-import javax.annotation.concurrent.GuardedBy
import scala.collection.{mutable, Map}
import scala.jdk.CollectionConverters.ConcurrentMapHasAsScala
-import scala.ref.WeakReference
import scala.util.Try
+import scala.util.control.NonFatal
import org.apache.hadoop.conf.Configuration
import org.json4s.{Formats, NoTypeHints}
@@ -38,7 +37,7 @@ import org.rocksdb.CompressionType._
import org.rocksdb.TickerType._
import org.apache.spark.TaskContext
-import org.apache.spark.internal.{LogEntry, Logging, LogKeys, MDC}
+import org.apache.spark.internal.{LogEntry, Logging, LogKeys}
import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
import org.apache.spark.sql.errors.QueryExecutionErrors
import org.apache.spark.util.{NextIterator, Utils}
@@ -75,7 +74,8 @@ class RocksDB(
useColumnFamilies: Boolean = false,
enableStateStoreCheckpointIds: Boolean = false,
partitionId: Int = 0,
- eventForwarder: Option[RocksDBEventForwarder] = None) extends Logging {
+ eventForwarder: Option[RocksDBEventForwarder] = None,
+ uniqueId: Option[String] = None) extends Logging {
import RocksDB._
@@ -158,7 +158,7 @@ class RocksDB(
private val byteArrayPair = new ByteArrayPair()
private val commitLatencyMs = new mutable.HashMap[String, Long]()
- private val acquireLock = new Object
+ private val loadMetrics = new mutable.HashMap[String, Long]()
@volatile private var db: NativeRocksDB = _
@volatile private var changelogWriter: Option[StateStoreChangelogWriter] = None
@@ -185,6 +185,25 @@ class RocksDB(
protected var sessionStateStoreCkptId: Option[String] = None
protected[sql] val lineageManager: RocksDBLineageManager = new RocksDBLineageManager
+ // Memory tracking fields for unmanaged memory monitoring
+ // This allows the UnifiedMemoryManager to track RocksDB memory usage without
+ // directly accessing RocksDB from the polling thread, avoiding segmentation faults
+
+ // Timestamp of the last memory usage update in milliseconds.
+ // Used to enforce the update interval and prevent excessive memory queries.
+ private val lastMemoryUpdateTime = new AtomicLong(0L)
+
+ // Minimum interval between memory usage updates in milliseconds.
+ // This prevents performance impact from querying RocksDB memory too frequently.
+ private val memoryUpdateIntervalMs = conf.memoryUpdateIntervalMs
+
+ // Generate a unique ID if not provided to ensure proper memory tracking
+ private val instanceUniqueId = uniqueId.getOrElse(UUID.randomUUID().toString)
+
+ // Register with RocksDBMemoryManager
+ // Initial registration with zero memory usage
+ RocksDBMemoryManager.updateMemoryUsage(instanceUniqueId, 0L, conf.boundedMemoryUsage)
+
@volatile private var numKeysOnLoadedVersion = 0L
@volatile private var numKeysOnWritingVersion = 0L
@@ -195,24 +214,14 @@ class RocksDB(
// SPARK-46249 - Keep track of recorded metrics per version which can be used for querying later
// Updates and access to recordedMetrics are protected by the DB instance lock
- @GuardedBy("acquireLock")
@volatile private var recordedMetrics: Option[RocksDBMetrics] = None
- @GuardedBy("acquireLock")
- @volatile private var acquiredThreadInfo: AcquiredThreadInfo = _
-
- // This is accessed and updated only between load and commit
- // which means it is implicitly guarded by acquireLock
- @GuardedBy("acquireLock")
private val colFamilyNameToInfoMap = new ConcurrentHashMap[String, ColumnFamilyInfo]()
- @GuardedBy("acquireLock")
private val colFamilyIdToNameMap = new ConcurrentHashMap[Short, String]()
- @GuardedBy("acquireLock")
private val maxColumnFamilyId: AtomicInteger = new AtomicInteger(-1)
- @GuardedBy("acquireLock")
private val shouldForceSnapshot: AtomicBoolean = new AtomicBoolean(false)
private def getColumnFamilyInfo(cfName: String): ColumnFamilyInfo = {
@@ -300,11 +309,6 @@ class RocksDB(
colFamilyNameToInfoMap.asScala.values.toSeq.count(_.isInternal == isInternal)
}
- // Mapping of local SST files to DFS files for file reuse.
- // This mapping should only be updated using the Task thread - at version load and commit time.
- // If same mapping instance is updated from different threads,
- // it will result in undefined behavior (and most likely incorrect mapping state).
- @GuardedBy("acquireLock")
private val rocksDBFileMapping: RocksDBFileMapping = new RocksDBFileMapping()
// We send snapshots that needs to be uploaded by the maintenance thread to this queue
@@ -583,9 +587,13 @@ class RocksDB(
version: Long,
stateStoreCkptId: Option[String] = None,
readOnly: Boolean = false): RocksDB = {
+ val startTime = System.currentTimeMillis()
+
assert(version >= 0)
- acquire(LoadStore)
recordedMetrics = None
+ // Reset the load metrics before loading
+ loadMetrics.clear()
+
logInfo(log"Loading ${MDC(LogKeys.VERSION_NUM, version)} with stateStoreCkptId: ${
MDC(LogKeys.UUID, stateStoreCkptId.getOrElse(""))}")
if (stateStoreCkptId.isDefined || enableStateStoreCheckpointIds && version == 0) {
@@ -593,6 +601,15 @@ class RocksDB(
} else {
loadWithoutCheckpointId(version, readOnly)
}
+
+ // Record the metrics after loading
+ val duration = System.currentTimeMillis() - startTime
+ loadMetrics ++= Map(
+ "load" -> duration
+ )
+ // Register with memory manager after successful load
+ updateMemoryUsageIfNeeded()
+
this
}
@@ -608,9 +625,12 @@ class RocksDB(
* Source.
*/
def loadFromSnapshot(snapshotVersion: Long, endVersion: Long): RocksDB = {
+ val startTime = System.currentTimeMillis()
+
assert(snapshotVersion >= 0 && endVersion >= snapshotVersion)
- acquire(LoadStore)
recordedMetrics = None
+ loadMetrics.clear()
+
logInfo(
log"Loading snapshot at version ${MDC(LogKeys.VERSION_NUM, snapshotVersion)} and apply " +
log"changelog files to version ${MDC(LogKeys.VERSION_NUM, endVersion)}.")
@@ -627,6 +647,12 @@ class RocksDB(
}
// Report this snapshot version to the coordinator
reportSnapshotUploadToCoordinator(snapshotVersion)
+
+ // Record the metrics after loading
+ loadMetrics ++= Map(
+ "loadFromSnapshot" -> (System.currentTimeMillis() - startTime)
+ )
+
this
}
@@ -644,6 +670,18 @@ class RocksDB(
workingDir, rocksDBFileMapping)
loadedVersion = snapshotVersion
lastSnapshotVersion = snapshotVersion
+
+ setInitialCFInfo()
+ metadata.columnFamilyMapping.foreach { mapping =>
+ mapping.foreach { case (colFamilyName, cfId) =>
+ addToColFamilyMaps(colFamilyName, cfId, isInternalColFamily(colFamilyName, metadata))
+ }
+ }
+
+ metadata.maxColumnFamilyId.foreach { maxId =>
+ maxColumnFamilyId.set(maxId)
+ }
+
openDB()
val (numKeys, numInternalKeys) = if (!conf.trackTotalNumberOfRows) {
@@ -680,6 +718,8 @@ class RocksDB(
* Replay change log from the loaded version to the target version.
*/
private def replayChangelog(versionsAndUniqueIds: Array[(Long, Option[String])]): Unit = {
+ val startTime = System.currentTimeMillis()
+
assert(!versionsAndUniqueIds.isEmpty && versionsAndUniqueIds.head._1 == loadedVersion + 1,
s"Replay changelog should start from one version after loadedVersion: $loadedVersion," +
s" but it is not."
@@ -701,13 +741,13 @@ class RocksDB(
changelogReader.foreach { case (recordType, key, value) =>
recordType match {
case RecordType.PUT_RECORD =>
- put(key, value, includesPrefix = true)
+ put(key, value, includesPrefix = true, deriveCfName = true)
case RecordType.DELETE_RECORD =>
- remove(key, includesPrefix = true)
+ remove(key, includesPrefix = true, deriveCfName = true)
case RecordType.MERGE_RECORD =>
- merge(key, value, includesPrefix = true)
+ merge(key, value, includesPrefix = true, deriveCfName = true)
}
}
} else {
@@ -728,6 +768,12 @@ class RocksDB(
if (changelogReader != null) changelogReader.closeIfNeeded()
}
}
+
+ val duration = System.currentTimeMillis() - startTime
+ loadMetrics ++= Map(
+ "replayChangelog" -> Math.max(duration, 1L), // avoid flaky tests
+ "numReplayChangeLogFiles" -> versionsAndUniqueIds.length
+ )
}
/**
@@ -763,6 +809,7 @@ class RocksDB(
def get(
key: Array[Byte],
cfName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Array[Byte] = {
+ updateMemoryUsageIfNeeded()
val keyWithPrefix = if (useColumnFamilies) {
encodeStateRowWithPrefix(key, cfName)
} else {
@@ -829,14 +876,23 @@ class RocksDB(
key: Array[Byte],
value: Array[Byte],
cfName: String = StateStore.DEFAULT_COL_FAMILY_NAME,
- includesPrefix: Boolean = false): Unit = {
+ includesPrefix: Boolean = false,
+ deriveCfName: Boolean = false): Unit = {
+ updateMemoryUsageIfNeeded()
val keyWithPrefix = if (useColumnFamilies && !includesPrefix) {
encodeStateRowWithPrefix(key, cfName)
} else {
key
}
- handleMetricsUpdate(keyWithPrefix, cfName, isPutOrMerge = true)
+ val columnFamilyName = if (deriveCfName && useColumnFamilies) {
+ val (_, cfName) = decodeStateRowWithPrefix(keyWithPrefix)
+ cfName
+ } else {
+ cfName
+ }
+
+ handleMetricsUpdate(keyWithPrefix, columnFamilyName, isPutOrMerge = true)
db.put(writeOptions, keyWithPrefix, value)
changelogWriter.foreach(_.put(keyWithPrefix, value))
}
@@ -856,14 +912,23 @@ class RocksDB(
key: Array[Byte],
value: Array[Byte],
cfName: String = StateStore.DEFAULT_COL_FAMILY_NAME,
- includesPrefix: Boolean = false): Unit = {
+ includesPrefix: Boolean = false,
+ deriveCfName: Boolean = false): Unit = {
+ updateMemoryUsageIfNeeded()
val keyWithPrefix = if (useColumnFamilies && !includesPrefix) {
encodeStateRowWithPrefix(key, cfName)
} else {
key
}
- handleMetricsUpdate(keyWithPrefix, cfName, isPutOrMerge = true)
+ val columnFamilyName = if (deriveCfName && useColumnFamilies) {
+ val (_, cfName) = decodeStateRowWithPrefix(keyWithPrefix)
+ cfName
+ } else {
+ cfName
+ }
+
+ handleMetricsUpdate(keyWithPrefix, columnFamilyName, isPutOrMerge = true)
db.merge(writeOptions, keyWithPrefix, value)
changelogWriter.foreach(_.merge(keyWithPrefix, value))
}
@@ -875,14 +940,23 @@ class RocksDB(
def remove(
key: Array[Byte],
cfName: String = StateStore.DEFAULT_COL_FAMILY_NAME,
- includesPrefix: Boolean = false): Unit = {
+ includesPrefix: Boolean = false,
+ deriveCfName: Boolean = false): Unit = {
+ updateMemoryUsageIfNeeded()
val keyWithPrefix = if (useColumnFamilies && !includesPrefix) {
encodeStateRowWithPrefix(key, cfName)
} else {
key
}
- handleMetricsUpdate(keyWithPrefix, cfName, isPutOrMerge = false)
+ val columnFamilyName = if (deriveCfName && useColumnFamilies) {
+ val (_, cfName) = decodeStateRowWithPrefix(keyWithPrefix)
+ cfName
+ } else {
+ cfName
+ }
+
+ handleMetricsUpdate(keyWithPrefix, columnFamilyName, isPutOrMerge = false)
db.delete(writeOptions, keyWithPrefix)
changelogWriter.foreach(_.delete(keyWithPrefix))
}
@@ -890,7 +964,8 @@ class RocksDB(
/**
* Get an iterator of all committed and uncommitted key-value pairs.
*/
- def iterator(): Iterator[ByteArrayPair] = {
+ def iterator(): NextIterator[ByteArrayPair] = {
+ updateMemoryUsageIfNeeded()
val iter = db.newIterator()
logInfo(log"Getting iterator from version ${MDC(LogKeys.LOADED_VERSION, loadedVersion)}")
iter.seekToFirst()
@@ -926,7 +1001,8 @@ class RocksDB(
/**
* Get an iterator of all committed and uncommitted key-value pairs for the given column family.
*/
- def iterator(cfName: String): Iterator[ByteArrayPair] = {
+ def iterator(cfName: String): NextIterator[ByteArrayPair] = {
+ updateMemoryUsageIfNeeded()
if (!useColumnFamilies) {
iterator()
} else {
@@ -975,7 +1051,8 @@ class RocksDB(
def prefixScan(
prefix: Array[Byte],
- cfName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Iterator[ByteArrayPair] = {
+ cfName: String = StateStore.DEFAULT_COL_FAMILY_NAME): NextIterator[ByteArrayPair] = {
+ updateMemoryUsageIfNeeded()
val iter = db.newIterator()
val updatedPrefix = if (useColumnFamilies) {
encodeStateRowWithPrefix(prefix, cfName)
@@ -1013,6 +1090,8 @@ class RocksDB(
}
}
+ def release(): Unit = {}
+
/**
* Commit all the updates made as a version to DFS. The steps it needs to do to commits are:
* - Flush all changes to disk
@@ -1020,6 +1099,8 @@ class RocksDB(
* - Sync the checkpoint dir files to DFS
*/
def commit(): (Long, StateStoreCheckpointInfo) = {
+ commitLatencyMs.clear()
+ updateMemoryUsageIfNeeded()
val newVersion = loadedVersion + 1
try {
logInfo(log"Flushing updates for ${MDC(LogKeys.VERSION_NUM, newVersion)}")
@@ -1031,13 +1112,14 @@ class RocksDB(
commitLatencyMs ++= snapshotLatency
}
+ var isUploaded = false
+
logInfo(log"Syncing checkpoint for ${MDC(LogKeys.VERSION_NUM, newVersion)} to DFS")
val fileSyncTimeMs = timeTakenMs {
if (enableChangelogCheckpointing) {
// If we have changed the columnFamilyId mapping, we have set a new
// snapshot and need to upload this to the DFS even if changelog checkpointing
// is enabled.
- var isUploaded = false
if (shouldForceSnapshot.get()) {
assert(snapshot.isDefined)
uploadSnapshot(snapshot.get)
@@ -1048,7 +1130,14 @@ class RocksDB(
// ensure that changelog files are always written
try {
assert(changelogWriter.isDefined)
- changelogWriter.foreach(_.commit())
+ val changeLogWriterCommitTimeMs = timeTakenMs {
+ changelogWriter.foreach(_.commit())
+ }
+ // Record the commit time for the changelog writer
+ commitLatencyMs ++= Map(
+ "changeLogWriterCommit" -> changeLogWriterCommitTimeMs
+ )
+
if (!isUploaded) {
snapshot.foreach(snapshotsToUploadQueue.offer)
}
@@ -1059,9 +1148,19 @@ class RocksDB(
assert(changelogWriter.isEmpty)
assert(snapshot.isDefined)
uploadSnapshot(snapshot.get)
+ isUploaded = true
}
}
+ if (isUploaded) {
+ // If we have uploaded the snapshot, the fileManagerMetrics will be cleared and updated
+ // in uploadSnapshot. If there are new metrics needed to be added specific to this commit,
+ // add them here to not accidentally use old fileManagerMetrics from the maintenance threads
+ commitLatencyMs ++= Map(
+ "saveZipFiles" -> fileManagerMetrics.saveZipFilesTimeMs.getOrElse(0L)
+ )
+ }
+
if (enableStateStoreCheckpointIds) {
lastCommitBasedStateStoreCkptId = loadedStateStoreCkptId
lastCommittedStateStoreCkptId = sessionStateStoreCkptId
@@ -1093,10 +1192,6 @@ class RocksDB(
case t: Throwable =>
loadedVersion = -1 // invalidate loaded version
throw t
- } finally {
- // reset resources as either 1) we already pushed the changes and it has been committed or
- // 2) commit has failed and the current version is "invalidated".
- release(LoadStore)
}
}
@@ -1180,23 +1275,18 @@ class RocksDB(
* Drop uncommitted changes, and roll back to previous version.
*/
def rollback(): Unit = {
- acquire(RollbackStore)
- try {
- numKeysOnWritingVersion = numKeysOnLoadedVersion
- numInternalKeysOnWritingVersion = numInternalKeysOnLoadedVersion
- loadedVersion = -1L
- lastCommitBasedStateStoreCkptId = None
- lastCommittedStateStoreCkptId = None
- loadedStateStoreCkptId = None
- sessionStateStoreCkptId = None
- lineageManager.clear()
- changelogWriter.foreach(_.abort())
- // Make sure changelogWriter gets recreated next time.
- changelogWriter = None
- logInfo(log"Rolled back to ${MDC(LogKeys.VERSION_NUM, loadedVersion)}")
- } finally {
- release(RollbackStore)
- }
+ numKeysOnWritingVersion = numKeysOnLoadedVersion
+ numInternalKeysOnWritingVersion = numInternalKeysOnLoadedVersion
+ loadedVersion = -1L
+ lastCommitBasedStateStoreCkptId = None
+ lastCommittedStateStoreCkptId = None
+ loadedStateStoreCkptId = None
+ sessionStateStoreCkptId = None
+ lineageManager.clear()
+ changelogWriter.foreach(_.abort())
+ // Make sure changelogWriter gets recreated next time.
+ changelogWriter = None
+ logInfo(log"Rolled back to ${MDC(LogKeys.VERSION_NUM, loadedVersion)}")
}
def doMaintenance(): Unit = {
@@ -1228,7 +1318,6 @@ class RocksDB(
/** Release all resources */
def close(): Unit = {
// Acquire DB instance lock and release at the end to allow for synchronized access
- acquire(CloseStore)
try {
closeDB()
@@ -1244,14 +1333,21 @@ class RocksDB(
snapshot = snapshotsToUploadQueue.poll()
}
+ // Unregister from RocksDBMemoryManager
+ try {
+ RocksDBMemoryManager.unregisterInstance(instanceUniqueId)
+ } catch {
+ case NonFatal(e) =>
+ logWarning(log"Failed to unregister from RocksDBMemoryManager " +
+ log"${MDC(LogKeys.EXCEPTION, e)}")
+ }
+
silentDeleteRecursively(localRootDir, "closing RocksDB")
// Clear internal maps to reset the state
clearColFamilyMaps()
} catch {
case e: Exception =>
logWarning("Error closing RocksDB", e)
- } finally {
- release(CloseStore)
}
}
@@ -1278,9 +1374,6 @@ class RocksDB(
private def metrics: RocksDBMetrics = {
import HistogramType._
val totalSSTFilesBytes = getDBProperty("rocksdb.total-sst-files-size")
- val readerMemUsage = getDBProperty("rocksdb.estimate-table-readers-mem")
- val memTableMemUsage = getDBProperty("rocksdb.size-all-mem-tables")
- val blockCacheUsage = getDBProperty("rocksdb.block-cache-usage")
val pinnedBlocksMemUsage = getDBProperty("rocksdb.block-cache-pinned-usage")
val nativeOpsHistograms = Seq(
"get" -> DB_GET,
@@ -1314,14 +1407,8 @@ class RocksDB(
nativeStats.getTickerCount(typ)
}
- // if bounded memory usage is enabled, we share the block cache across all state providers
- // running on the same node and account the usage to this single cache. In this case, its not
- // possible to provide partition level or query level memory usage.
- val memoryUsage = if (conf.boundedMemoryUsage) {
- 0L
- } else {
- readerMemUsage + memTableMemUsage + blockCacheUsage
- }
+ // Use RocksDBMemoryManager to calculate the memory usage accounting
+ val memoryUsage = RocksDBMemoryManager.getInstanceMemoryUsage(instanceUniqueId, getMemoryUsage)
RocksDBMetrics(
numKeysOnLoadedVersion,
@@ -1331,7 +1418,9 @@ class RocksDB(
pinnedBlocksMemUsage,
totalSSTFilesBytes,
nativeOpsLatencyMicros,
- commitLatencyMs,
+ // Ensure that the maps are cloned to avoid sharing these Maps
+ commitLatencyMs.clone(),
+ loadMetrics.clone(),
bytesCopied = fileManagerMetrics.bytesCopied,
filesCopied = fileManagerMetrics.filesCopied,
filesReused = fileManagerMetrics.filesReused,
@@ -1347,100 +1436,67 @@ class RocksDB(
*/
def metricsOpt: Option[RocksDBMetrics] = {
var rocksDBMetricsOpt: Option[RocksDBMetrics] = None
- acquire(ReportStoreMetrics)
try {
rocksDBMetricsOpt = recordedMetrics
} catch {
case ex: Exception =>
logInfo(log"Failed to acquire metrics with exception=${MDC(LogKeys.ERROR, ex)}")
- } finally {
- release(ReportStoreMetrics)
}
rocksDBMetricsOpt
}
/**
- * Function to acquire RocksDB instance lock that allows for synchronized access to the state
- * store instance
- *
- * @param opType - operation type requesting the lock
+ * Refresh the recorded metrics with the latest metrics.
*/
- private def acquire(opType: RocksDBOpType): Unit = acquireLock.synchronized {
- val newAcquiredThreadInfo = AcquiredThreadInfo()
- val waitStartTime = System.nanoTime()
- def timeWaitedMs = {
- val elapsedNanos = System.nanoTime() - waitStartTime
- TimeUnit.MILLISECONDS.convert(elapsedNanos, TimeUnit.NANOSECONDS)
- }
- def isAcquiredByDifferentThread = acquiredThreadInfo != null &&
- acquiredThreadInfo.threadRef.get.isDefined &&
- newAcquiredThreadInfo.threadRef.get.get.getId != acquiredThreadInfo.threadRef.get.get.getId
-
- while (isAcquiredByDifferentThread && timeWaitedMs < conf.lockAcquireTimeoutMs) {
- acquireLock.wait(10)
- }
- if (isAcquiredByDifferentThread) {
- val stackTraceOutput = acquiredThreadInfo.threadRef.get.get.getStackTrace.mkString("\n")
- throw QueryExecutionErrors.unreleasedThreadError(loggingId, opType.toString,
- newAcquiredThreadInfo.toString(), acquiredThreadInfo.toString(), timeWaitedMs,
- stackTraceOutput)
- } else {
- acquiredThreadInfo = newAcquiredThreadInfo
- // Add a listener to always release the lock when the task (if active) completes
- Option(TaskContext.get()).foreach(_.addTaskCompletionListener[Unit] {
- _ => this.release(StoreTaskCompletionListener, Some(newAcquiredThreadInfo))
- })
- logInfo(log"RocksDB instance was acquired by " +
- log"ownerThread=${MDC(LogKeys.THREAD, acquiredThreadInfo)} " +
- log"for opType=${MDC(LogKeys.OP_TYPE, opType.toString)}")
- }
+ private[state] def refreshRecordedMetricsForTest(): Unit = {
+ recordedMetrics = Some(metrics)
}
+ private def getDBProperty(property: String): Long = db.getProperty(property).toLong
+
/**
- * Function to release RocksDB instance lock that allows for synchronized access to the state
- * store instance. Optionally provide a thread to check against, and release only if provided
- * thread is the one that acquired the lock.
+ * Returns the current memory usage of this RocksDB instance in bytes.
+ * WARNING: This method should only be called from the task thread when
+ * RocksDB is in a safe state.
*
- * @param opType - operation type releasing the lock
- * @param releaseForThreadOpt - optional thread to check against acquired thread
+ * This includes memory from all major RocksDB components:
+ * - Table readers (indexes and filters in memory)
+ * - Memtables (write buffers)
+ * - Block cache (cached data blocks)
+ * - Block cache pinned usage (blocks pinned in cache)
+ *
+ * @return Total memory usage in bytes across all tracked components
*/
- private def release(
- opType: RocksDBOpType,
- releaseForThreadOpt: Option[AcquiredThreadInfo] = None): Unit = acquireLock.synchronized {
- if (acquiredThreadInfo != null) {
- val release = releaseForThreadOpt match {
- case Some(releaseForThread) if releaseForThread.threadRef.get.isEmpty =>
- logInfo(log"Thread reference is empty when attempting to release for " +
- log"opType=${MDC(LogKeys.OP_TYPE, opType.toString)}, ignoring release. " +
- log"Lock is held by ownerThread=${MDC(LogKeys.THREAD, acquiredThreadInfo)}")
- false
- // NOTE: we compare the entire acquiredThreadInfo object to ensure that we are
- // releasing not only for the right thread but the right task as well. This is
- // inconsistent with the logic for acquire which uses only the thread ID, consider
- // updating this in future.
- case Some(releaseForThread) if acquiredThreadInfo != releaseForThread =>
- logInfo(log"Thread info for " +
- log"releaseThread=${MDC(LogKeys.THREAD, releaseForThreadOpt.get)} " +
- log"does not match the acquired thread when attempting to " +
- log"release for opType=${MDC(LogKeys.OP_TYPE, opType.toString)}, ignoring release. " +
- log"Lock is held by ownerThread=${MDC(LogKeys.THREAD, acquiredThreadInfo)}")
- false
- case _ => true
- }
+ def getMemoryUsage: Long = {
+ require(db != null && !db.isClosed, "RocksDB must be open to get memory usage")
+ RocksDB.mainMemorySources.map(getDBProperty).sum
+ }
+
+ /**
+ * Updates the cached memory usage if enough time has passed.
+ * This is called from task thread operations, so it's already thread-safe.
+ */
+ def updateMemoryUsageIfNeeded(): Unit = {
+
+ val currentTime = System.currentTimeMillis()
+ val timeSinceLastUpdate = currentTime - lastMemoryUpdateTime.get()
- if (release) {
- logInfo(log"RocksDB instance was released by " +
- log"releaseThread=${MDC(LogKeys.THREAD, AcquiredThreadInfo())} " +
- log"with ownerThread=${MDC(LogKeys.THREAD, acquiredThreadInfo)} " +
- log"for opType=${MDC(LogKeys.OP_TYPE, opType.toString)}")
- acquiredThreadInfo = null
- acquireLock.notifyAll()
+ if (timeSinceLastUpdate >= memoryUpdateIntervalMs && db != null && !db.isClosed) {
+ try {
+ val usage = getMemoryUsage
+ lastMemoryUpdateTime.set(currentTime)
+ // Report usage to RocksDBMemoryManager
+ RocksDBMemoryManager.updateMemoryUsage(
+ instanceUniqueId,
+ usage,
+ conf.boundedMemoryUsage)
+ } catch {
+ case NonFatal(e) =>
+ logDebug(s"Failed to update RocksDB memory usage: ${e.getMessage}")
}
}
}
- private def getDBProperty(property: String): Long = db.getProperty(property).toLong
-
private def openDB(): Unit = {
assert(db == null)
db = NativeRocksDB.open(rocksDbOptions, workingDir.toString)
@@ -1462,11 +1518,6 @@ class RocksDB(
}
}
- private[state] def getAcquiredThreadInfo(): Option[AcquiredThreadInfo] =
- acquireLock.synchronized {
- Option(acquiredThreadInfo).map(_.copy())
- }
-
/** Upload the snapshot to DFS and remove it from snapshots pending */
private def uploadSnapshot(
snapshot: RocksDBSnapshot,
@@ -1565,6 +1616,12 @@ class RocksDB(
}
object RocksDB extends Logging {
+
+ val mainMemorySources: Seq[String] = Seq(
+ "rocksdb.estimate-table-readers-mem",
+ "rocksdb.size-all-mem-tables",
+ "rocksdb.block-cache-usage")
+
case class RocksDBSnapshot(
checkpointDir: File,
version: Long,
@@ -1705,6 +1762,24 @@ class RocksDBFileMapping {
}.getOrElse(None)
}
+ /**
+ * Remove all local file mappings that are incompatible with the current version we are
+ * trying to load.
+ *
+ * @return seq of purged mappings
+ */
+ def purgeIncompatibleMappingsForLoad(versionToLoad: Long):
+ Seq[(String, (Long, RocksDBImmutableFile))] = {
+ val filesToRemove = localFileMappings.filter {
+ case (_, (dfsFileMappedVersion, _)) =>
+ dfsFileMappedVersion >= versionToLoad
+ }.toSeq
+ filesToRemove.foreach { case (localFileName, _) =>
+ remove(localFileName)
+ }
+ filesToRemove
+ }
+
def mapToDfsFile(
localFileName: String,
dfsFile: RocksDBImmutableFile,
@@ -1788,6 +1863,7 @@ case class RocksDBConf(
totalMemoryUsageMB: Long,
writeBufferCacheRatio: Double,
highPriorityPoolRatio: Double,
+ memoryUpdateIntervalMs: Long,
compressionCodec: String,
allowFAllocate: Boolean,
compression: String,
@@ -1874,6 +1950,12 @@ object RocksDBConf {
private val HIGH_PRIORITY_POOL_RATIO_CONF = SQLConfEntry(HIGH_PRIORITY_POOL_RATIO_CONF_KEY,
"0.1")
+ // Memory usage update interval for unmanaged memory tracking
+ val MEMORY_UPDATE_INTERVAL_MS_CONF_KEY = "memoryUpdateIntervalMs"
+ private val MEMORY_UPDATE_INTERVAL_MS_CONF = SQLConfEntry(MEMORY_UPDATE_INTERVAL_MS_CONF_KEY,
+ "1000")
+
+
// Allow files to be pre-allocated on disk using fallocate
// Disabling may slow writes, but can solve an issue where
// significant quantities of disk are wasted if there are
@@ -1972,6 +2054,7 @@ object RocksDBConf {
getLongConf(MAX_MEMORY_USAGE_MB_CONF),
getRatioConf(WRITE_BUFFER_CACHE_RATIO_CONF),
getRatioConf(HIGH_PRIORITY_POOL_RATIO_CONF),
+ getPositiveLongConf(MEMORY_UPDATE_INTERVAL_MS_CONF),
storeConf.compressionCodec,
getBooleanConf(ALLOW_FALLOCATE_CONF),
getStringConf(COMPRESSION_CONF),
@@ -1991,6 +2074,7 @@ case class RocksDBMetrics(
totalSSTFilesBytes: Long,
nativeOpsHistograms: Map[String, RocksDBNativeHistogram],
lastCommitLatencyMs: Map[String, Long],
+ loadMetrics: Map[String, Long],
filesCopied: Long,
bytesCopied: Long,
filesReused: Long,
@@ -2023,21 +2107,6 @@ object RocksDBNativeHistogram {
}
}
-case class AcquiredThreadInfo(
- threadRef: WeakReference[Thread] = new WeakReference[Thread](Thread.currentThread()),
- tc: TaskContext = TaskContext.get()) {
- override def toString(): String = {
- val taskStr = if (tc != null) {
- val taskDetails =
- s"partition ${tc.partitionId()}.${tc.attemptNumber()} in stage " +
- s"${tc.stageId()}.${tc.stageAttemptNumber()}, TID ${tc.taskAttemptId()}"
- s", task: $taskDetails"
- } else ""
-
- s"[ThreadId: ${threadRef.get.map(_.getId)}$taskStr]"
- }
-}
-
/**
* A helper class to manage the lineage information when checkpoint unique id is enabled.
* "lineage" is an array of LineageItem (version, uniqueId) pair.
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala
index 562a57aafbd41..0063f960c4757 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala
@@ -30,17 +30,18 @@ import com.fasterxml.jackson.annotation.JsonInclude.Include
import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper}
import com.fasterxml.jackson.databind.annotation.JsonDeserialize
import com.fasterxml.jackson.module.scala.{ClassTagExtensions, DefaultScalaModule}
-import org.apache.commons.io.{FilenameUtils, IOUtils}
+import org.apache.commons.io.FilenameUtils
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileStatus, FileSystem, Path, PathFilter}
import org.json4s.{Formats, NoTypeHints}
import org.json4s.jackson.Serialization
import org.apache.spark.{SparkConf, SparkEnv, SparkException}
-import org.apache.spark.internal.{Logging, LogKeys, MDC, MessageWithContext}
+import org.apache.spark.internal.{Logging, LogKeys, MessageWithContext}
+import org.apache.spark.internal.LogKeys.{DFS_FILE, VERSION_NUM}
import org.apache.spark.io.CompressionCodec
import org.apache.spark.sql.errors.QueryExecutionErrors
-import org.apache.spark.sql.execution.streaming.CheckpointFileManager
+import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.util.ArrayImplicits._
import org.apache.spark.util.Utils
@@ -298,22 +299,31 @@ class RocksDBFileManager(
logDebug(log"Written metadata for version ${MDC(LogKeys.VERSION_NUM, version)}:\n" +
log"${MDC(LogKeys.METADATA_JSON, metadata.prettyJson)}")
- if (version <= 1 && numKeys <= 0) {
- // If we're writing the initial version and there's no data, we have to explicitly initialize
- // the root directory. Normally saveImmutableFilesToDfs will do this initialization, but
- // when there's no data that method won't write any files, and zipToDfsFile uses the
- // CheckpointFileManager.createAtomic API which doesn't auto-initialize parent directories.
- // Moreover, once we disable to track the number of keys, in which the numKeys is -1, we
- // still need to create the initial dfs root directory anyway.
- if (!rootDirChecked) {
- val path = new Path(dfsRootDir)
- if (!fm.exists(path)) fm.mkdirs(path)
- rootDirChecked = true
+ val (_, zipFilesTimeMs) = Utils.timeTakenMs {
+ if (version <= 1 && numKeys <= 0) {
+ // If we're writing the initial version and there's no data, we have to initialize
+ // the root directory. Normally saveImmutableFilesToDfs will do this initialization, but
+ // when there's no data that method won't write any files, and zipToDfsFile uses the
+ // CheckpointFileManager.createAtomic API which doesn't auto-initialize parent directories.
+ // Moreover, once we disable to track the number of keys, in which the numKeys is -1, we
+ // still need to create the initial dfs root directory anyway.
+ if (!rootDirChecked) {
+ val path = new Path(dfsRootDir)
+ if (!fm.exists(path)) fm.mkdirs(path)
+ rootDirChecked = true
+ }
}
+ zipToDfsFile(localOtherFiles :+ metadataFile, dfsBatchZipFile(version, checkpointUniqueId))
+ logInfo(log"Saved checkpoint file for version ${MDC(LogKeys.VERSION_NUM, version)} " +
+ log"checkpointUniqueId: ${MDC(LogKeys.UUID, checkpointUniqueId.getOrElse(""))}")
}
- zipToDfsFile(localOtherFiles :+ metadataFile, dfsBatchZipFile(version, checkpointUniqueId))
- logInfo(log"Saved checkpoint file for version ${MDC(LogKeys.VERSION_NUM, version)} " +
- log"checkpointUniqueId: ${MDC(LogKeys.UUID, checkpointUniqueId.getOrElse(""))}")
+
+ // populate the SaveCheckpointMetrics
+ saveCheckpointMetrics =
+ saveCheckpointMetrics.copy(
+ // Round up to 1ms to reassure that we've logged successfully and avoid flaky tests
+ saveZipFilesTimeMs = Some(Math.max(zipFilesTimeMs, 1L))
+ )
}
/**
@@ -335,7 +345,7 @@ class RocksDBFileManager(
versionToRocksDBFiles.keySet().removeIf(_._1 >= version)
val metadata = if (version == 0) {
if (localDir.exists) Utils.deleteRecursively(localDir)
- localDir.mkdirs()
+ Utils.createDirectory(localDir)
// Since we cleared the local dir, we should also clear the local file mapping
rocksDBFileMapping.clear()
RocksDBCheckpointMetadata(Seq.empty, 0)
@@ -785,6 +795,17 @@ class RocksDBFileManager(
}
}
+ // Delete remaining unnecessary local immutable file mappings.
+ // Files present in the file mapping but not the filesystem may lead to
+ // versionID mismatch error (SPARK-52637), so we should explicitly delete
+ // them.
+ rocksDBFileMapping.purgeIncompatibleMappingsForLoad(version).foreach {
+ case (_, (dfsFileMappedVersion, dfsFile)) =>
+ logInfo(log"Deleted local fileMapping to ${MDC(DFS_FILE, dfsFile)} because " +
+ log"mapped file version ${MDC(VERSION_NUM, dfsFileMappedVersion)} was " +
+ log"incompatible with versionToLoad ${MDC(VERSION_NUM, version)}")
+ }
+
var filesCopied = 0L
var bytesCopied = 0L
var filesReused = 0L
@@ -828,7 +849,7 @@ class RocksDBFileManager(
private def getImmutableFilesFromVersionZip(
version: Long, checkpointUniqueId: Option[String] = None): Seq[RocksDBImmutableFile] = {
Utils.deleteRecursively(localTempDir)
- localTempDir.mkdirs()
+ Utils.createDirectory(localTempDir)
Utils.unzipFilesFromFile(fs, dfsBatchZipFile(version, checkpointUniqueId), localTempDir)
val metadataFile = localMetadataFile(localTempDir)
val metadata = RocksDBCheckpointMetadata.readFromFile(metadataFile)
@@ -849,7 +870,7 @@ class RocksDBFileManager(
files.foreach { file =>
zout.putNextEntry(new ZipEntry(file.getName))
in = new FileInputStream(file)
- val bytes = IOUtils.copy(in, zout)
+ val bytes = in.transferTo(zout)
in.close()
zout.closeEntry()
totalBytes += bytes
@@ -868,8 +889,8 @@ class RocksDBFileManager(
throw e
} finally {
// Close everything no matter what happened
- IOUtils.closeQuietly(in)
- IOUtils.closeQuietly(zout)
+ Utils.closeQuietly(in)
+ Utils.closeQuietly(zout)
}
}
@@ -950,7 +971,9 @@ case class RocksDBFileManagerMetrics(
bytesCopied: Long,
filesReused: Long,
@JsonDeserialize(contentAs = classOf[java.lang.Long])
- zipFileBytesUncompressed: Option[Long] = None)
+ zipFileBytesUncompressed: Option[Long] = None,
+ @JsonDeserialize(contentAs = classOf[java.lang.Long])
+ saveZipFilesTimeMs: Option[Long] = None)
/**
* Metrics to return when requested but no operation has been performed.
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBMemoryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBMemoryManager.scala
index 273cbbc5e87d5..82ee1803b3172 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBMemoryManager.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBMemoryManager.scala
@@ -17,22 +17,117 @@
package org.apache.spark.sql.execution.streaming.state
+import java.util.concurrent.ConcurrentHashMap
+
+import scala.jdk.CollectionConverters._
+
import org.rocksdb._
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.SparkEnv
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys._
+import org.apache.spark.memory.{MemoryMode, UnifiedMemoryManager, UnmanagedMemoryConsumer, UnmanagedMemoryConsumerId}
/**
* Singleton responsible for managing cache and write buffer manager associated with all RocksDB
* state store instances running on a single executor if boundedMemoryUsage is enabled for RocksDB.
* If boundedMemoryUsage is disabled, a new cache object is returned.
+ * This also implements UnmanagedMemoryConsumer to report RocksDB memory usage to Spark's
+ * UnifiedMemoryManager, allowing Spark to account for RocksDB memory when making
+ * memory allocation decisions.
*/
-object RocksDBMemoryManager extends Logging {
+object RocksDBMemoryManager extends Logging with UnmanagedMemoryConsumer {
private var writeBufferManager: WriteBufferManager = null
private var cache: Cache = null
+ // Tracks memory usage and bounded memory mode per unique ID
+ private case class InstanceMemoryInfo(memoryUsage: Long, isBoundedMemory: Boolean)
+ private val instanceMemoryMap = new ConcurrentHashMap[String, InstanceMemoryInfo]()
+
+ override def unmanagedMemoryConsumerId: UnmanagedMemoryConsumerId = {
+ UnmanagedMemoryConsumerId("RocksDB", "RocksDB-Memory-Manager")
+ }
+
+ override def memoryMode: MemoryMode = {
+ // RocksDB uses native/off-heap memory for its data structures
+ MemoryMode.OFF_HEAP
+ }
+
+ override def getMemBytesUsed: Long = {
+ val memoryInfos = instanceMemoryMap.values().asScala.toSeq
+ if (memoryInfos.isEmpty) {
+ return 0L
+ }
+
+ // Separate instances by bounded vs unbounded memory mode
+ val (bounded, unbounded) = memoryInfos.partition(_.isBoundedMemory)
+
+ // For bounded memory instances, they all share the same memory pool,
+ // so just take the max value (they should all be similar)
+ val boundedMemory = if (bounded.nonEmpty) bounded.map(_.memoryUsage).max else 0L
+
+ // For unbounded memory instances, sum their individual usages
+ val unboundedMemory = unbounded.map(_.memoryUsage).sum
+
+ // Total is bounded memory (shared) + sum of unbounded memory (individual)
+ boundedMemory + unboundedMemory
+ }
+
+ /**
+ * Register/update a RocksDB instance with its memory usage.
+ * @param uniqueId The instance's unique identifier
+ * @param memoryUsage The current memory usage in bytes
+ * @param isBoundedMemory Whether this instance uses bounded memory mode
+ */
+ def updateMemoryUsage(
+ uniqueId: String,
+ memoryUsage: Long,
+ isBoundedMemory: Boolean): Unit = {
+ instanceMemoryMap.put(uniqueId, InstanceMemoryInfo(memoryUsage, isBoundedMemory))
+ logDebug(s"Updated memory usage for $uniqueId: $memoryUsage bytes " +
+ s"(bounded=$isBoundedMemory)")
+ }
+
+ /**
+ * Unregister a RocksDB instance.
+ * @param uniqueId The instance's unique identifier
+ */
+ def unregisterInstance(uniqueId: String): Unit = {
+ instanceMemoryMap.remove(uniqueId)
+ logDebug(s"Unregistered instance $uniqueId")
+ }
+
+ def getNumRocksDBInstances(boundedMemory: Boolean): Long = {
+ instanceMemoryMap.values().asScala.count(_.isBoundedMemory == boundedMemory)
+ }
+
+ /**
+ * Get the memory usage for a specific instance, accounting for bounded memory sharing.
+ * @param uniqueId The instance's unique identifier
+ * @param totalMemoryUsage The total memory usage of this instance
+ * @return The adjusted memory usage accounting for sharing in bounded memory mode
+ */
+ def getInstanceMemoryUsage(uniqueId: String, totalMemoryUsage: Long): Long = {
+ val instanceInfo = instanceMemoryMap.
+ getOrDefault(uniqueId, InstanceMemoryInfo(0L, isBoundedMemory = false))
+ if (instanceInfo.isBoundedMemory) {
+ // In bounded memory mode, divide by the number of bounded instances
+ // since they share the same memory pool
+ val numBoundedInstances = getNumRocksDBInstances(true)
+ totalMemoryUsage / numBoundedInstances
+ } else {
+ // In unbounded memory mode, each instance has its own memory
+ totalMemoryUsage
+ }
+ }
+
def getOrCreateRocksDBMemoryManagerAndCache(conf: RocksDBConf): (WriteBufferManager, Cache)
= synchronized {
+ // Register with UnifiedMemoryManager (idempotent operation)
+ if (SparkEnv.get != null) {
+ UnifiedMemoryManager.registerUnmanagedMemoryConsumer(this)
+ }
+
if (conf.boundedMemoryUsage) {
if (writeBufferManager == null) {
assert(cache == null)
@@ -72,5 +167,6 @@ object RocksDBMemoryManager extends Logging {
def resetWriteBufferManagerAndCache: Unit = synchronized {
writeBufferManager = null
cache = null
+ instanceMemoryMap.clear()
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala
index cf5f8ba5f2ebe..f49c79f96b9ce 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala
@@ -36,7 +36,8 @@ import org.apache.spark.sql.avro.{AvroDeserializer, AvroOptions, AvroSerializer,
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{BoundReference, JoinedRow, UnsafeProjection, UnsafeRow}
import org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter
-import org.apache.spark.sql.execution.streaming.{CheckpointFileManager, StateStoreColumnFamilySchemaUtils}
+import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager
+import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.StateStoreColumnFamilySchemaUtils
import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider.{SCHEMA_ID_PREFIX_BYTES, STATE_ENCODING_NUM_VERSION_BYTES, STATE_ENCODING_VERSION}
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.Platform
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateMachine.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateMachine.scala
new file mode 100644
index 0000000000000..80ef42f9bd75b
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateMachine.scala
@@ -0,0 +1,330 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.streaming.state
+
+import java.util.concurrent.TimeUnit
+import java.util.concurrent.atomic.AtomicLong
+import javax.annotation.concurrent.GuardedBy
+
+import scala.ref.WeakReference
+
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.{Logging, LogKeys}
+import org.apache.spark.sql.errors.QueryExecutionErrors
+
+/**
+ * A state machine that manages the lifecycle of a RocksDB instance
+ *
+ * This class enforces proper state transitions and ensures thread-safety for accessing
+ * RocksDB instances.
+ * It prevents concurrent modifications to the same native RocksDB instance by using
+ * a stamp-based locking mechanism.
+ *
+ * State Lifecycle:
+ * - RELEASED: The RocksDB instance is not being accessed by any thread
+ * - ACQUIRED: The RocksDB instance is currently being accessed by a thread
+ * - CLOSED: The RocksDB instance has been closed and can no longer be used
+ *
+ * Valid Transitions:
+ * - RELEASED -> ACQUIRED: When a thread acquires the RocksDB instance
+ * - ACQUIRED -> RELEASED: When a thread releases the RocksDB instance
+ * - RELEASED -> CLOSED: When the RocksDB instance is shut down
+ * - ACQUIRED -> MAINTENANCE: Maintenance can be performed on an acquired RocksDB instance
+ * - RELEASED -> MAINTENANCE: Maintenance can be performed on a released RocksDB instance
+ *
+ * Stamps:
+ * Each time a RocksDB instance is acquired, a unique stamp is generated. This stamp must be
+ * presented when performing operations on the RocksDB instance and when releasing it. This ensures
+ * that only the stamp owner that acquired the RocksDB instance can release it or perform
+ * operations.
+ */
+class RocksDBStateMachine(
+ stateStoreId: StateStoreId,
+ rocksDBConf: RocksDBConf) extends Logging {
+
+ private sealed trait STATE
+ private case object RELEASED extends STATE
+ private case object ACQUIRED extends STATE
+ private case object CLOSED extends STATE
+
+ private sealed abstract class OPERATION(name: String) {
+ override def toString: String = name
+ }
+ private case object LOAD extends OPERATION("load")
+ private case object RELEASE extends OPERATION("release")
+ private case object CLOSE extends OPERATION("close")
+ private case object MAINTENANCE extends OPERATION("maintenance")
+
+ private val stateMachineLock = new Object()
+ @GuardedBy("stateMachineLock")
+ private var state: STATE = RELEASED
+
+ // This is only maintained for logging purposes
+ @GuardedBy("stateMachineLock")
+ private var acquiredThreadInfo: AcquiredThreadInfo = _
+
+ private val RELEASED_STATE_MACHINE_STAMP: Long = -1L
+
+ /**
+ * Map defining all valid state transitions in the state machine.
+ * Key: (currentState, operation) -> Value: nextState
+ *
+ * Valid transitions:
+ * - (RELEASED, LOAD) -> ACQUIRED: Acquire exclusive access to the RocksDB instance
+ * - (ACQUIRED, RELEASE) -> RELEASED: Release exclusive access
+ * - (RELEASED, CLOSE) -> CLOSED: Permanently close the RocksDB instance
+ * - (CLOSED, CLOSE) -> CLOSED: Close is idempotent
+ * - (RELEASED, MAINTENANCE) -> RELEASED: Maintenance on released RocksDB instance
+ * - (ACQUIRED, MAINTENANCE) -> ACQUIRED: Maintenance on acquired RocksDB instance
+ */
+ private val allowedStateTransitions: Map[(STATE, OPERATION), STATE] = Map(
+ (RELEASED, LOAD) -> ACQUIRED,
+ (ACQUIRED, RELEASE) -> RELEASED,
+ (RELEASED, CLOSE) -> CLOSED,
+ (CLOSED, CLOSE) -> CLOSED, // Idempotent close operation
+ (RELEASED, MAINTENANCE) -> RELEASED,
+ (ACQUIRED, MAINTENANCE) -> ACQUIRED
+ )
+
+ /**
+ * Returns information about the thread that currently has the RocksDB instance acquired.
+ * This method is exposed for testing purposes only.
+ *
+ * @return Some(AcquiredThreadInfo) if a thread currently has the RocksDB instance acquired,
+ * None if the RocksDB instance is in RELEASED state
+ */
+ private[spark] def getAcquiredThreadInfo: Option[AcquiredThreadInfo] =
+ stateMachineLock.synchronized {
+ Option(acquiredThreadInfo).map(_.copy())
+ }
+
+ // Can be read without holding any locks, but should only be updated when
+ // stateMachineLock is held.
+ private[state] val currentValidStamp = new AtomicLong(RELEASED_STATE_MACHINE_STAMP)
+ @GuardedBy("stateMachineLock")
+ private var lastValidStamp: Long = 0L
+
+ /**
+ * This method is marked "WithLock" because it MUST only be called when the caller
+ * already holds the stateMachineLock. Calling this method without holding the lock
+ * will result in race conditions and data corruption.
+ *
+ * @return A new unique stamp value
+ */
+ @GuardedBy("stateMachineLock")
+ private def incAndGetStampWithLock: Long = {
+ assert(Thread.holdsLock(stateMachineLock), "Instance lock must be held")
+ lastValidStamp += 1
+ currentValidStamp.set(lastValidStamp)
+ logInfo(log"New stamp: ${MDC(LogKeys.STAMP, currentValidStamp.get())} issued for " +
+ log"${MDC(LogKeys.STATE_STORE_ID, stateStoreId)}")
+ lastValidStamp
+ }
+
+ /**
+ * This method is marked "WithLock" because it MUST only be called when the caller
+ * already holds the stateMachineLock. The method uses stateMachineLock.wait() which
+ * requires the calling stamp owner to own the monitor. Calling this without holding the
+ * lock will throw IllegalMonitorStateException.
+ *
+ * @param operation The operation being attempted (used for error reporting)
+ * @throws QueryExecutionErrors.unreleasedThreadError if timeout occurs
+ */
+ @GuardedBy("stateMachineLock")
+ private def awaitNotAcquiredWithLock(operation: OPERATION): Unit = {
+ assert(Thread.holdsLock(stateMachineLock), "Instance lock must be held")
+ val waitStartTime = System.nanoTime()
+ def timeWaitedMs = {
+ val elapsedNanos = System.nanoTime() - waitStartTime
+ // Convert from nanoseconds to milliseconds
+ TimeUnit.MILLISECONDS.convert(elapsedNanos, TimeUnit.NANOSECONDS)
+ }
+ while (state == ACQUIRED && timeWaitedMs < rocksDBConf.lockAcquireTimeoutMs) {
+ stateMachineLock.wait(10)
+ // log every 30 seconds
+ if (timeWaitedMs % (30 * 1000) == 0) {
+ logInfo(log"Waiting to acquire lock for ${MDC(LogKeys.STATE_STORE_ID, stateStoreId)}")
+ }
+ }
+ if (state == ACQUIRED) {
+ val newAcquiredThreadInfo = AcquiredThreadInfo()
+ val stackTraceOutput = acquiredThreadInfo.threadRef.get.get.getStackTrace.mkString("\n")
+ val loggingId = s"StateStoreId(opId=${stateStoreId.operatorId}," +
+ s"partId=${stateStoreId.partitionId},name=${stateStoreId.storeName})"
+ throw QueryExecutionErrors.unreleasedThreadError(loggingId, operation.toString,
+ newAcquiredThreadInfo.toString(), acquiredThreadInfo.toString(), timeWaitedMs,
+ stackTraceOutput)
+ }
+ }
+
+ /**
+ * Validates a state operation and updates the internal state if the transition is legal.
+ *
+ * This method is the core of the state machine that ensures thread-safe access to RocksDB
+ * instances. It uses a map-based approach to define valid state transitions,
+ * making the state machine logic cleaner and more maintainable.
+ *
+ * Thread Safety Requirements:
+ * - Caller MUST hold the stateMachineLock before calling this method
+ * - This is enforced by the synchronized blocks in all public methods
+ *
+ * Side Effects:
+ * - Updates the internal state variable
+ * - Sets acquiredThreadInfo when transitioning to ACQUIRED state
+ * - Logs state transitions for debugging
+ *
+ * @param operation The requested state operation (LOAD, RELEASE, CLOSE, or MAINTENANCE)
+ * @return A tuple of (oldState, newState) representing the state before and after operation
+ * @throws StateStoreInvalidStateMachineTransition if the requested operation is not allowed
+ * from the current state
+ */
+ @GuardedBy("stateMachineLock")
+ private def validateAndTransitionState(operation: OPERATION): (STATE, STATE) = {
+ assert(Thread.holdsLock(stateMachineLock), "Instance lock must be held")
+ val oldState = state
+ val newState = allowedStateTransitions.get((oldState, operation)) match {
+ case Some(nextState) => nextState
+ case None =>
+ // Determine expected state for better error message
+ val expectedState = operation match {
+ case LOAD => "ACQUIRED"
+ case RELEASE => "RELEASED"
+ case CLOSE => "CLOSED"
+ case MAINTENANCE => oldState.toString
+ }
+ throw StateStoreErrors.invalidStateMachineTransition(
+ oldState.toString, expectedState, operation.toString, stateStoreId)
+ }
+ state = newState
+ if (newState == ACQUIRED) {
+ acquiredThreadInfo = AcquiredThreadInfo()
+ }
+ logInfo(log"Transitioned state from ${MDC(LogKeys.STATE_STORE_STATE, oldState)} " +
+ log"to ${MDC(LogKeys.STATE_STORE_STATE, newState)} " +
+ log"with operation ${MDC(LogKeys.OPERATION, operation.toString)} " +
+ log"for StateStoreId ${MDC(LogKeys.STATE_STORE_ID, stateStoreId)}")
+ (oldState, newState)
+ }
+
+ /**
+ * Verifies that the provided stamp matches the current valid stamp.
+ * This ensures that operations are performed by the task that acquired the RocksDB instance.
+ *
+ * @param stamp The stamp to verify against the current valid stamp
+ * @throws StateStoreInvalidStamp if the stamp does not match the current valid stamp
+ */
+ def verifyStamp(stamp: Long): Unit = {
+ val currentStamp = currentValidStamp.get()
+ if (stamp != currentStamp) {
+ throw StateStoreErrors.invalidStamp(stamp, currentStamp)
+ }
+ }
+
+ /**
+ * Releases the RocksDB instance, transitioning it from ACQUIRED to RELEASED state.
+ * This can only be called by the stamp owner that acquired the RocksDB instance.
+ *
+ * @param stamp The stamp that was returned when the RocksDB instance was acquired
+ * @param throwEx Whether to throw an exception if the stamp is invalid (default: true)
+ * @return true if the RocksDB instance was successfully released, false if stamp was invalid
+ * and throwEx=false
+ * @throws StateStoreInvalidStamp if stamp is invalid and throwEx=true
+ * @throws StateStoreInvalidStateMachineTransition if the current state doesn't allow release
+ */
+ def releaseStamp(stamp: Long, throwEx: Boolean = true): Boolean = stateMachineLock.synchronized {
+ currentValidStamp.compareAndSet(stamp, RELEASED_STATE_MACHINE_STAMP) match {
+ case true =>
+ validateAndTransitionState(RELEASE)
+ true
+ case false =>
+ throwEx match {
+ case true =>
+ val actualStamp = currentValidStamp.get()
+ throw StateStoreErrors.invalidStamp(stamp, actualStamp)
+ case false =>
+ false
+ }
+ }
+ }
+
+ /**
+ * Acquires the RocksDB instance for exclusive use by the calling task.
+ * Transitions the state from RELEASED to ACQUIRED.
+ *
+ * This method will block if another task currently has a stamp for the RocksDB instance,
+ * waiting up to the configured timeout before throwing an exception.
+ *
+ * @return A unique stamp that must be used for subsequent operations and release
+ * @throws StateStoreInvalidStateMachineTransition if the RocksDB instance is in CLOSED state
+ * @throws QueryExecutionErrors.unreleasedThreadError if timeout occurs waiting for another thread
+ */
+ def acquireStamp(): Long = stateMachineLock.synchronized {
+ awaitNotAcquiredWithLock(LOAD)
+ validateAndTransitionState(LOAD)
+ incAndGetStampWithLock
+ }
+
+ /**
+ * This verifies that it is in a state that allows maintenance to be performed.
+ * This operation is allowed in both RELEASED and ACQUIRED states.
+ *
+ * @throws StateStoreInvalidStateMachineTransition if the RocksDB instance is in CLOSED state
+ */
+ def verifyForMaintenance(): Unit = stateMachineLock.synchronized {
+ validateAndTransitionState(MAINTENANCE)
+ }
+
+ /**
+ * Closes the RocksDB instance permanently, transitioning it to CLOSED state.
+ * Once closed, the RocksDB instance cannot be used again and all future operations will fail.
+ *
+ * This method will block if another task currently has a stamp for the RocksDB instance,
+ * waiting up to the configured timeout before throwing an exception.
+ *
+ * @throws StateStoreInvalidStateMachineTransition if called multiple times (idempotent)
+ * @throws QueryExecutionErrors.unreleasedThreadError if timeout occurs waiting for another thread
+ */
+ def close(): Boolean = stateMachineLock.synchronized {
+ // return boolean as to whether we need to close or not
+ if (state == CLOSED) {
+ false
+ } else {
+ logInfo(log"Trying to close store ${MDC(LogKeys.STATE_STORE_ID, stateStoreId)}")
+ awaitNotAcquiredWithLock(CLOSE)
+ logInfo(log"Finished waiting to acquire lock," +
+ log" transitioning to close store ${MDC(LogKeys.STATE_STORE_ID, stateStoreId)}")
+ validateAndTransitionState(CLOSE)
+ true
+ }
+ }
+}
+
+case class AcquiredThreadInfo(
+ threadRef: WeakReference[Thread] = new WeakReference[Thread](Thread.currentThread()),
+ tc: TaskContext = TaskContext.get()) {
+ override def toString(): String = {
+ val taskStr = if (tc != null) {
+ val taskDetails =
+ s"partition ${tc.partitionId()}.${tc.attemptNumber()} in stage " +
+ s"${tc.stageId()}.${tc.stageAttemptNumber()}, TID ${tc.taskAttemptId()}"
+ s", task: $taskDetails"
+ } else ""
+
+ s"[ThreadId: ${threadRef.get.map(_.getId)}$taskStr]"
+ }
+}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala
index 6efdf8d67137b..4230ea3a21662 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala
@@ -26,13 +26,13 @@ import scala.util.control.NonFatal
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
-import org.apache.spark.{SparkConf, SparkEnv}
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.{SparkConf, SparkEnv, TaskContext}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.internal.LogKeys._
import org.apache.spark.io.CompressionCodec
import org.apache.spark.sql.catalyst.expressions.UnsafeRow
import org.apache.spark.sql.errors.QueryExecutionErrors
-import org.apache.spark.sql.execution.streaming.CheckpointFileManager
+import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager
import org.apache.spark.sql.execution.streaming.state.StateStoreEncoding.Avro
import org.apache.spark.sql.types.StructType
import org.apache.spark.unsafe.Platform
@@ -43,20 +43,149 @@ private[sql] class RocksDBStateStoreProvider
with SupportsFineGrainedReplay {
import RocksDBStateStoreProvider._
- class RocksDBStateStore(lastVersion: Long) extends StateStore {
+ class RocksDBStateStore(
+ lastVersion: Long,
+ private[RocksDBStateStoreProvider] val stamp: Long,
+ private[RocksDBStateStoreProvider] var readOnly: Boolean) extends StateStore {
+
+ private sealed trait OPERATION
+ private case object UPDATE extends OPERATION
+ private case object ABORT extends OPERATION
+ private case object RELEASE extends OPERATION
+ private case object COMMIT extends OPERATION
+ private case object METRICS extends OPERATION
+
/** Trait and classes representing the internal state of the store */
trait STATE
case object UPDATING extends STATE
case object COMMITTED extends STATE
case object ABORTED extends STATE
+ case object RELEASED extends STATE
@volatile private var state: STATE = UPDATING
@volatile private var isValidated = false
+ /**
+ * Map defining all valid state transitions for the RocksDB state store.
+ * Key: (currentState, operation) -> Value: nextState
+ *
+ * Valid transitions:
+ * - (UPDATING, UPDATE) -> UPDATING: Continue updating
+ * - (UPDATING, ABORT) -> ABORTED: Abort during update
+ * - (UPDATING, RELEASE) -> RELEASED: Release during update
+ * - (UPDATING, COMMIT) -> COMMITTED: Direct commit
+ * - (COMMITTED, METRICS) -> COMMITTED: Allow metrics after commit
+ * - (ABORTED, ABORT) -> ABORTED: Abort is idempotent
+ * - (ABORTED, METRICS) -> ABORTED: Allow metrics after abort
+ * - (RELEASED, RELEASE) -> RELEASED: Release is idempotent
+ * - (RELEASED, METRICS) -> RELEASED: Allow metrics after release
+ */
+ private val allowedStateTransitions: Map[(STATE, OPERATION), STATE] = Map(
+ // From UPDATING state
+ (UPDATING, UPDATE) -> UPDATING,
+ (UPDATING, ABORT) -> ABORTED,
+ (UPDATING, RELEASE) -> RELEASED,
+ (UPDATING, COMMIT) -> COMMITTED,
+ // From COMMITTED state
+ (COMMITTED, METRICS) -> COMMITTED,
+ // From ABORTED state
+ (ABORTED, ABORT) -> ABORTED, // Idempotent
+ (ABORTED, METRICS) -> ABORTED,
+ // From RELEASED state
+ (RELEASED, RELEASE) -> RELEASED, // Idempotent
+ (RELEASED, METRICS) -> RELEASED
+ )
+
override def id: StateStoreId = RocksDBStateStoreProvider.this.stateStoreId
override def version: Long = lastVersion
+ /**
+ * Validates the expected state, throws exception if state is not as expected.
+ * Returns the current state
+ *
+ * @param possibleStates Expected possible states
+ * @return current state of StateStore
+ */
+ private def validateState(possibleStates: STATE*): STATE = {
+ if (!possibleStates.contains(state)) {
+ throw StateStoreErrors.stateStoreOperationOutOfOrder(
+ s"Expected possible states ${possibleStates.mkString("(", ", ", ")")} but found $state")
+ }
+ state
+ }
+
+ /**
+ * Throws error if transition is illegal.
+ * MUST be called for every StateStore method.
+ *
+ * @param operation The transition type of the operation.
+ */
+ private def validateAndTransitionState(operation: OPERATION): Unit = {
+ val oldState = state
+
+ // Operations requiring stamp verification
+ val needsStampVerification = operation match {
+ case ABORT if state == ABORTED => false // ABORT is idempotent
+ case RELEASE if state == RELEASED => false // RELEASE is idempotent
+ case UPDATE | ABORT | RELEASE | COMMIT => true
+ case METRICS => false
+ }
+
+ if (needsStampVerification) {
+ stateMachine.verifyStamp(stamp)
+ }
+
+ val newState = allowedStateTransitions.get((oldState, operation)) match {
+ case Some(nextState) => nextState
+ case None =>
+ val errorMsg = operation match {
+ case UPDATE => s"Cannot update after ${oldState.toString}"
+ case ABORT => s"Cannot abort after ${oldState.toString}"
+ case RELEASE => s"Cannot release after ${oldState.toString}"
+ case COMMIT => s"Cannot commit after ${oldState.toString}"
+ case METRICS => s"Cannot get metrics in ${oldState} state"
+ }
+ throw StateStoreErrors.stateStoreOperationOutOfOrder(errorMsg)
+ }
+
+ // Special handling for COMMIT operation - release the store
+ if (operation == COMMIT || operation == RELEASE) {
+ stateMachine.releaseStamp(stamp)
+ }
+
+ if (operation != UPDATE) {
+ logInfo(log"Transitioned state from ${MDC(LogKeys.STATE_STORE_STATE, oldState)} " +
+ log"to ${MDC(LogKeys.STATE_STORE_STATE, newState)} " +
+ log"for StateStoreId ${MDC(LogKeys.STATE_STORE_ID, stateStoreId)} " +
+ log"with transition ${MDC(LogKeys.OPERATION, operation.toString)}")
+ }
+ state = newState
+ }
+
+ Option(TaskContext.get()).foreach { ctxt =>
+ ctxt.addTaskCompletionListener[Unit](ctx => {
+ try {
+ if (state == UPDATING) {
+ if (readOnly) {
+ release()
+ } else {
+ abort() // Abort since this is an error if stateful task completes
+ }
+ }
+ } catch {
+ case NonFatal(e) =>
+ logWarning("Failed to abort or release state store", e)
+ } finally {
+ stateMachine.releaseStamp(stamp, throwEx = false)
+ }
+ })
+ // Abort the async commit stores only when the task has failed and store is not committed.
+ ctxt.addTaskFailureListener((_, _) => {
+ if (!hasCommitted) abort()
+ })
+ }
+
override def createColFamilyIfAbsent(
colFamilyName: String,
keySchema: StructType,
@@ -64,6 +193,7 @@ private[sql] class RocksDBStateStoreProvider
keyStateEncoderSpec: KeyStateEncoderSpec,
useMultipleValuesPerKey: Boolean = false,
isInternal: Boolean = false): Unit = {
+ validateAndTransitionState(UPDATE)
verifyColFamilyCreationOrDeletion("create_col_family", colFamilyName, isInternal)
val cfId = rocksDB.createColFamilyIfAbsent(colFamilyName, isInternal)
val dataEncoderCacheKey = StateRowEncoderCacheKey(
@@ -105,6 +235,7 @@ private[sql] class RocksDBStateStoreProvider
}
override def get(key: UnsafeRow, colFamilyName: String): UnsafeRow = {
+ validateAndTransitionState(UPDATE)
verify(key != null, "Key cannot be null")
verifyColFamilyOperations("get", colFamilyName)
@@ -131,6 +262,7 @@ private[sql] class RocksDBStateStoreProvider
* values per key.
*/
override def valuesIterator(key: UnsafeRow, colFamilyName: String): Iterator[UnsafeRow] = {
+ validateAndTransitionState(UPDATE)
verify(key != null, "Key cannot be null")
verifyColFamilyOperations("valuesIterator", colFamilyName)
@@ -147,6 +279,7 @@ private[sql] class RocksDBStateStoreProvider
override def merge(key: UnsafeRow, value: UnsafeRow,
colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = {
+ validateAndTransitionState(UPDATE)
verify(state == UPDATING, "Cannot merge after already committed or aborted")
verifyColFamilyOperations("merge", colFamilyName)
@@ -162,6 +295,7 @@ private[sql] class RocksDBStateStoreProvider
}
override def put(key: UnsafeRow, value: UnsafeRow, colFamilyName: String): Unit = {
+ validateAndTransitionState(UPDATE)
verify(state == UPDATING, "Cannot put after already committed or aborted")
verify(key != null, "Key cannot be null")
require(value != null, "Cannot put a null value")
@@ -172,6 +306,7 @@ private[sql] class RocksDBStateStoreProvider
}
override def remove(key: UnsafeRow, colFamilyName: String): Unit = {
+ validateAndTransitionState(UPDATE)
verify(state == UPDATING, "Cannot remove after already committed or aborted")
verify(key != null, "Key cannot be null")
verifyColFamilyOperations("remove", colFamilyName)
@@ -180,16 +315,18 @@ private[sql] class RocksDBStateStoreProvider
rocksDB.remove(kvEncoder._1.encodeKey(key), colFamilyName)
}
- override def iterator(colFamilyName: String): Iterator[UnsafeRowPair] = {
+ override def iterator(colFamilyName: String): StateStoreIterator[UnsafeRowPair] = {
+ validateAndTransitionState(UPDATE)
// Note this verify function only verify on the colFamilyName being valid,
// we are actually doing prefix when useColumnFamilies,
// but pass "iterator" to throw correct error message
verifyColFamilyOperations("iterator", colFamilyName)
val kvEncoder = keyValueEncoderMap.get(colFamilyName)
val rowPair = new UnsafeRowPair()
-
if (useColumnFamilies) {
- rocksDB.iterator(colFamilyName).map { kv =>
+ val rocksDbIter = rocksDB.iterator(colFamilyName)
+
+ val iter = rocksDbIter.map { kv =>
rowPair.withRows(kvEncoder._1.decodeKey(kv.key),
kvEncoder._2.decodeValue(kv.value))
if (!isValidated && rowPair.value != null && !useColumnFamilies) {
@@ -199,8 +336,12 @@ private[sql] class RocksDBStateStoreProvider
}
rowPair
}
+
+ new StateStoreIterator(iter, rocksDbIter.closeIfNeeded)
} else {
- rocksDB.iterator().map { kv =>
+ val rocksDbIter = rocksDB.iterator()
+
+ val iter = rocksDbIter.map { kv =>
rowPair.withRows(kvEncoder._1.decodeKey(kv.key),
kvEncoder._2.decodeValue(kv.value))
if (!isValidated && rowPair.value != null && !useColumnFamilies) {
@@ -210,11 +351,15 @@ private[sql] class RocksDBStateStoreProvider
}
rowPair
}
+
+ new StateStoreIterator(iter, rocksDbIter.closeIfNeeded)
}
}
- override def prefixScan(prefixKey: UnsafeRow, colFamilyName: String):
- Iterator[UnsafeRowPair] = {
+ override def prefixScan(
+ prefixKey: UnsafeRow,
+ colFamilyName: String): StateStoreIterator[UnsafeRowPair] = {
+ validateAndTransitionState(UPDATE)
verifyColFamilyOperations("prefixScan", colFamilyName)
val kvEncoder = keyValueEncoderMap.get(colFamilyName)
@@ -223,22 +368,36 @@ private[sql] class RocksDBStateStoreProvider
val rowPair = new UnsafeRowPair()
val prefix = kvEncoder._1.encodePrefixKey(prefixKey)
- rocksDB.prefixScan(prefix, colFamilyName).map { kv =>
+
+ val rocksDbIter = rocksDB.prefixScan(prefix, colFamilyName)
+ val iter = rocksDbIter.map { kv =>
rowPair.withRows(kvEncoder._1.decodeKey(kv.key),
kvEncoder._2.decodeValue(kv.value))
rowPair
}
+
+ new StateStoreIterator(iter, rocksDbIter.closeIfNeeded)
}
var checkpointInfo: Option[StateStoreCheckpointInfo] = None
+ private var storedMetrics: Option[RocksDBMetrics] = None
+
override def commit(): Long = synchronized {
+ validateState(UPDATING)
try {
- verify(state == UPDATING, "Cannot commit after already committed or aborted")
+ stateMachine.verifyStamp(stamp)
val (newVersion, newCheckpointInfo) = rocksDB.commit()
checkpointInfo = Some(newCheckpointInfo)
- state = COMMITTED
+ storedMetrics = rocksDB.metricsOpt
+ validateAndTransitionState(COMMIT)
logInfo(log"Committed ${MDC(VERSION_NUM, newVersion)} " +
log"for ${MDC(STATE_STORE_ID, id)}")
+
+ // Report the commit to StateStoreCoordinator for tracking
+ if (storeConf.commitValidationEnabled) {
+ StateStore.reportCommitToCoordinator(newVersion, stateStoreId, hadoopConf)
+ }
+
newVersion
} catch {
case e: Throwable =>
@@ -246,16 +405,38 @@ private[sql] class RocksDBStateStoreProvider
}
}
+ override def release(): Unit = {
+ assert(readOnly, "Release can only be called on a read-only store")
+ if (state != RELEASED) {
+ logInfo(log"Releasing ${MDC(VERSION_NUM, version + 1)} " +
+ log"for ${MDC(STATE_STORE_ID, id)}")
+ rocksDB.release()
+ validateAndTransitionState(RELEASE)
+ } else {
+ // Optionally log at DEBUG level that it's already released
+ logDebug(log"State store already released")
+ }
+ }
+
override def abort(): Unit = {
- verify(state == UPDATING || state == ABORTED, "Cannot abort after already committed")
- logInfo(log"Aborting ${MDC(VERSION_NUM, version + 1)} " +
- log"for ${MDC(STATE_STORE_ID, id)}")
- rocksDB.rollback()
- state = ABORTED
+ if (validateState(UPDATING, ABORTED) != ABORTED) {
+ try {
+ validateAndTransitionState(ABORT)
+ logInfo(log"Aborting ${MDC(VERSION_NUM, version + 1)} " +
+ log"for ${MDC(STATE_STORE_ID, id)}")
+ rocksDB.rollback()
+ } finally {
+ stateMachine.releaseStamp(stamp)
+ }
+ } else {
+ logInfo(log"Skipping abort for ${MDC(VERSION_NUM, version + 1)} " +
+ log"for ${MDC(STATE_STORE_ID, id)} as we already aborted")
+ }
}
override def metrics: StateStoreMetrics = {
- val rocksDBMetricsOpt = rocksDB.metricsOpt
+ validateAndTransitionState(METRICS)
+ val rocksDBMetricsOpt = storedMetrics
if (rocksDBMetricsOpt.isDefined) {
val rocksDBMetrics = rocksDBMetricsOpt.get
@@ -263,6 +444,9 @@ private[sql] class RocksDBStateStoreProvider
def commitLatencyMs(typ: String): Long =
rocksDBMetrics.lastCommitLatencyMs.getOrElse(typ, 0L)
+ def loadMetrics(typ: String): Long =
+ rocksDBMetrics.loadMetrics.getOrElse(typ, 0L)
+
def nativeOpsLatencyMillis(typ: String): Long = {
rocksDBMetrics.nativeOpsMetrics.get(typ).map(_ * 1000).getOrElse(0)
}
@@ -298,6 +482,13 @@ private[sql] class RocksDBStateStoreProvider
CUSTOM_METRIC_COMMIT_COMPACT_TIME -> commitLatencyMs("compact"),
CUSTOM_METRIC_CHECKPOINT_TIME -> commitLatencyMs("checkpoint"),
CUSTOM_METRIC_FILESYNC_TIME -> commitLatencyMs("fileSync"),
+ CUSTOM_METRIC_CHANGE_LOG_WRITER_COMMIT_TIME -> commitLatencyMs("changeLogWriterCommit"),
+ CUSTOM_METRIC_SAVE_ZIP_FILES_TIME -> commitLatencyMs("saveZipFiles"),
+
+ CUSTOM_METRIC_LOAD_FROM_SNAPSHOT_TIME -> loadMetrics("loadFromSnapshot"),
+ CUSTOM_METRIC_LOAD_TIME -> loadMetrics("load"),
+ CUSTOM_METRIC_REPLAY_CHANGE_LOG -> loadMetrics("replayChangelog"),
+ CUSTOM_METRIC_NUM_REPLAY_CHANGE_LOG_FILES -> loadMetrics("numReplayChangeLogFiles"),
CUSTOM_METRIC_BYTES_COPIED -> rocksDBMetrics.bytesCopied,
CUSTOM_METRIC_FILES_COPIED -> rocksDBMetrics.filesCopied,
CUSTOM_METRIC_FILES_REUSED -> rocksDBMetrics.filesReused,
@@ -337,6 +528,7 @@ private[sql] class RocksDBStateStoreProvider
}
override def getStateStoreCheckpointInfo(): StateStoreCheckpointInfo = {
+ validateAndTransitionState(METRICS)
checkpointInfo match {
case Some(info) => info
case None => throw StateStoreErrors.stateStoreOperationOutOfOrder(
@@ -347,8 +539,7 @@ private[sql] class RocksDBStateStoreProvider
override def hasCommitted: Boolean = state == COMMITTED
override def toString: String = {
- s"RocksDBStateStore[id=(op=${id.operatorId},part=${id.partitionId})," +
- s"dir=${id.storeCheckpointLocation()}]"
+ s"RocksDBStateStore[stateStoreId=$stateStoreId_, version=$version]"
}
/** Return the [[RocksDB]] instance in this store. This is exposed mainly for testing. */
@@ -393,6 +584,10 @@ private[sql] class RocksDBStateStoreProvider
this.rocksDBEventForwarder =
Some(RocksDBEventForwarder(StateStoreProvider.getRunId(hadoopConf), stateStoreId))
+ // Initialize StateStoreProviderId for memory tracking
+ val queryRunId = UUID.fromString(StateStoreProvider.getRunId(hadoopConf))
+ this.stateStoreProviderId = StateStoreProviderId(stateStoreId, queryRunId)
+
if (useMultipleValuesPerKey) {
require(useColumnFamilies, "Multiple values per key support requires column families to be" +
" enabled in RocksDBStateStore.")
@@ -446,48 +641,120 @@ private[sql] class RocksDBStateStoreProvider
override def stateStoreId: StateStoreId = stateStoreId_
- override def getStore(version: Long, uniqueId: Option[String] = None): StateStore = {
+ private lazy val stateMachine: RocksDBStateMachine =
+ new RocksDBStateMachine(stateStoreId, RocksDBConf(storeConf))
+
+ override protected def logName: String = s"${super.logName} ${stateStoreProviderId}"
+
+ /**
+ * Creates and returns a state store with the specified parameters.
+ *
+ * @param version The version of the state store to load
+ * @param uniqueId Optional unique identifier for checkpoint
+ * @param readOnly Whether to open the store in read-only mode
+ * @param existingStore Optional existing store to reuse instead of creating a new one
+ * @return The loaded state store
+ */
+ private def loadStateStore(
+ version: Long,
+ uniqueId: Option[String] = None,
+ readOnly: Boolean,
+ existingStore: Option[RocksDBStateStore] = None): StateStore = {
+ var acquiredStamp: Option[Long] = None
+ var storeLoaded = false
try {
if (version < 0) {
throw QueryExecutionErrors.unexpectedStateStoreVersion(version)
}
- rocksDB.load(
- version,
- stateStoreCkptId = if (storeConf.enableStateStoreCheckpointIds) uniqueId else None)
- new RocksDBStateStore(version)
- }
- catch {
- case e: OutOfMemoryError =>
- throw QueryExecutionErrors.notEnoughMemoryToLoadStore(
- stateStoreId.toString,
- "ROCKSDB_STORE_PROVIDER",
- e)
- case e: Throwable => throw StateStoreErrors.cannotLoadStore(e)
- }
- }
- override def getReadStore(version: Long, uniqueId: Option[String] = None): StateStore = {
- try {
- if (version < 0) {
- throw QueryExecutionErrors.unexpectedStateStoreVersion(version)
+ // Early validation of the existing store type before loading RocksDB
+ existingStore.foreach { store =>
+ if (!store.readOnly) {
+ throw new IllegalArgumentException(
+ s"Existing store must be readOnly, but got a read-write store")
+ }
+ }
+
+ // if the existing store is None, then we need to acquire the stamp before
+ // loading RocksDB
+ val stamp = existingStore match {
+ case None =>
+ val s = stateMachine.acquireStamp()
+ acquiredStamp = Some(s)
+ Some(s)
+ case Some(store: RocksDBStateStore) =>
+ val s = store.stamp
+ stateMachine.verifyStamp(s)
+ Some(s)
}
+
rocksDB.load(
version,
stateStoreCkptId = if (storeConf.enableStateStoreCheckpointIds) uniqueId else None,
- readOnly = true)
- new RocksDBStateStore(version)
- }
- catch {
+ readOnly = readOnly)
+
+ // Create or reuse store instance
+ val store = existingStore match {
+ case Some(store: RocksDBStateStore) =>
+ // Mark store as being used for write operations
+ store.readOnly = readOnly
+ store
+ case None =>
+ // Create new store instance. The stamp should be defined
+ // in this case
+ new RocksDBStateStore(version, stamp.get, readOnly)
+ }
+ storeLoaded = true
+ store
+ } catch {
case e: OutOfMemoryError =>
throw QueryExecutionErrors.notEnoughMemoryToLoadStore(
stateStoreId.toString,
"ROCKSDB_STORE_PROVIDER",
e)
+ case e: StateStoreInvalidStateMachineTransition =>
+ throw e
case e: Throwable => throw StateStoreErrors.cannotLoadStore(e)
+ } finally {
+ // If we acquired a stamp but failed to load the store, release it.
+ // Note: We cannot rely on the task completion listener to clean up the stamp in this case
+ // because the listener is only registered in the RocksDBStateStore constructor. If the
+ // store fails to load (e.g., rocksDB.load() throws an exception), the RocksDBStateStore
+ // instance is never created, so no completion listener exists to release the stamp.
+ // This finally block ensures proper cleanup even when store creation fails early.
+ if (!storeLoaded && acquiredStamp.isDefined) {
+ acquiredStamp.foreach(stamp => stateMachine.releaseStamp(stamp, throwEx = false))
+ }
}
}
+ override def getStore(
+ version: Long, uniqueId: Option[String] = None): StateStore = {
+ loadStateStore(version, uniqueId, readOnly = false)
+ }
+
+ override def upgradeReadStoreToWriteStore(
+ readStore: ReadStateStore,
+ version: Long,
+ uniqueId: Option[String] = None): StateStore = {
+ assert(version == readStore.version,
+ s"Can only upgrade readStore to writeStore with the same version," +
+ s" readStoreVersion: ${readStore.version}, writeStoreVersion: ${version}")
+ assert(this.stateStoreId == readStore.id, "Can only upgrade readStore to writeStore with" +
+ " the same stateStoreId")
+ assert(readStore.isInstanceOf[RocksDBStateStore], "Can only upgrade state store if it is a " +
+ "RocksDBStateStore")
+ loadStateStore(version, uniqueId, readOnly = false, existingStore =
+ Some(readStore.asInstanceOf[RocksDBStateStore]))
+ }
+
+ override def getReadStore(
+ version: Long, uniqueId: Option[String] = None): StateStore = {
+ loadStateStore(version, uniqueId, readOnly = true)
+ }
+
override def doMaintenance(): Unit = {
+ stateMachine.verifyForMaintenance()
try {
rocksDB.doMaintenance()
} catch {
@@ -500,7 +767,9 @@ private[sql] class RocksDBStateStoreProvider
}
override def close(): Unit = {
- rocksDB.close()
+ if (stateMachine.close()) {
+ rocksDB.close()
+ }
}
override def supportedCustomMetrics: Seq[StateStoreCustomMetric] = ALL_CUSTOM_METRICS
@@ -520,6 +789,7 @@ private[sql] class RocksDBStateStoreProvider
@volatile private var stateStoreEncoding: String = _
@volatile private var stateSchemaProvider: Option[StateSchemaProvider] = _
@volatile private var rocksDBEventForwarder: Option[RocksDBEventForwarder] = _
+ @volatile private var stateStoreProviderId: StateStoreProviderId = _
protected def createRocksDB(
dfsRootDir: String,
@@ -530,7 +800,8 @@ private[sql] class RocksDBStateStoreProvider
useColumnFamilies: Boolean,
enableStateStoreCheckpointIds: Boolean,
partitionId: Int = 0,
- eventForwarder: Option[RocksDBEventForwarder] = None): RocksDB = {
+ eventForwarder: Option[RocksDBEventForwarder] = None,
+ uniqueId: Option[String] = None): RocksDB = {
new RocksDB(
dfsRootDir,
conf,
@@ -540,18 +811,21 @@ private[sql] class RocksDBStateStoreProvider
useColumnFamilies,
enableStateStoreCheckpointIds,
partitionId,
- eventForwarder)
+ eventForwarder,
+ uniqueId)
}
private[sql] lazy val rocksDB = {
val dfsRootDir = stateStoreId.storeCheckpointLocation().toString
val storeIdStr = s"StateStoreId(opId=${stateStoreId.operatorId}," +
s"partId=${stateStoreId.partitionId},name=${stateStoreId.storeName})"
+ val loggingId = stateStoreProviderId.toString
val sparkConf = Option(SparkEnv.get).map(_.conf).getOrElse(new SparkConf)
val localRootDir = Utils.createTempDir(Utils.getLocalDir(sparkConf), storeIdStr)
- createRocksDB(dfsRootDir, RocksDBConf(storeConf), localRootDir, hadoopConf, storeIdStr,
+ createRocksDB(dfsRootDir, RocksDBConf(storeConf), localRootDir, hadoopConf, loggingId,
useColumnFamilies, storeConf.enableStateStoreCheckpointIds, stateStoreId.partitionId,
- rocksDBEventForwarder)
+ rocksDBEventForwarder,
+ Some(s"${stateStoreProviderId.toString}_${UUID.randomUUID().toString}"))
}
private val keyValueEncoderMap = new java.util.concurrent.ConcurrentHashMap[String,
@@ -572,7 +846,8 @@ private[sql] class RocksDBStateStoreProvider
* @param endVersion checkpoint version to end with
* @return [[StateStore]]
*/
- override def replayStateFromSnapshot(snapshotVersion: Long, endVersion: Long): StateStore = {
+ override def replayStateFromSnapshot(
+ snapshotVersion: Long, endVersion: Long, readOnly: Boolean): StateStore = {
try {
if (snapshotVersion < 1) {
throw QueryExecutionErrors.unexpectedStateStoreVersion(snapshotVersion)
@@ -580,8 +855,15 @@ private[sql] class RocksDBStateStoreProvider
if (endVersion < snapshotVersion) {
throw QueryExecutionErrors.unexpectedStateStoreVersion(endVersion)
}
- rocksDB.loadFromSnapshot(snapshotVersion, endVersion)
- new RocksDBStateStore(endVersion)
+ val stamp = stateMachine.acquireStamp()
+ try {
+ rocksDB.loadFromSnapshot(snapshotVersion, endVersion)
+ new RocksDBStateStore(endVersion, stamp, readOnly)
+ } catch {
+ case e: Throwable =>
+ stateMachine.releaseStamp(stamp)
+ throw e
+ }
}
catch {
case e: OutOfMemoryError =>
@@ -851,6 +1133,25 @@ object RocksDBStateStoreProvider {
"rocksdbFilesReused", "RocksDB: file manager - files reused")
val CUSTOM_METRIC_ZIP_FILE_BYTES_UNCOMPRESSED = StateStoreCustomSizeMetric(
"rocksdbZipFileBytesUncompressed", "RocksDB: file manager - uncompressed zip file bytes")
+ val CUSTOM_METRIC_CHANGE_LOG_WRITER_COMMIT_TIME = StateStoreCustomTimingMetric(
+ "rocksdbChangeLogWriterCommitLatencyMs",
+ "RocksDB: commit - changelog commit time")
+ val CUSTOM_METRIC_SAVE_ZIP_FILES_TIME = StateStoreCustomTimingMetric(
+ "rocksdbSaveZipFilesLatencyMs",
+ "RocksDB: commit - zip files sync to external storage time")
+
+ val CUSTOM_METRIC_LOAD_FROM_SNAPSHOT_TIME = StateStoreCustomTimingMetric(
+ "rocksdbLoadFromSnapshotLatencyMs",
+ "RocksDB: load from snapshot - time taken to load the store from snapshot")
+ val CUSTOM_METRIC_LOAD_TIME = StateStoreCustomTimingMetric(
+ "rocksdbLoadLatencyMs",
+ "RocksDB: load - time taken to load the store")
+ val CUSTOM_METRIC_REPLAY_CHANGE_LOG = StateStoreCustomTimingMetric(
+ "rocksdbReplayChangeLogLatencyMs",
+ "RocksDB: load - time taken to replay the change log")
+ val CUSTOM_METRIC_NUM_REPLAY_CHANGE_LOG_FILES = StateStoreCustomSizeMetric(
+ "rocksdbNumReplayChangelogFiles",
+ "RocksDB: load - number of change log files replayed")
val CUSTOM_METRIC_BLOCK_CACHE_MISS = StateStoreCustomSumMetric(
"rocksdbReadBlockCacheMissCount",
@@ -905,13 +1206,16 @@ object RocksDBStateStoreProvider {
CUSTOM_METRIC_FLUSH_TIME, CUSTOM_METRIC_COMMIT_COMPACT_TIME,
CUSTOM_METRIC_CHECKPOINT_TIME, CUSTOM_METRIC_FILESYNC_TIME,
CUSTOM_METRIC_BYTES_COPIED, CUSTOM_METRIC_FILES_COPIED, CUSTOM_METRIC_FILES_REUSED,
- CUSTOM_METRIC_ZIP_FILE_BYTES_UNCOMPRESSED, CUSTOM_METRIC_GET_COUNT, CUSTOM_METRIC_PUT_COUNT,
+ CUSTOM_METRIC_ZIP_FILE_BYTES_UNCOMPRESSED, CUSTOM_METRIC_CHANGE_LOG_WRITER_COMMIT_TIME,
+ CUSTOM_METRIC_SAVE_ZIP_FILES_TIME, CUSTOM_METRIC_GET_COUNT, CUSTOM_METRIC_PUT_COUNT,
CUSTOM_METRIC_BLOCK_CACHE_MISS, CUSTOM_METRIC_BLOCK_CACHE_HITS, CUSTOM_METRIC_BYTES_READ,
CUSTOM_METRIC_BYTES_WRITTEN, CUSTOM_METRIC_ITERATOR_BYTES_READ, CUSTOM_METRIC_STALL_TIME,
CUSTOM_METRIC_TOTAL_COMPACT_TIME, CUSTOM_METRIC_COMPACT_READ_BYTES,
CUSTOM_METRIC_COMPACT_WRITTEN_BYTES, CUSTOM_METRIC_FLUSH_WRITTEN_BYTES,
CUSTOM_METRIC_PINNED_BLOCKS_MEM_USAGE, CUSTOM_METRIC_NUM_INTERNAL_COL_FAMILIES_KEYS,
- CUSTOM_METRIC_NUM_EXTERNAL_COL_FAMILIES, CUSTOM_METRIC_NUM_INTERNAL_COL_FAMILIES)
+ CUSTOM_METRIC_NUM_EXTERNAL_COL_FAMILIES, CUSTOM_METRIC_NUM_INTERNAL_COL_FAMILIES,
+ CUSTOM_METRIC_LOAD_FROM_SNAPSHOT_TIME, CUSTOM_METRIC_LOAD_TIME, CUSTOM_METRIC_REPLAY_CHANGE_LOG,
+ CUSTOM_METRIC_NUM_REPLAY_CHANGE_LOG_FILES)
val CUSTOM_INSTANCE_METRIC_SNAPSHOT_LAST_UPLOADED = StateStoreSnapshotLastUploadInstanceMetric()
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SchemaHelper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SchemaHelper.scala
index d67eb40fde2c2..7da8c5a6bd3ca 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SchemaHelper.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SchemaHelper.scala
@@ -25,7 +25,7 @@ import org.apache.hadoop.fs.{FSDataInputStream, FSDataOutputStream}
import org.json4s.DefaultFormats
import org.json4s.jackson.JsonMethods
-import org.apache.spark.sql.execution.streaming.MetadataVersionUtil
+import org.apache.spark.sql.execution.streaming.checkpointing.MetadataVersionUtil
import org.apache.spark.sql.types.StructType
import org.apache.spark.util.Utils
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala
index 17a36e5210b94..ca18ce9067b36 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala
@@ -25,14 +25,15 @@ import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FSDataInputStream, Path}
import org.apache.spark.SparkUnsupportedOperationException
-import org.apache.spark.internal.{Logging, LogKeys, MDC}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.avro.{AvroDeserializer, AvroSerializer, SchemaConverters}
import org.apache.spark.sql.catalyst.util.UnsafeRowUtils
-import org.apache.spark.sql.execution.streaming.{CheckpointFileManager, StatefulOperatorStateInfo}
+import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager
+import org.apache.spark.sql.execution.streaming.operators.stateful.StatefulOperatorStateInfo
import org.apache.spark.sql.execution.streaming.state.SchemaHelper.{SchemaReader, SchemaWriter}
import org.apache.spark.sql.execution.streaming.state.StateSchemaCompatibilityChecker.SCHEMA_FORMAT_V3
-import org.apache.spark.sql.internal.SessionState
+import org.apache.spark.sql.internal.{SessionState, SQLConf}
import org.apache.spark.sql.types._
// Result returned after validating the schema of the state store for schema changes
@@ -88,7 +89,7 @@ class StateSchemaCompatibilityChecker(
// per query. This variable is the latest one
private val schemaFileLocation = if (oldSchemaFilePaths.isEmpty) {
val storeCpLocation = providerId.storeId.storeCheckpointLocation()
- schemaFile(storeCpLocation)
+ StateSchemaCompatibilityChecker.schemaFile(storeCpLocation)
} else {
oldSchemaFilePaths.last
}
@@ -97,7 +98,7 @@ class StateSchemaCompatibilityChecker(
fm.mkdirs(schemaFileLocation.getParent)
- private val conf = SparkSession.getActiveSession.get.sessionState.conf
+ private val conf = SparkSession.getActiveSession.map(_.sessionState.conf).getOrElse(new SQLConf())
// Read most recent schema file
def readSchemaFile(): List[StateStoreColFamilySchema] = {
@@ -302,9 +303,6 @@ class StateSchemaCompatibilityChecker(
newSchemaFileWritten
}
}
-
- private def schemaFile(storeCpLocation: Path): Path =
- new Path(new Path(storeCpLocation, "_metadata"), "schema")
}
object StateSchemaCompatibilityChecker extends Logging {
@@ -432,4 +430,7 @@ object StateSchemaCompatibilityChecker extends Logging {
StateSchemaValidationResult(evolvedSchema, schemaFileLocation)
}
+
+ def schemaFile(storeCpLocation: Path): Path =
+ new Path(new Path(storeCpLocation, "_metadata"), "schema")
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala
index 9a85169ad451a..604a27866f62e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala
@@ -17,11 +17,13 @@
package org.apache.spark.sql.execution.streaming.state
+import java.io.Closeable
import java.util.UUID
-import java.util.concurrent.{ScheduledFuture, TimeUnit}
+import java.util.concurrent.{ConcurrentLinkedQueue, ScheduledFuture, TimeUnit}
import javax.annotation.concurrent.GuardedBy
import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
import scala.util.control.NonFatal
import org.apache.hadoop.conf.Configuration
@@ -31,16 +33,38 @@ import org.json4s.JsonAST.JValue
import org.json4s.JsonDSL._
import org.json4s.jackson.JsonMethods.{compact, render}
-import org.apache.spark.{SparkContext, SparkEnv, SparkException}
-import org.apache.spark.internal.{Logging, LogKeys, MDC}
+import org.apache.spark.{SparkContext, SparkEnv, SparkException, TaskContext}
+import org.apache.spark.internal.{Logging, LogKeys}
+import org.apache.spark.internal.LogKeys.{EXCEPTION, STATE_STORE_ID, VERSION_NUM}
import org.apache.spark.sql.catalyst.expressions.UnsafeRow
import org.apache.spark.sql.catalyst.util.UnsafeRowUtils
import org.apache.spark.sql.errors.QueryExecutionErrors
import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
-import org.apache.spark.sql.execution.streaming.{StatefulOperatorStateInfo, StreamExecution}
+import org.apache.spark.sql.execution.streaming.operators.stateful.StatefulOperatorStateInfo
+import org.apache.spark.sql.execution.streaming.runtime.StreamExecution
+import org.apache.spark.sql.execution.streaming.state.MaintenanceTaskType._
import org.apache.spark.sql.types.StructType
import org.apache.spark.util.{NextIterator, ThreadUtils, Utils}
+/**
+ * Represents an iterator that provides additional functionalities for state store use cases.
+ *
+ * `close()` is useful for freeing underlying iterator resources when the iterator is no longer
+ * needed.
+ *
+ * The caller MUST call `close()` on the iterator if it was not fully consumed, and it is no
+ * longer needed.
+ */
+class StateStoreIterator[A](
+ val iter: Iterator[A],
+ val onClose: () => Unit = () => {}) extends Iterator[A] with Closeable {
+ override def hasNext: Boolean = iter.hasNext
+
+ override def next(): A = iter.next()
+
+ override def close(): Unit = onClose()
+}
+
sealed trait StateStoreEncoding {
override def toString: String = this match {
case StateStoreEncoding.UnsafeRow => "unsaferow"
@@ -53,6 +77,14 @@ object StateStoreEncoding {
case object Avro extends StateStoreEncoding
}
+sealed trait MaintenanceTaskType
+
+object MaintenanceTaskType {
+ case object FromUnloadedProvidersQueue extends MaintenanceTaskType
+ case object FromTaskThread extends MaintenanceTaskType
+ case object FromLoadedProviders extends MaintenanceTaskType
+}
+
/**
* Base trait for a versioned key-value store which provides read operations. Each instance of a
* `ReadStateStore` represents a specific version of state data, and such instances are created
@@ -106,10 +138,11 @@ trait ReadStateStore {
*/
def prefixScan(
prefixKey: UnsafeRow,
- colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Iterator[UnsafeRowPair]
+ colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): StateStoreIterator[UnsafeRowPair]
/** Return an iterator containing all the key-value pairs in the StateStore. */
- def iterator(colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Iterator[UnsafeRowPair]
+ def iterator(
+ colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): StateStoreIterator[UnsafeRowPair]
/**
* Clean up the resource.
@@ -117,6 +150,22 @@ trait ReadStateStore {
* The method name is to respect backward compatibility on [[StateStore]].
*/
def abort(): Unit
+
+
+ /**
+ * Releases resources associated with this read-only state store.
+ *
+ * This method should be called when the store is no longer needed but has completed
+ * successfully (i.e., no errors occurred during reading). It performs any necessary
+ * cleanup operations without invalidating or rolling back the data that was read.
+ *
+ * In contrast to `abort()`, which is called on error paths to cancel operations,
+ * `release()` is the proper method to call in success scenarios when a read-only
+ * store is no longer needed.
+ *
+ * This method is idempotent and safe to call multiple times.
+ */
+ def release(): Unit
}
/**
@@ -188,6 +237,10 @@ trait StateStore extends ReadStateStore {
*/
override def abort(): Unit
+ override def release(): Unit = {
+ throw new UnsupportedOperationException("Should only call release() on ReadStateStore")
+ }
+
/**
* Return an iterator containing all the key-value pairs in the StateStore. Implementations must
* ensure that updates (puts, removes) can be made while iterating over this iterator.
@@ -196,8 +249,8 @@ trait StateStore extends ReadStateStore {
* performed after initialization of the iterator. Callers should perform all updates before
* calling this method if all updates should be visible in the returned iterator.
*/
- override def iterator(colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME):
- Iterator[UnsafeRowPair]
+ override def iterator(colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME)
+ : StateStoreIterator[UnsafeRowPair]
/** Current metrics of the state store */
def metrics: StateStoreMetrics
@@ -229,14 +282,16 @@ class WrappedReadStateStore(store: StateStore) extends ReadStateStore {
colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): UnsafeRow = store.get(key,
colFamilyName)
- override def iterator(colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME):
- Iterator[UnsafeRowPair] = store.iterator(colFamilyName)
+ override def iterator(colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME)
+ : StateStoreIterator[UnsafeRowPair] = store.iterator(colFamilyName)
override def abort(): Unit = store.abort()
+ override def release(): Unit = store.release()
+
override def prefixScan(prefixKey: UnsafeRow,
- colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Iterator[UnsafeRowPair] =
- store.prefixScan(prefixKey, colFamilyName)
+ colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME)
+ : StateStoreIterator[UnsafeRowPair] = store.prefixScan(prefixKey, colFamilyName)
override def valuesIterator(key: UnsafeRow, colFamilyName: String): Iterator[UnsafeRow] = {
store.valuesIterator(key, colFamilyName)
@@ -554,7 +609,11 @@ trait StateStoreProvider {
*/
def stateStoreId: StateStoreId
- /** Called when the provider instance is unloaded from the executor */
+ /**
+ * Called when the provider instance is unloaded from the executor
+ * WARNING: IF PROVIDER FROM [[StateStore.loadedProviders]],
+ * CLOSE MUST ONLY BE CALLED FROM MAINTENANCE THREAD!
+ */
def close(): Unit
/**
@@ -575,6 +634,30 @@ trait StateStoreProvider {
def getReadStore(version: Long, uniqueId: Option[String] = None): ReadStateStore =
new WrappedReadStateStore(getStore(version, uniqueId))
+ /**
+ * Creates a writable store from an existing read-only store for the specified version.
+ *
+ * This method enables an important optimization pattern for stateful operations where
+ * the same state store needs to be accessed for both reading and writing within a task.
+ * Instead of opening two separate state store instances (which can cause contention issues),
+ * this method converts an existing read-only store to a writable store that can commit changes.
+ *
+ * This approach is particularly beneficial when:
+ * - A stateful operation needs to first read the existing state, then update it
+ * - The state store has locking mechanisms that prevent concurrent access
+ * - Multiple state store connections would cause unnecessary resource duplication
+ *
+ * @param readStore The existing read-only store instance to convert to a writable store
+ * @param version The version of the state store (must match the read store's version)
+ * @param uniqueId Optional unique identifier for checkpointing
+ * @return A writable StateStore instance that can be used to update and commit changes
+ */
+ def upgradeReadStoreToWriteStore(
+ readStore: ReadStateStore,
+ version: Long,
+ uniqueId: Option[String] = None): StateStore = getStore(version, uniqueId)
+
+
/** Optional method for providers to allow for background maintenance (e.g. compactions) */
def doMaintenance(): Unit = { }
@@ -664,7 +747,7 @@ object StateStoreProvider extends Logging {
}
/**
- * Get the runId from the provided hadoopConf. If it is not found, generate a random UUID.
+ * Get the runId from the provided hadoopConf. If it is not found, an error will be thrown.
*
* @param hadoopConf Hadoop configuration used by the StateStore to save state data
*/
@@ -717,7 +800,8 @@ trait SupportsFineGrainedReplay {
* @param snapshotVersion checkpoint version of the snapshot to start with
* @param endVersion checkpoint version to end with
*/
- def replayStateFromSnapshot(snapshotVersion: Long, endVersion: Long): StateStore
+ def replayStateFromSnapshot(
+ snapshotVersion: Long, endVersion: Long, readOnly: Boolean = false): StateStore
/**
* Return an instance of [[ReadStateStore]] representing state data of the given version.
@@ -730,7 +814,7 @@ trait SupportsFineGrainedReplay {
* @param endVersion checkpoint version to end with
*/
def replayReadStateFromSnapshot(snapshotVersion: Long, endVersion: Long): ReadStateStore = {
- new WrappedReadStateStore(replayStateFromSnapshot(snapshotVersion, endVersion))
+ new WrappedReadStateStore(replayStateFromSnapshot(snapshotVersion, endVersion, readOnly = true))
}
/**
@@ -759,7 +843,11 @@ trait SupportsFineGrainedReplay {
* Note that `queryRunId` is used uniquely identify a provider, so that the same provider
* instance is not reused across query restarts.
*/
-case class StateStoreProviderId(storeId: StateStoreId, queryRunId: UUID)
+case class StateStoreProviderId(storeId: StateStoreId, queryRunId: UUID) {
+ override def toString: String = {
+ s"StateStoreProviderId[ storeId=$storeId, queryRunId=$queryRunId ]"
+ }
+}
object StateStoreProviderId {
private[sql] def apply(
@@ -802,9 +890,7 @@ case class StateStoreId(
}
override def toString: String = {
- s"""StateStoreId[ checkpointRootLocation=$checkpointRootLocation, operatorId=$operatorId,
- | partitionId=$partitionId, storeName=$storeName ]
- |""".stripMargin.replaceAll("\n", "")
+ s"StateStoreId[ operatorId=$operatorId, partitionId=$partitionId, storeName=$storeName ]"
}
}
@@ -841,12 +927,37 @@ object StateStore extends Logging {
private val maintenanceThreadPoolLock = new Object
+ private val unloadedProvidersToClose =
+ new ConcurrentLinkedQueue[(StateStoreProviderId, StateStoreProvider)]
+
// This set is to keep track of the partitions that are queued
// for maintenance or currently have maintenance running on them
// to prevent the same partition from being processed concurrently.
@GuardedBy("maintenanceThreadPoolLock")
private val maintenancePartitions = new mutable.HashSet[StateStoreProviderId]
+ /** Reports to the coordinator that a StateStore has committed */
+ def reportCommitToCoordinator(
+ version: Long,
+ stateStoreId: StateStoreId,
+ hadoopConf: Configuration): Unit = {
+ try {
+ val runId = UUID.fromString(StateStoreProvider.getRunId(hadoopConf))
+ val providerId = StateStoreProviderId(stateStoreId, runId)
+ // The coordinator will handle whether tracking is active for this batch
+ // If tracking is not active, it will just reply without processing
+ StateStoreProvider.coordinatorRef.foreach(
+ _.reportStateStoreCommit(providerId, version, stateStoreId.storeName)
+ )
+ logDebug(log"Reported commit for store " +
+ log"${MDC(STATE_STORE_ID, stateStoreId)} at version ${MDC(VERSION_NUM, version)}")
+ } catch {
+ case NonFatal(e) =>
+ // Log but don't fail the commit if reporting fails
+ logWarning(log"Failed to report StateStore commit: ${MDC(EXCEPTION, e)}")
+ }
+ }
+
/**
* Runs the `task` periodically and bubbles any exceptions that it encounters.
*
@@ -948,6 +1059,56 @@ object StateStore extends Logging {
storeProvider.getReadStore(version, stateStoreCkptId)
}
+ /**
+ * Converts an existing read-only state store to a writable state store.
+ *
+ * This method provides an optimization for stateful operations that need to both read and update
+ * state within the same task. Instead of opening separate read and write instances (which may
+ * cause resource contention or duplication), this method reuses the already loaded read store
+ * and transforms it into a writable store.
+ *
+ * The optimization is particularly valuable for state stores with expensive initialization costs
+ * or limited concurrency capabilities (like RocksDB). It eliminates redundant loading of the same
+ * state data and reduces resource usage.
+ *
+ * @param readStore The existing read-only state store to convert to a writable store
+ * @param storeProviderId Unique identifier for the state store provider
+ * @param keySchema Schema of the state store keys
+ * @param valueSchema Schema of the state store values
+ * @param keyStateEncoderSpec Specification for encoding the state keys
+ * @param version The version of the state store (must match the read store's version)
+ * @param stateStoreCkptId Optional checkpoint identifier for the state store
+ * @param stateSchemaBroadcast Optional broadcast of the state schema
+ * @param useColumnFamilies Whether to use column families in the state store
+ * @param storeConf Configuration for the state store
+ * @param hadoopConf Hadoop configuration
+ * @param useMultipleValuesPerKey Whether the store supports multiple values per key
+ * @return A writable StateStore instance that can be used to update and commit changes
+ * @throws SparkException If the store cannot be loaded or if there's insufficient memory
+ */
+ def getWriteStore(
+ readStore: ReadStateStore,
+ storeProviderId: StateStoreProviderId,
+ keySchema: StructType,
+ valueSchema: StructType,
+ keyStateEncoderSpec: KeyStateEncoderSpec,
+ version: Long,
+ stateStoreCkptId: Option[String],
+ stateSchemaBroadcast: Option[StateSchemaBroadcast],
+ useColumnFamilies: Boolean,
+ storeConf: StateStoreConf,
+ hadoopConf: Configuration,
+ useMultipleValuesPerKey: Boolean = false): StateStore = {
+ hadoopConf.set(StreamExecution.RUN_ID_KEY, storeProviderId.queryRunId.toString)
+ if (version < 0) {
+ throw QueryExecutionErrors.unexpectedStateStoreVersion(version)
+ }
+ val storeProvider = getStateStoreProvider(storeProviderId, keySchema, valueSchema,
+ keyStateEncoderSpec, useColumnFamilies, storeConf, hadoopConf, useMultipleValuesPerKey,
+ stateSchemaBroadcast)
+ storeProvider.upgradeReadStoreToWriteStore(readStore, version, stateStoreCkptId)
+ }
+
/** Get or create a store associated with the id. */
def get(
storeProviderId: StateStoreProviderId,
@@ -1010,7 +1171,21 @@ object StateStore extends Logging {
if (!storeConf.unloadOnCommit) {
val otherProviderIds = loadedProviders.keys.filter(_ != storeProviderId).toSeq
val providerIdsToUnload = reportActiveStoreInstance(storeProviderId, otherProviderIds)
- providerIdsToUnload.foreach(unload(_))
+ val taskContextIdLogLine = Option(TaskContext.get()).map { tc =>
+ log"taskId=${MDC(LogKeys.TASK_ID, tc.taskAttemptId())}"
+ }.getOrElse(log"")
+ providerIdsToUnload.foreach(id => {
+ loadedProviders.remove(id).foreach( provider => {
+ // Trigger maintenance thread to immediately do maintenance on and close the provider.
+ // Doing maintenance first allows us to do maintenance for a constantly-moving state
+ // store.
+ logInfo(log"Submitted maintenance from task thread to close " +
+ log"provider=${MDC(LogKeys.STATE_STORE_PROVIDER_ID, id)}." + taskContextIdLogLine +
+ log"Removed provider from loadedProviders")
+ submitMaintenanceWorkForProvider(
+ id, provider, storeConf, MaintenanceTaskType.FromTaskThread)
+ })
+ })
}
provider
@@ -1027,14 +1202,30 @@ object StateStore extends Logging {
}
}
- /** Unload a state store provider */
- def unload(storeProviderId: StateStoreProviderId): Unit = loadedProviders.synchronized {
- loadedProviders.remove(storeProviderId).foreach(_.close())
+ /**
+ * Unload a state store provider.
+ * If alreadyRemovedFromLoadedProviders is None, provider will be
+ * removed from loadedProviders and closed.
+ * If alreadyRemovedFromLoadedProviders is Some, provider will be closed
+ * using passed in provider.
+ * WARNING: CAN ONLY BE CALLED FROM MAINTENANCE THREAD!
+ */
+ def removeFromLoadedProvidersAndClose(
+ storeProviderId: StateStoreProviderId,
+ alreadyRemovedProvider: Option[StateStoreProvider] = None): Unit = {
+ val providerToClose = alreadyRemovedProvider.orElse {
+ loadedProviders.synchronized {
+ loadedProviders.remove(storeProviderId)
+ }
+ }
+ providerToClose.foreach { provider =>
+ provider.close()
+ }
}
/** Unload all state store providers: unit test purpose */
private[sql] def unloadAll(): Unit = loadedProviders.synchronized {
- loadedProviders.keySet.foreach { key => unload(key) }
+ loadedProviders.keySet.foreach { key => removeFromLoadedProvidersAndClose(key) }
loadedProviders.clear()
}
@@ -1073,7 +1264,7 @@ object StateStore extends Logging {
/** Unload and stop all state store providers */
def stop(): Unit = loadedProviders.synchronized {
- loadedProviders.keySet.foreach { key => unload(key) }
+ loadedProviders.keySet.foreach { key => removeFromLoadedProvidersAndClose(key) }
loadedProviders.clear()
_coordRef = null
stopMaintenanceTask()
@@ -1088,7 +1279,7 @@ object StateStore extends Logging {
if (SparkEnv.get != null && !isMaintenanceRunning && !storeConf.unloadOnCommit) {
maintenanceTask = new MaintenanceTask(
storeConf.maintenanceInterval,
- task = { doMaintenance() }
+ task = { doMaintenance(storeConf) }
)
maintenanceThreadPool = new MaintenanceThreadPool(numMaintenanceThreads,
maintenanceShutdownTimeout)
@@ -1097,6 +1288,27 @@ object StateStore extends Logging {
}
}
+ // Wait until this partition can be processed
+ private def awaitProcessThisPartition(
+ id: StateStoreProviderId,
+ timeoutMs: Long): Boolean = maintenanceThreadPoolLock synchronized {
+ val startTime = System.currentTimeMillis()
+ val endTime = startTime + timeoutMs
+
+ // If immediate processing fails, wait with timeout
+ var canProcessThisPartition = processThisPartition(id)
+ while (!canProcessThisPartition && System.currentTimeMillis() < endTime) {
+ maintenanceThreadPoolLock.wait(timeoutMs)
+ canProcessThisPartition = processThisPartition(id)
+ }
+ val elapsedTime = System.currentTimeMillis() - startTime
+ logInfo(log"Waited for ${MDC(LogKeys.TOTAL_TIME, elapsedTime)} ms to be able to process " +
+ log"maintenance for partition ${MDC(LogKeys.STATE_STORE_PROVIDER_ID, id)}")
+ canProcessThisPartition
+ }
+
+ private def doMaintenance(): Unit = doMaintenance(StateStoreConf.empty)
+
private def processThisPartition(id: StateStoreProviderId): Boolean = {
maintenanceThreadPoolLock.synchronized {
if (!maintenancePartitions.contains(id)) {
@@ -1112,56 +1324,42 @@ object StateStore extends Logging {
* Execute background maintenance task in all the loaded store providers if they are still
* the active instances according to the coordinator.
*/
- private def doMaintenance(): Unit = {
+ private def doMaintenance(storeConf: StateStoreConf): Unit = {
logDebug("Doing maintenance")
if (SparkEnv.get == null) {
throw new IllegalStateException("SparkEnv not active, cannot do maintenance on StateStores")
}
+
+ // Providers that couldn't be processed now and need to be added back to the queue
+ val providersToRequeue = new ArrayBuffer[(StateStoreProviderId, StateStoreProvider)]()
+
+ // unloadedProvidersToClose are StateStoreProviders that have been removed from
+ // loadedProviders, and can now be processed for maintenance. This queue contains
+ // providers for which we weren't able to process for maintenance on the previous iteration
+ while (!unloadedProvidersToClose.isEmpty) {
+ val (providerId, provider) = unloadedProvidersToClose.poll()
+
+ if (processThisPartition(providerId)) {
+ submitMaintenanceWorkForProvider(
+ providerId, provider, storeConf, MaintenanceTaskType.FromUnloadedProvidersQueue)
+ } else {
+ providersToRequeue += ((providerId, provider))
+ }
+ }
+
+ if (providersToRequeue.nonEmpty) {
+ logInfo(log"Had to requeue ${MDC(LogKeys.SIZE, providersToRequeue.size)} providers " +
+ log"for maintenance in doMaintenance")
+ }
+
+ providersToRequeue.foreach(unloadedProvidersToClose.offer)
+
loadedProviders.synchronized {
loadedProviders.toSeq
}.foreach { case (id, provider) =>
if (processThisPartition(id)) {
- maintenanceThreadPool.execute(() => {
- val startTime = System.currentTimeMillis()
- try {
- provider.doMaintenance()
- if (!verifyIfStoreInstanceActive(id)) {
- unload(id)
- logInfo(log"Unloaded ${MDC(LogKeys.STATE_STORE_PROVIDER, provider)}")
- }
- } catch {
- case NonFatal(e) =>
- logWarning(log"Error managing ${MDC(LogKeys.STATE_STORE_PROVIDER, provider)}, " +
- log"unloading state store provider", e)
- // When we get a non-fatal exception, we just unload the provider.
- //
- // By not bubbling the exception to the maintenance task thread or the query execution
- // thread, it's possible for a maintenance thread pool task to continue failing on
- // the same partition. Additionally, if there is some global issue that will cause
- // all maintenance thread pool tasks to fail, then bubbling the exception and
- // stopping the pool is faster than waiting for all tasks to see the same exception.
- //
- // However, we assume that repeated failures on the same partition and global issues
- // are rare. The benefit to unloading just the partition with an exception is that
- // transient issues on a given provider do not affect any other providers; so, in
- // most cases, this should be a more performant solution.
- unload(id)
- } finally {
- val duration = System.currentTimeMillis() - startTime
- val logMsg =
- log"Finished maintenance task for " +
- log"provider=${MDC(LogKeys.STATE_STORE_PROVIDER_ID, id)}" +
- log" in elapsed_time=${MDC(LogKeys.TIME_UNITS, duration)}\n"
- if (duration > 5000) {
- logInfo(logMsg)
- } else {
- logDebug(logMsg)
- }
- maintenanceThreadPoolLock.synchronized {
- maintenancePartitions.remove(id)
- }
- }
- })
+ submitMaintenanceWorkForProvider(
+ id, provider, storeConf, MaintenanceTaskType.FromLoadedProviders)
} else {
logInfo(log"Not processing partition ${MDC(LogKeys.PARTITION_ID, id)} " +
log"for maintenance because it is currently " +
@@ -1170,6 +1368,108 @@ object StateStore extends Logging {
}
}
+ /**
+ * Submits maintenance work for a provider to the maintenance thread pool.
+ *
+ * @param id The StateStore provider ID to perform maintenance on
+ * @param provider The StateStore provider instance
+ */
+ private def submitMaintenanceWorkForProvider(
+ id: StateStoreProviderId,
+ provider: StateStoreProvider,
+ storeConf: StateStoreConf,
+ source: MaintenanceTaskType = FromLoadedProviders): Unit = {
+ maintenanceThreadPool.execute(() => {
+ val startTime = System.currentTimeMillis()
+ // Determine if we can process this partition based on the source
+ val canProcessThisPartition = source match {
+ case FromTaskThread =>
+ // Provider from task thread needs to wait for lock
+ // We potentially need to wait for ongoing maintenance to finish processing
+ // this partition
+ val timeoutMs = storeConf.stateStoreMaintenanceProcessingTimeout * 1000
+ val ableToProcessNow = awaitProcessThisPartition(id, timeoutMs)
+ if (!ableToProcessNow) {
+ // Add to queue for later processing if we can't process now
+ // This will be resubmitted for maintenance later by the background maintenance task
+ unloadedProvidersToClose.add((id, provider))
+ }
+ ableToProcessNow
+
+ case FromUnloadedProvidersQueue =>
+ // Provider from queue can be processed immediately
+ // (we've already removed it from loadedProviders)
+ true
+
+ case FromLoadedProviders =>
+ // Provider from loadedProviders can be processed immediately
+ // as it's in maintenancePartitions
+ true
+ }
+
+ if (canProcessThisPartition) {
+ val awaitingPartitionDuration = System.currentTimeMillis() - startTime
+ try {
+ provider.doMaintenance()
+ // Handle unloading based on source
+ source match {
+ case FromTaskThread | FromUnloadedProvidersQueue =>
+ // Provider already removed from loadedProviders, just close it
+ removeFromLoadedProvidersAndClose(id, Some(provider))
+
+ case FromLoadedProviders =>
+ // Check if provider should be unloaded
+ if (!verifyIfStoreInstanceActive(id)) {
+ removeFromLoadedProvidersAndClose(id)
+ }
+ }
+ logInfo(log"Unloaded ${MDC(LogKeys.STATE_STORE_PROVIDER_ID, id)}")
+ } catch {
+ case NonFatal(e) =>
+ logWarning(log"Error doing maintenance on provider:" +
+ log" ${MDC(LogKeys.STATE_STORE_PROVIDER_ID, id)}. " +
+ log"Could not unload state store provider", e)
+ // When we get a non-fatal exception, we just unload the provider.
+ //
+ // By not bubbling the exception to the maintenance task thread or the query execution
+ // thread, it's possible for a maintenance thread pool task to continue failing on
+ // the same partition. Additionally, if there is some global issue that will cause
+ // all maintenance thread pool tasks to fail, then bubbling the exception and
+ // stopping the pool is faster than waiting for all tasks to see the same exception.
+ //
+ // However, we assume that repeated failures on the same partition and global issues
+ // are rare. The benefit to unloading just the partition with an exception is that
+ // transient issues on a given provider do not affect any other providers; so, in
+ // most cases, this should be a more performant solution.
+ source match {
+ case FromTaskThread | FromUnloadedProvidersQueue =>
+ removeFromLoadedProvidersAndClose(id, Some(provider))
+
+ case FromLoadedProviders =>
+ removeFromLoadedProvidersAndClose(id)
+ }
+ } finally {
+ val duration = System.currentTimeMillis() - startTime
+ val logMsg =
+ log"Finished maintenance task for " +
+ log"provider=${MDC(LogKeys.STATE_STORE_PROVIDER_ID, id)}" +
+ log" in elapsed_time=${MDC(LogKeys.TIME_UNITS, duration)}" +
+ log" and awaiting_partition_time=" +
+ log"${MDC(LogKeys.TIME_UNITS, awaitingPartitionDuration)}\n"
+ if (duration > 5000) {
+ logInfo(logMsg)
+ } else {
+ logDebug(logMsg)
+ }
+ maintenanceThreadPoolLock.synchronized {
+ maintenancePartitions.remove(id)
+ maintenanceThreadPoolLock.notifyAll()
+ }
+ }
+ }
+ })
+ }
+
private def reportActiveStoreInstance(
storeProviderId: StateStoreProviderId,
otherProviderIds: Seq[StateStoreProviderId]): Seq[StateStoreProviderId] = {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala
index bcaff4c60d08f..4c5dea63baeab 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala
@@ -21,21 +21,20 @@ import java.io.{DataInputStream, DataOutputStream, FileNotFoundException, IOExce
import scala.util.control.NonFatal
-import com.google.common.io.ByteStreams
-import org.apache.commons.io.IOUtils
import org.apache.hadoop.fs.{FSError, Path}
import org.json4s._
import org.json4s.jackson.Serialization
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys._
import org.apache.spark.io.CompressionCodec
import org.apache.spark.sql.catalyst.expressions.UnsafeRow
import org.apache.spark.sql.errors.QueryExecutionErrors
-import org.apache.spark.sql.execution.streaming.CheckpointFileManager
-import org.apache.spark.sql.execution.streaming.CheckpointFileManager.CancellableFSDataOutputStream
+import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager
+import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager.CancellableFSDataOutputStream
import org.apache.spark.sql.execution.streaming.state.RecordType.RecordType
import org.apache.spark.util.NextIterator
+import org.apache.spark.util.Utils
/**
* Enum used to write record types to changelog files used with RocksDBStateStoreProvider.
@@ -132,7 +131,7 @@ abstract class StateStoreChangelogWriter(
def abort(): Unit = {
try {
if (backingFileStream != null) backingFileStream.cancel()
- if (compressedStream != null) IOUtils.closeQuietly(compressedStream)
+ if (compressedStream != null) Utils.closeQuietly(compressedStream)
} catch {
// Closing the compressedStream causes the stream to write/flush data into the
// rawStream. Since the rawStream is already closed, there may be errors.
@@ -371,7 +370,11 @@ class StateStoreChangelogReaderFactory(
// Or if the first record in the changelog file in V1 has a large enough
// key, readUTF() will throw a UTFDataFormatException so we should return
// version 1 (SPARK-51922).
- case _: java.io.EOFException | _: java.io.UTFDataFormatException => 1
+ case _: java.io.EOFException |
+ _: java.io.UTFDataFormatException |
+ // SPARK-52553 - Can throw this if the bytes in the file is coincidentally
+ // decoded as UTF string like "v)".
+ _: NumberFormatException => 1
}
}
@@ -481,14 +484,14 @@ class StateStoreChangelogReaderV1(
} else {
// TODO: reuse the key buffer and value buffer across records.
val keyBuffer = new Array[Byte](keySize)
- ByteStreams.readFully(input, keyBuffer, 0, keySize)
+ Utils.readFully(input, keyBuffer, 0, keySize)
val valueSize = input.readInt()
if (valueSize < 0) {
// A deletion record
(RecordType.DELETE_RECORD, keyBuffer, null)
} else {
val valueBuffer = new Array[Byte](valueSize)
- ByteStreams.readFully(input, valueBuffer, 0, valueSize)
+ Utils.readFully(input, valueBuffer, 0, valueSize)
// A put record.
(RecordType.PUT_RECORD, keyBuffer, valueBuffer)
}
@@ -512,7 +515,7 @@ class StateStoreChangelogReaderV2(
private def parseBuffer(input: DataInputStream): Array[Byte] = {
val blockSize = input.readInt()
val blockBuffer = new Array[Byte](blockSize)
- ByteStreams.readFully(input, blockBuffer, 0, blockSize)
+ Utils.readFully(input, blockBuffer, 0, blockSize)
blockBuffer
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreConf.scala
index 9a994200baeb0..00bb7de46dc4d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreConf.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreConf.scala
@@ -17,12 +17,14 @@
package org.apache.spark.sql.execution.streaming.state
-import org.apache.spark.sql.execution.streaming.StatefulOperatorStateInfo
+import org.apache.spark.sql.execution.streaming.operators.stateful.StatefulOperatorStateInfo
import org.apache.spark.sql.internal.SQLConf
/** A class that contains configuration parameters for [[StateStore]]s. */
class StateStoreConf(
- @transient private[state] val sqlConf: SQLConf,
+ // Should be private because it could be null under serialization (due to
+ // the transient annotation)
+ @transient private val sqlConf: SQLConf,
val extraOptions: Map[String, String] = Map.empty)
extends Serializable {
@@ -38,6 +40,8 @@ class StateStoreConf(
*/
val stateStoreMaintenanceShutdownTimeout: Long = sqlConf.stateStoreMaintenanceShutdownTimeout
+ val stateStoreMaintenanceProcessingTimeout: Long = sqlConf.stateStoreMaintenanceProcessingTimeout
+
/**
* Minimum number of delta files in a chain after which HDFSBackedStateStore will
* consider generating a snapshot.
@@ -67,6 +71,13 @@ class StateStoreConf(
/** Whether validate the underlying format or not. */
val formatValidationEnabled: Boolean = sqlConf.stateStoreFormatValidationEnabled
+ /**
+ * Whether to validate StateStore commits for ForeachBatch sinks to ensure all partitions
+ * are processed. This helps detect incomplete processing due to operations like show()
+ * or limit().
+ */
+ val commitValidationEnabled = sqlConf.stateStoreCommitValidationEnabled
+
/**
* Whether to validate the value side. This config is applied to both validators as below:
*
@@ -106,6 +117,9 @@ class StateStoreConf(
/** Whether to unload the store on task completion. */
val unloadOnCommit = sqlConf.stateStoreUnloadOnCommit
+ /** The version of the state store checkpoint format. */
+ val stateStoreCheckpointFormatVersion: Int = sqlConf.stateStoreCheckpointFormatVersion
+
/**
* Additional configurations related to state store. This will capture all configs in
* SQLConf that start with `spark.sql.streaming.stateStore.`
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinator.scala
index 903f27fb2a223..37e35c917a521 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinator.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinator.scala
@@ -22,7 +22,7 @@ import java.util.UUID
import scala.collection.mutable
import org.apache.spark.SparkEnv
-import org.apache.spark.internal.{Logging, LogKeys, MDC}
+import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint}
import org.apache.spark.scheduler.ExecutorCacheTaskLocation
import org.apache.spark.sql.internal.SQLConf
@@ -53,6 +53,24 @@ private case class VerifyIfInstanceActive(storeId: StateStoreProviderId, executo
private case class GetLocation(storeId: StateStoreProviderId)
extends StateStoreCoordinatorMessage
+/** Report that a StateStore has committed for tracking purposes */
+private case class ReportStateStoreCommit(
+ storeId: StateStoreProviderId,
+ version: Long,
+ storeName: String = StateStoreId.DEFAULT_STORE_NAME)
+ extends StateStoreCoordinatorMessage
+
+/** Start tracking StateStore commits for a batch */
+private case class StartStateStoreCommitTrackingForBatch(
+ runId: UUID,
+ batchId: Long,
+ expectedStores: Map[Long, Map[String, Int]]) // operatorId -> (storeName -> numPartitions)
+ extends StateStoreCoordinatorMessage
+
+/** Validate that all expected StateStores have committed for a batch */
+private case class ValidateStateStoreCommitForBatch(runId: UUID, batchId: Long)
+ extends StateStoreCoordinatorMessage
+
private case class DeactivateInstances(runId: UUID)
extends StateStoreCoordinatorMessage
@@ -176,6 +194,29 @@ class StateStoreCoordinatorRef private(rpcEndpointRef: RpcEndpointRef) {
LogLaggingStateStores(queryRunId, latestVersion, isTerminatingTrigger))
}
+
+ /** Start tracking StateStore commits for a batch */
+ private[sql] def startStateStoreCommitTrackingForBatch(
+ runId: UUID,
+ batchId: Long,
+ expectedStores: Map[Long, Map[String, Int]]): Unit = {
+ rpcEndpointRef.askSync[Unit](
+ StartStateStoreCommitTrackingForBatch(runId, batchId, expectedStores))
+ }
+
+ /** Report that a StateStore has committed */
+ private[sql] def reportStateStoreCommit(
+ storeId: StateStoreProviderId,
+ version: Long,
+ storeName: String = StateStoreId.DEFAULT_STORE_NAME): Unit = {
+ rpcEndpointRef.askSync[Unit](ReportStateStoreCommit(storeId, version, storeName))
+ }
+
+ /** Validate that all expected StateStores have committed for a batch */
+ private[sql] def validateStateStoreCommitForBatch(runId: UUID, batchId: Long): Unit = {
+ rpcEndpointRef.askSync[Unit](ValidateStateStoreCommitForBatch(runId, batchId))
+ }
+
/**
* Endpoint used for testing.
* Get the latest snapshot version uploaded for a state store.
@@ -222,6 +263,10 @@ private class StateStoreCoordinator(
// Default snapshot upload event to use when a provider has never uploaded a snapshot
private val defaultSnapshotUploadEvent = SnapshotUploadEvent(0, 0)
+ // Tracking structure for StateStore commits per batch
+ // Key: (runId, batchId) -> Value: CommitTracker
+ private val batchCommitTrackers = new mutable.HashMap[(UUID, Long), BatchCommitTracker]
+
// Stores the last timestamp in milliseconds for each queryRunId indicating when the
// coordinator did a report on instances lagging behind on snapshot uploads.
// The initial timestamp is defaulted to 0 milliseconds.
@@ -264,6 +309,10 @@ private class StateStoreCoordinator(
val storeIdsToRemove =
instances.keys.filter(_.queryRunId == runId).toSeq
instances --= storeIdsToRemove
+
+ val runIdsToRemove = batchCommitTrackers.keys.filter(_._1 == runId)
+ batchCommitTrackers --= runIdsToRemove
+
// Also remove these instances from snapshot upload event tracking
stateStoreLatestUploadedSnapshot --= storeIdsToRemove
// Remove the corresponding run id entries for report time and starting time
@@ -336,6 +385,49 @@ private class StateStoreCoordinator(
}
context.reply(true)
+ case StartStateStoreCommitTrackingForBatch(runId, batchId, expectedStores) =>
+ val key = (runId, batchId)
+ if (batchCommitTrackers.contains(key)) {
+ context.sendFailure(new IllegalStateException(
+ s"Batch commit tracker already exists for runId=$runId, batchId=$batchId"))
+ } else {
+ batchCommitTrackers.put(key, new BatchCommitTracker(runId, batchId, expectedStores))
+ logInfo(s"Started tracking commits for batch $batchId with " +
+ s"${expectedStores.values.map(_.values.sum).sum} expected stores")
+ context.reply(())
+ }
+
+ case ReportStateStoreCommit(storeId, version, storeName) =>
+ // StateStore version = batchId + 1, so we need to adjust
+ val batchId = version - 1
+ val key = (storeId.queryRunId, batchId)
+ batchCommitTrackers.get(key) match {
+ case Some(tracker) =>
+ tracker.recordCommit(storeId, storeName)
+ context.reply(())
+ case None =>
+ // In case no commit tracker for this batch was found
+ context.reply(())
+ }
+
+ case ValidateStateStoreCommitForBatch(runId, batchId) =>
+ val key = (runId, batchId)
+ batchCommitTrackers.get(key) match {
+ case Some(tracker) =>
+ try {
+ tracker.validateAllCommitted()
+ batchCommitTrackers.remove(key) // Clean up after validation
+ context.reply(())
+ } catch {
+ case e: StateStoreCommitValidationFailed =>
+ batchCommitTrackers.remove(key) // Clean up even on failure
+ context.sendFailure(e)
+ }
+ case None =>
+ context.sendFailure(new IllegalStateException(
+ s"No commit tracker found for runId=$runId, batchId=$batchId"))
+ }
+
case GetLatestSnapshotVersionForTesting(providerId) =>
val version = stateStoreLatestUploadedSnapshot.get(providerId).map(_.version)
logDebug(s"Got latest snapshot version of the state store $providerId: $version")
@@ -402,6 +494,55 @@ private class StateStoreCoordinator(
}
}
+/**
+ * Tracks StateStore commits for a batch to ensure all expected stores commit
+ */
+private class BatchCommitTracker(
+ runId: UUID,
+ batchId: Long,
+ expectedStores: Map[Long, Map[String, Int]]) extends Logging {
+
+ // Track committed stores: (operatorId, partitionId, storeName) -> committed
+ private val committedStores = new mutable.HashSet[(Long, Int, String)]()
+
+ def recordCommit(storeId: StateStoreProviderId, storeName: String): Unit = {
+ val key = (storeId.storeId.operatorId, storeId.storeId.partitionId, storeName)
+ committedStores.add(key)
+ logDebug(s"Recorded commit for store $storeId with name $storeName for batch $batchId")
+ }
+
+ def validateAllCommitted(): Unit = {
+ val missingCommits = new mutable.ArrayBuffer[String]()
+
+ expectedStores.foreach { case (operatorId, storeMap) =>
+ storeMap.foreach { case (storeName, numPartitions) =>
+ for (partitionId <- 0 until numPartitions) {
+ val key = (operatorId, partitionId, storeName)
+ if (!committedStores.contains(key)) {
+ missingCommits += s"(operator=$operatorId, partition=$partitionId, store=$storeName)"
+ }
+ }
+ }
+ }
+
+ if (missingCommits.nonEmpty) {
+ val totalExpected = expectedStores.values.map(_.values.sum).sum
+ val errorMsg = s"Not all StateStores committed for batch $batchId. " +
+ s"Expected $totalExpected commits but got ${committedStores.size}. " +
+ s"Missing commits: ${missingCommits.mkString(", ")}"
+ logError(errorMsg)
+ throw StateStoreErrors.stateStoreCommitValidationFailed(
+ batchId,
+ totalExpected,
+ committedStores.size,
+ missingCommits.mkString(", ")
+ )
+ }
+
+ logInfo(s"All ${committedStores.size} StateStores successfully committed for batch $batchId")
+ }
+}
+
case class SnapshotUploadEvent(
version: Long,
timestamp: Long
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreErrors.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreErrors.scala
index 633fd96f90567..43682de034462 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreErrors.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreErrors.scala
@@ -92,6 +92,18 @@ object StateStoreErrors {
new StateStoreIncorrectNumOrderingColsForPrefixScan(numPrefixCols)
}
+ def invalidStateMachineTransition(
+ oldState: String,
+ newState: String,
+ transition: String,
+ storeId: StateStoreId): StateStoreInvalidStateMachineTransition = {
+ new StateStoreInvalidStateMachineTransition(oldState, newState, transition, storeId)
+ }
+
+ def invalidStamp(providedStamp: Long, currentStamp: Long): StateStoreInvalidStamp = {
+ new StateStoreInvalidStamp(providedStamp, currentStamp)
+ }
+
def incorrectNumOrderingColsForRangeScan(numOrderingCols: String):
StateStoreIncorrectNumOrderingColsForRangeScan = {
new StateStoreIncorrectNumOrderingColsForRangeScan(numOrderingCols)
@@ -175,6 +187,11 @@ object StateStoreErrors {
numSchemaFiles, schemaFilesThreshold, addedColFamilies, removedColFamilies)
}
+ def streamingStateCheckpointLocationNotEmpty(checkpointLocation: String)
+ : StateStoreCheckpointLocationNotEmpty = {
+ new StateStoreCheckpointLocationNotEmpty(checkpointLocation)
+ }
+
def stateStoreColumnFamilyMismatch(
columnFamilyName: String,
oldColumnFamilySchema: String,
@@ -204,6 +221,14 @@ object StateStoreErrors {
new StateStoreInvalidConfigAfterRestart(configName, oldConfig, newConfig)
}
+ def stateStoreCommitValidationFailed(
+ batchId: Long,
+ expectedCommits: Int,
+ actualCommits: Int,
+ missingCommits: String): StateStoreCommitValidationFailed = {
+ new StateStoreCommitValidationFailed(batchId, expectedCommits, actualCommits, missingCommits)
+ }
+
def duplicateStateVariableDefined(stateName: String):
StateStoreDuplicateStateVariableDefined = {
new StateStoreDuplicateStateVariableDefined(stateName)
@@ -343,6 +368,30 @@ class StateStoreVariableSizeOrderingColsNotSupported(fieldName: String, index: S
errorClass = "STATE_STORE_VARIABLE_SIZE_ORDERING_COLS_NOT_SUPPORTED",
messageParameters = Map("fieldName" -> fieldName, "index" -> index))
+class StateStoreInvalidStateMachineTransition(
+ oldState: String,
+ newState: String,
+ operation: String,
+ storeId: StateStoreId)
+ extends SparkRuntimeException(
+ errorClass = "STATE_STORE_INVALID_STATE_MACHINE_TRANSITION",
+ messageParameters = Map(
+ "oldState" -> oldState,
+ "newState" -> newState,
+ "operation" -> operation,
+ "storeId" -> storeId.toString
+ )
+ )
+
+class StateStoreInvalidStamp(providedStamp: Long, currentStamp: Long)
+ extends SparkRuntimeException(
+ errorClass = "STATE_STORE_INVALID_STAMP",
+ messageParameters = Map(
+ "providedStamp" -> providedStamp.toString,
+ "currentStamp" -> currentStamp.toString
+ )
+ )
+
class StateStoreNullTypeOrderingColsNotSupported(fieldName: String, index: String)
extends SparkUnsupportedOperationException(
errorClass = "STATE_STORE_NULL_TYPE_ORDERING_COLS_NOT_SUPPORTED",
@@ -430,6 +479,13 @@ class StateStoreStateSchemaFilesThresholdExceeded(
"addedColumnFamilies" -> addedColFamilies.mkString("(", ",", ")"),
"removedColumnFamilies" -> removedColFamilies.mkString("(", ",", ")")))
+class StateStoreCheckpointLocationNotEmpty(
+ checkpointLocation: String)
+ extends SparkUnsupportedOperationException(
+ errorClass = "STATE_STORE_CHECKPOINT_LOCATION_NOT_EMPTY",
+ messageParameters = Map(
+ "checkpointLocation" -> checkpointLocation))
+
class StateStoreSnapshotFileNotFound(fileToRead: String, clazz: String)
extends SparkRuntimeException(
errorClass = "CANNOT_LOAD_STATE_STORE.CANNOT_READ_MISSING_SNAPSHOT_FILE",
@@ -488,3 +544,18 @@ class StateStoreOperationOutOfOrder(errorMsg: String)
errorClass = "STATE_STORE_OPERATION_OUT_OF_ORDER",
messageParameters = Map("errorMsg" -> errorMsg)
)
+
+class StateStoreCommitValidationFailed(
+ batchId: Long,
+ expectedCommits: Int,
+ actualCommits: Int,
+ missingCommits: String)
+ extends SparkRuntimeException(
+ errorClass = "STATE_STORE_COMMIT_VALIDATION_FAILED",
+ messageParameters = Map(
+ "batchId" -> batchId.toString,
+ "expectedCommits" -> expectedCommits.toString,
+ "actualCommits" -> actualCommits.toString,
+ "missingCommits" -> missingCommits
+ )
+ )
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDD.scala
index 4a3e045811686..c95faada419e2 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDD.scala
@@ -27,6 +27,28 @@ import org.apache.spark.sql.internal.SessionState
import org.apache.spark.sql.types.StructType
import org.apache.spark.util.SerializableConfiguration
+/**
+ * Thread local storage for sharing StateStore instances between RDDs.
+ * This allows a ReadStateStore to be reused by a subsequent StateStore operation.
+ */
+object StateStoreThreadLocalTracker {
+ /** Case class to hold both the store and its usage state */
+
+ private val storeInfo: ThreadLocal[ReadStateStore] = new ThreadLocal[ReadStateStore]
+
+ def setStore(store: ReadStateStore): Unit = {
+ storeInfo.set(store)
+ }
+
+ def getStore: Option[ReadStateStore] = {
+ Option(storeInfo.get())
+ }
+
+ def clearStore(): Unit = {
+ storeInfo.remove()
+ }
+}
+
abstract class BaseStateStoreRDD[T: ClassTag, U: ClassTag](
dataRDD: RDD[T],
checkpointLocation: String,
@@ -95,6 +117,7 @@ class ReadStateStoreRDD[T: ClassTag, U: ClassTag](
stateStoreCkptIds.map(_.apply(partition.index).head),
stateSchemaBroadcast,
useColumnFamilies, storeConf, hadoopConfBroadcast.value.value)
+ StateStoreThreadLocalTracker.setStore(store)
storeReadFunction(store, inputIter)
}
}
@@ -130,12 +153,22 @@ class StateStoreRDD[T: ClassTag, U: ClassTag](
val storeProviderId = getStateProviderId(partition)
val inputIter = dataRDD.iterator(partition, ctxt)
- val store = StateStore.get(
- storeProviderId, keySchema, valueSchema, keyStateEncoderSpec, storeVersion,
- uniqueId.map(_.apply(partition.index).head),
- stateSchemaBroadcast,
- useColumnFamilies, storeConf, hadoopConfBroadcast.value.value,
- useMultipleValuesPerKey)
+ val store = StateStoreThreadLocalTracker.getStore match {
+ case Some(readStateStore: ReadStateStore) =>
+ StateStore.getWriteStore(readStateStore, storeProviderId,
+ keySchema, valueSchema, keyStateEncoderSpec, storeVersion,
+ uniqueId.map(_.apply(partition.index).head),
+ stateSchemaBroadcast,
+ useColumnFamilies, storeConf, hadoopConfBroadcast.value.value,
+ useMultipleValuesPerKey)
+ case None =>
+ StateStore.get(
+ storeProviderId, keySchema, valueSchema, keyStateEncoderSpec, storeVersion,
+ uniqueId.map(_.apply(partition.index).head),
+ stateSchemaBroadcast,
+ useColumnFamilies, storeConf, hadoopConfBroadcast.value.value,
+ useMultipleValuesPerKey)
+ }
if (storeConf.unloadOnCommit) {
ctxt.addTaskCompletionListener[Unit](_ => {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/package.scala
index a82eff4812953..d9b80ed99689d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/package.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/package.scala
@@ -23,6 +23,7 @@ import org.apache.spark.TaskContext
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.SQLContext
import org.apache.spark.sql.classic.ClassicConversions.castToImpl
+import org.apache.spark.sql.execution.streaming.operators.stateful.StatefulOperatorStateInfo
import org.apache.spark.sql.internal.SessionState
import org.apache.spark.sql.types.StructType
@@ -109,8 +110,9 @@ package object state {
val cleanedF = dataRDD.sparkContext.clean(storeReadFn)
val wrappedF = (store: ReadStateStore, iter: Iterator[T]) => {
// Clean up the state store.
- TaskContext.get().addTaskCompletionListener[Unit](_ => {
- store.abort()
+ val ctxt = TaskContext.get()
+ ctxt.addTaskCompletionListener[Unit](_ => {
+ StateStoreThreadLocalTracker.clearStore()
})
cleanedF(store, iter)
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala
index f680860231f01..4dbee24f0fecd 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala
@@ -25,7 +25,7 @@ import scala.jdk.CollectionConverters._
import scala.util.control.NonFatal
import org.apache.spark.{JobExecutionStatus, SparkConf}
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.CLASS_NAME
import org.apache.spark.internal.config.Status._
import org.apache.spark.scheduler._
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryHistoryServerPlugin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryHistoryServerPlugin.scala
index 76f64dcb64451..1c2abf5179f28 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryHistoryServerPlugin.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryHistoryServerPlugin.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.ui
import org.apache.spark.SparkConf
import org.apache.spark.scheduler.SparkListener
-import org.apache.spark.sql.execution.streaming.StreamingQueryListenerBus
+import org.apache.spark.sql.execution.streaming.runtime.StreamingQueryListenerBus
import org.apache.spark.sql.streaming.ui.{StreamingQueryStatusListener, StreamingQueryTab}
import org.apache.spark.status.{AppHistoryServerPlugin, ElementTrackingStore}
import org.apache.spark.ui.SparkUI
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowEvaluatorFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowEvaluatorFactory.scala
index 9ff056a279466..d59a0e9f4639b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowEvaluatorFactory.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowEvaluatorFactory.scala
@@ -45,6 +45,7 @@ class WindowEvaluatorFactory(
private val factories = windowFrameExpressionFactoryPairs.map(_._2).toArray
private val inMemoryThreshold = conf.windowExecBufferInMemoryThreshold
private val spillThreshold = conf.windowExecBufferSpillThreshold
+ private val spillSizeThreshold = conf.windowExecBufferSpillSizeThreshold
override def eval(
partitionIndex: Int,
@@ -82,7 +83,8 @@ class WindowEvaluatorFactory(
// Manage the current partition.
val buffer: ExternalAppendOnlyUnsafeRowArray =
- new ExternalAppendOnlyUnsafeRowArray(inMemoryThreshold, spillThreshold)
+ new ExternalAppendOnlyUnsafeRowArray(inMemoryThreshold, spillThreshold,
+ spillSizeThreshold)
var bufferIterator: Iterator[UnsafeRow] = _
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowEvaluatorFactoryBase.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowEvaluatorFactoryBase.scala
index 7d13dbbe2a06a..c2dedda832e2e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowEvaluatorFactoryBase.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowEvaluatorFactoryBase.scala
@@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer
import org.apache.spark.SparkException
import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.{Add, AggregateWindowFunction, Ascending, Attribute, BoundReference, CurrentRow, DateAdd, DateAddYMInterval, DecimalAddNoOverflowCheck, Descending, Expression, ExtractANSIIntervalDays, FrameLessOffsetWindowFunction, FrameType, IdentityProjection, IntegerLiteral, MutableProjection, NamedExpression, OffsetWindowFunction, PythonFuncExpression, RangeFrame, RowFrame, RowOrdering, SortOrder, SpecifiedWindowFrame, TimeAdd, TimestampAddYMInterval, UnaryMinus, UnboundedFollowing, UnboundedPreceding, UnsafeProjection, WindowExpression}
+import org.apache.spark.sql.catalyst.expressions.{Add, AggregateWindowFunction, Ascending, Attribute, BoundReference, CurrentRow, DateAdd, DateAddYMInterval, DecimalAddNoOverflowCheck, Descending, Expression, ExtractANSIIntervalDays, FrameLessOffsetWindowFunction, FrameType, IdentityProjection, IntegerLiteral, MutableProjection, NamedExpression, OffsetWindowFunction, PythonFuncExpression, RangeFrame, RowFrame, RowOrdering, SortOrder, SpecifiedWindowFrame, TimestampAddInterval, TimestampAddYMInterval, UnaryMinus, UnboundedFollowing, UnboundedPreceding, UnsafeProjection, WindowExpression}
import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
import org.apache.spark.sql.execution.metric.SQLMetric
import org.apache.spark.sql.internal.SQLConf
@@ -105,11 +105,11 @@ trait WindowEvaluatorFactoryBase {
case (DateType, DayTimeIntervalType(DAY, DAY)) =>
DateAdd(expr, ExtractANSIIntervalDays(boundOffset))
case (TimestampType | TimestampNTZType, CalendarIntervalType) =>
- TimeAdd(expr, boundOffset, Some(timeZone))
+ TimestampAddInterval(expr, boundOffset, Some(timeZone))
case (TimestampType | TimestampNTZType, _: YearMonthIntervalType) =>
TimestampAddYMInterval(expr, boundOffset, Some(timeZone))
case (TimestampType | TimestampNTZType, _: DayTimeIntervalType) =>
- TimeAdd(expr, boundOffset, Some(timeZone))
+ TimestampAddInterval(expr, boundOffset, Some(timeZone))
case (d: DecimalType, _: DecimalType) => DecimalAddNoOverflowCheck(expr, boundOffset, d)
case (a, b) if a == b => Add(expr, boundOffset)
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala
index 334616a7709e0..ada30cde27cd0 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala
@@ -22,7 +22,7 @@ import org.apache.spark.sql.artifact.ArtifactManager
import org.apache.spark.sql.catalyst.analysis.{Analyzer, EvalSubqueriesForTimeTravel, FunctionRegistry, InvokeProcedures, ReplaceCharWithVarchar, ResolveDataSource, ResolveSessionCatalog, ResolveTranspose, TableFunctionRegistry}
import org.apache.spark.sql.catalyst.analysis.resolver.ResolverExtension
import org.apache.spark.sql.catalyst.catalog.{FunctionExpressionBuilder, SessionCatalog}
-import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.{Expression, ExtractSemiStructuredFields}
import org.apache.spark.sql.catalyst.optimizer.Optimizer
import org.apache.spark.sql.catalyst.parser.ParserInterface
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
@@ -37,7 +37,7 @@ import org.apache.spark.sql.execution.analysis.DetectAmbiguousSelfJoin
import org.apache.spark.sql.execution.command.CommandCheck
import org.apache.spark.sql.execution.datasources._
import org.apache.spark.sql.execution.datasources.v2.{TableCapabilityCheck, V2SessionCatalog}
-import org.apache.spark.sql.execution.streaming.ResolveWriteToStream
+import org.apache.spark.sql.execution.streaming.runtime.ResolveWriteToStream
import org.apache.spark.sql.expressions.UserDefinedAggregateFunction
import org.apache.spark.sql.util.ExecutionListenerManager
@@ -244,6 +244,7 @@ abstract class BaseSessionStateBuilder(
new EvalSubqueriesForTimeTravel +:
new ResolveTranspose(session) +:
new InvokeProcedures(session) +:
+ ExtractSemiStructuredFields +:
customResolutionRules
override val postHocResolutionRules: Seq[Rule[LogicalPlan]] =
@@ -406,7 +407,8 @@ abstract class BaseSessionStateBuilder(
*/
protected def createQueryExecution:
(LogicalPlan, CommandExecutionMode.Value) => QueryExecution =
- (plan, mode) => new QueryExecution(session, plan, mode = mode)
+ (plan, mode) => new QueryExecution(session, plan, mode = mode,
+ shuffleCleanupMode = QueryExecution.determineShuffleCleanupMode(session.sessionState.conf))
/**
* Interface to start and stop streaming queries.
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionStateHelper.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionStateHelper.scala
new file mode 100644
index 0000000000000..6279f8c123765
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionStateHelper.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 org.apache.spark.sql.internal
+
+import org.apache.hadoop.conf.Configuration
+
+import org.apache.spark.{SparkConf, SparkContext}
+import org.apache.spark.sql.SparkSession
+
+/**
+ * Helper trait to access session state related configurations and utilities.
+ * It also provides type annotations for IDEs to build indexes.
+ */
+trait SessionStateHelper {
+ private def sessionState(sparkSession: SparkSession): SessionState = {
+ sparkSession.sessionState
+ }
+
+ private def sparkContext(sparkSession: SparkSession): SparkContext = {
+ sparkSession.sparkContext
+ }
+
+ def getSparkConf(sparkSession: SparkSession): SparkConf = {
+ sparkContext(sparkSession).conf
+ }
+
+ def getSqlConf(sparkSession: SparkSession): SQLConf = {
+ sessionState(sparkSession).conf
+ }
+
+ def getHadoopConf(
+ sparkSession: SparkSession,
+ options: Map[String, String]): Configuration = {
+ sessionState(sparkSession).newHadoopConfWithOptions(options)
+ }
+}
+
+object SessionStateHelper extends SessionStateHelper
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala
index 2b1451493398f..af1f38caab686 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala
@@ -29,12 +29,12 @@ import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FsUrlStreamHandlerFactory, Path}
import org.apache.spark.{SparkConf, SparkContext}
-import org.apache.spark.internal.{Logging, MDC}
+import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{CONFIG, CONFIG2, PATH, VALUE}
import org.apache.spark.sql.catalyst.catalog._
import org.apache.spark.sql.errors.QueryExecutionErrors
import org.apache.spark.sql.execution.CacheManager
-import org.apache.spark.sql.execution.streaming.StreamExecution
+import org.apache.spark.sql.execution.streaming.runtime.StreamExecution
import org.apache.spark.sql.execution.ui.{SQLAppStatusListener, SQLAppStatusStore, SQLTab, StreamingQueryStatusStore}
import org.apache.spark.sql.internal.StaticSQLConf._
import org.apache.spark.sql.streaming.ui.{StreamingQueryStatusListener, StreamingQueryTab}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DatabricksDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DatabricksDialect.scala
index f4fc670470328..9124c1b889098 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DatabricksDialect.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DatabricksDialect.scala
@@ -59,7 +59,12 @@ private case class DatabricksDialect() extends JdbcDialect with NoLegacyJDBCErro
}
override def quoteIdentifier(colName: String): String = {
- s"`$colName`"
+ // Per Databricks documentation:
+ // https://docs.databricks.com/aws/en/sql/language-manual/sql-ref-identifiers
+ //
+ // "Any character from the Unicode character set. Use ` to escape ` itself."
+ val escapedColName = colName.replace("`", "``")
+ s"`$escapedColName`"
}
override def supportsLimit: Boolean = true
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala
index b5ee88aebd7d6..966c5d14bc662 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala
@@ -25,8 +25,6 @@ import java.util.concurrent.ConcurrentHashMap
import scala.jdk.CollectionConverters._
import scala.util.control.NonFatal
-import org.apache.commons.lang3.StringUtils
-
import org.apache.spark.{SparkThrowable, SparkUnsupportedOperationException}
import org.apache.spark.sql.catalyst.analysis.{IndexAlreadyExistsException, NoSuchIndexException, NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException}
import org.apache.spark.sql.connector.catalog.Identifier
@@ -35,6 +33,7 @@ import org.apache.spark.sql.connector.catalog.index.TableIndex
import org.apache.spark.sql.connector.expressions.{Expression, FieldReference, NamedReference}
import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcUtils}
import org.apache.spark.sql.types.{BooleanType, ByteType, DataType, DecimalType, MetadataBuilder, ShortType, StringType, TimestampType}
+import org.apache.spark.util.SparkStringUtils
private[sql] case class H2Dialect() extends JdbcDialect with NoLegacyJDBCError {
override def canHandle(url: String): Boolean =
@@ -181,7 +180,7 @@ private[sql] case class H2Dialect() extends JdbcDialect with NoLegacyJDBCError {
indexMap += (indexName -> newIndex)
} else {
val properties = new util.Properties()
- if (StringUtils.isNotEmpty(indexComment)) properties.put("COMMENT", indexComment)
+ if (SparkStringUtils.isNotEmpty(indexComment)) properties.put("COMMENT", indexComment)
val index = new TableIndex(indexName, indexType, Array(FieldReference(colName)),
new util.HashMap[NamedReference, util.Properties](), properties)
indexMap += (indexName -> index)
@@ -307,4 +306,6 @@ private[sql] case class H2Dialect() extends JdbcDialect with NoLegacyJDBCError {
override def supportsLimit: Boolean = true
override def supportsOffset: Boolean = true
+
+ override def supportsJoin: Boolean = true
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala
index da0df734bbeca..ce4c347cad349 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala
@@ -26,8 +26,6 @@ import java.util.concurrent.TimeUnit
import scala.collection.mutable.ArrayBuilder
import scala.util.control.NonFatal
-import org.apache.commons.lang3.StringUtils
-
import org.apache.spark.{SparkRuntimeException, SparkThrowable, SparkUnsupportedOperationException}
import org.apache.spark.annotation.{DeveloperApi, Since}
import org.apache.spark.internal.Logging
@@ -244,7 +242,9 @@ abstract class JdbcDialect extends Serializable with Logging {
* name is a reserved keyword, or in case it contains characters that require quotes (e.g. space).
*/
def quoteIdentifier(colName: String): String = {
- s""""$colName""""
+ // By ANSI standard, quotes are escaped with another quotes.
+ val escapedColName = colName.replace("\"", "\"\"")
+ s""""$escapedColName""""
}
/**
@@ -352,7 +352,7 @@ abstract class JdbcDialect extends Serializable with Logging {
*/
@Since("2.3.0")
protected[jdbc] def escapeSql(value: String): String =
- if (value == null) null else StringUtils.replace(value, "'", "''")
+ if (value == null) null else value.replace("'", "''")
/**
* Converts value to SQL expression.
@@ -853,6 +853,11 @@ abstract class JdbcDialect extends Serializable with Logging {
def supportsHint: Boolean = false
+ /**
+ * Returns true if dialect supports JOIN operator.
+ */
+ def supportsJoin: Boolean = false
+
/**
* Return the DB-specific quoted and fully qualified table name
*/
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcSQLQueryBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcSQLQueryBuilder.scala
index 95be14f816a76..93af5890711c6 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcSQLQueryBuilder.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcSQLQueryBuilder.scala
@@ -91,6 +91,20 @@ class JdbcSQLQueryBuilder(dialect: JdbcDialect, options: JDBCOptions) {
this
}
+ def withAliasedColumns(
+ columns: Array[String],
+ aliases: Array[Option[String]]): JdbcSQLQueryBuilder = {
+ if (columns.nonEmpty) {
+ assert(columns.length == aliases.length,
+ "Number of columns does not match the number of provided aliases")
+
+ columnList = columns.zip(aliases).map {
+ case (column, alias) => if (alias.isDefined) s"$column AS ${alias.get}" else column
+ }.mkString(",")
+ }
+ this
+ }
+
/**
* Constructs the WHERE clause that following dialect's SQL syntax.
*/
@@ -164,6 +178,38 @@ class JdbcSQLQueryBuilder(dialect: JdbcDialect, options: JDBCOptions) {
this
}
+ /**
+ * Represents JOIN subquery in case Join has been pushed down. This value should be used
+ * instead of options.tableOrQuery if join has been pushed down.
+ */
+ private var joinQuery: Option[String] = None
+
+ def withJoin(
+ left: JdbcSQLQueryBuilder,
+ right: JdbcSQLQueryBuilder,
+ leftSideQualifier: String,
+ rightSideQualifier: String,
+ columns: Array[String],
+ joinType: String,
+ joinCondition: String): JdbcSQLQueryBuilder = {
+ columnList = columns.mkString(",")
+ joinQuery = Some(
+ s"""(
+ |SELECT ${columns.mkString(",")} FROM
+ |(${left.build()}) $leftSideQualifier
+ |$joinType
+ |(${right.build()}) $rightSideQualifier
+ |ON $joinCondition
+ |) ${JoinPushdownAliasGenerator.getSubqueryQualifier}""".stripMargin
+ )
+
+ this
+ }
+
+ // If join has been pushed down, reuse join query as a subquery. Otherwise, fallback to
+ // what is provided in options.
+ protected final def tableOrQuery: String = joinQuery.getOrElse(options.tableOrQuery)
+
/**
* Build the final SQL query that following dialect's SQL syntax.
*/
@@ -174,7 +220,15 @@ class JdbcSQLQueryBuilder(dialect: JdbcDialect, options: JDBCOptions) {
val offsetClause = dialect.getOffsetClause(offset)
options.prepareQuery +
- s"SELECT $hintClause$columnList FROM ${options.tableOrQuery} $tableSampleClause" +
+ s"SELECT $hintClause$columnList FROM $tableOrQuery $tableSampleClause" +
s" $whereClause $groupByClause $orderByClause $limitClause $offsetClause"
}
}
+
+object JoinPushdownAliasGenerator {
+ private val subQueryId = new java.util.concurrent.atomic.AtomicLong()
+
+ def getSubqueryQualifier: String = {
+ "join_subquery_" + subQueryId.getAndIncrement()
+ }
+}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala
index 3f06ea1a2fbfc..2954795836877 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala
@@ -265,7 +265,7 @@ private case class MsSqlServerDialect() extends JdbcDialect with NoLegacyJDBCErr
val limitClause = dialect.getLimitClause(limit)
options.prepareQuery +
- s"SELECT $limitClause $columnList FROM ${options.tableOrQuery}" +
+ s"SELECT $limitClause $columnList FROM $tableOrQuery" +
s" $whereClause $groupByClause $orderByClause"
}
}
@@ -286,6 +286,8 @@ private case class MsSqlServerDialect() extends JdbcDialect with NoLegacyJDBCErr
}
override def supportsLimit: Boolean = true
+
+ override def supportsJoin: Boolean = true
}
private object MsSqlServerDialect {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala
index 19377057844e5..41452ca45057c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala
@@ -196,7 +196,13 @@ private case class MySQLDialect() extends JdbcDialect with SQLConfHelper with No
}
override def quoteIdentifier(colName: String): String = {
- s"`$colName`"
+ // Per MySQL documentation: https://dev.mysql.com/doc/refman/8.4/en/identifiers.html
+ //
+ // Identifier quote characters can be included within an identifier if you quote the
+ // identifier. If the character to be included within the identifier is the same as
+ // that used to quote the identifier itself, then you need to double the character.
+ val escapedColName = colName.replace("`", "``")
+ s"`$escapedColName`"
}
override def schemasExists(conn: Connection, options: JDBCOptions, schema: String): Boolean = {
@@ -424,7 +430,7 @@ private case class MySQLDialect() extends JdbcDialect with SQLConfHelper with No
}
options.prepareQuery +
- s"SELECT $hintClause$columnList FROM ${options.tableOrQuery} $tableSampleClause" +
+ s"SELECT $hintClause$columnList FROM $tableOrQuery $tableSampleClause" +
s" $whereClause $groupByClause $orderByClause $limitOrOffsetStmt"
}
}
@@ -437,4 +443,6 @@ private case class MySQLDialect() extends JdbcDialect with SQLConfHelper with No
override def supportsOffset: Boolean = true
override def supportsHint: Boolean = true
+
+ override def supportsJoin: Boolean = true
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala
index a9f6a727a7241..81031b1ec13d4 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala
@@ -24,7 +24,7 @@ import scala.util.control.NonFatal
import org.apache.spark.{SparkThrowable, SparkUnsupportedOperationException}
import org.apache.spark.sql.catalyst.SQLConfHelper
-import org.apache.spark.sql.connector.expressions.{Expression, Literal}
+import org.apache.spark.sql.connector.expressions.{Expression, Extract, Literal}
import org.apache.spark.sql.errors.QueryCompilationErrors
import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions
import org.apache.spark.sql.jdbc.OracleDialect._
@@ -44,7 +44,7 @@ private case class OracleDialect() extends JdbcDialect with SQLConfHelper with N
// scalastyle:on line.size.limit
private val supportedAggregateFunctions =
Set("MAX", "MIN", "SUM", "COUNT", "AVG") ++ distinctUnsupportedAggregateFunctions
- private val supportedFunctions = supportedAggregateFunctions
+ private val supportedFunctions = supportedAggregateFunctions ++ Set("TRUNC")
override def isSupportedFunction(funcName: String): Boolean =
supportedFunctions.contains(funcName)
@@ -56,6 +56,30 @@ private case class OracleDialect() extends JdbcDialect with SQLConfHelper with N
class OracleSQLBuilder extends JDBCSQLBuilder {
+ override def visitExtract(extract: Extract): String = {
+ val field = extract.field
+ field match {
+ // YEAR, MONTH, DAY, HOUR, MINUTE are identical on Oracle and Spark for
+ // both datetime and interval types.
+ case "YEAR" | "MONTH" | "DAY" | "HOUR" | "MINUTE" =>
+ super.visitExtract(field, build(extract.source()))
+ // Oracle does not support the following date fields: DAY_OF_YEAR, WEEK, QUARTER,
+ // DAY_OF_WEEK, or YEAR_OF_WEEK.
+ // We can't push down SECOND due to the difference in result types between Spark and
+ // Oracle. Spark returns decimal(8, 6), but Oracle returns integer.
+ case _ =>
+ visitUnexpectedExpr(extract)
+ }
+ }
+
+ override def visitSQLFunction(funcName: String, inputs: Array[String]): String = {
+ funcName match {
+ case "TRUNC" =>
+ s"TRUNC(${inputs(0)}, 'IW')"
+ case _ => super.visitSQLFunction(funcName, inputs)
+ }
+ }
+
override def visitAggregateFunction(
funcName: String, isDistinct: Boolean, inputs: Array[String]): String =
if (isDistinct && distinctUnsupportedAggregateFunctions.contains(funcName)) {
@@ -233,7 +257,7 @@ private case class OracleDialect() extends JdbcDialect with SQLConfHelper with N
extends JdbcSQLQueryBuilder(dialect, options) {
override def build(): String = {
- val selectStmt = s"SELECT $hintClause$columnList FROM ${options.tableOrQuery}" +
+ val selectStmt = s"SELECT $hintClause$columnList FROM $tableOrQuery" +
s" $tableSampleClause $whereClause $groupByClause $orderByClause"
val finalSelectStmt = if (limit > 0) {
if (offset > 0) {
@@ -268,6 +292,8 @@ private case class OracleDialect() extends JdbcDialect with SQLConfHelper with N
override def supportsHint: Boolean = true
+ override def supportsJoin: Boolean = true
+
override def classifyException(
e: Throwable,
condition: String,
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala
index b5a0dd95e60c4..d7d730ed0cc3a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala
@@ -27,7 +27,6 @@ import scala.util.control.NonFatal
import org.apache.spark.SparkThrowable
import org.apache.spark.internal.LogKeys.COLUMN_NAME
-import org.apache.spark.internal.MDC
import org.apache.spark.sql.catalyst.SQLConfHelper
import org.apache.spark.sql.catalyst.analysis.{IndexAlreadyExistsException, NonEmptyNamespaceException, NoSuchIndexException}
import org.apache.spark.sql.connector.catalog.Identifier
@@ -363,6 +362,8 @@ private case class PostgresDialect()
override def supportsTableSample: Boolean = true
+ override def supportsJoin: Boolean = true
+
override def getTableSample(sample: TableSampleInfo): String = {
// hard-coded to BERNOULLI for now because Spark doesn't have a way to specify sample
// method name
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala
index fa8aaf6d81c2b..efc44f84cd2c2 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala
@@ -1174,7 +1174,7 @@ class ForStatementExec(
val defaultExpression = DefaultValueExpression(
Literal(null, queryColumnNameToDataType(varName)), "null")
val declareVariable = CreateVariable(
- UnresolvedIdentifier(Seq(varName)),
+ Seq(UnresolvedIdentifier(Seq(varName))),
defaultExpression,
replace = false
)
@@ -1187,7 +1187,7 @@ class ForStatementExec(
OneRowRelation()
)
val setIdentifierToCurrentRow =
- SetVariable(Seq(UnresolvedAttribute(varName)), projectNamedStruct)
+ SetVariable(Seq(UnresolvedAttribute.quoted(varName)), projectNamedStruct)
new SingleStatementExec(
setIdentifierToCurrentRow,
Origin(),
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreter.scala b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreter.scala
index 9ab45c4003f8c..e0e11183d3213 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreter.scala
@@ -122,12 +122,21 @@ case class SqlScriptingInterpreter(session: SparkSession) {
// Get NOT FOUND handler.
notFoundHandler = if (handler.exceptionHandlerTriggers.notFound) {
- Some(handlerExec)
+ if (notFoundHandler.isDefined) {
+ throw SqlScriptingErrors.duplicateHandlerForSameCondition(CurrentOrigin.get, "NOT FOUND")
+ } else {
+ Some(handlerExec)
+ }
} else None
// Get SQLEXCEPTION handler.
sqlExceptionHandler = if (handler.exceptionHandlerTriggers.sqlException) {
- Some(handlerExec)
+ if (sqlExceptionHandler.isDefined) {
+ throw SqlScriptingErrors
+ .duplicateHandlerForSameCondition(CurrentOrigin.get, "SQLEXCEPTION")
+ } else {
+ Some(handlerExec)
+ }
} else None
})
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingLocalVariableManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingLocalVariableManager.scala
index f875f2154a925..fb3560ddb9f38 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingLocalVariableManager.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingLocalVariableManager.scala
@@ -28,6 +28,9 @@ import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableErro
class SqlScriptingLocalVariableManager(context: SqlScriptingExecutionContext)
extends VariableManager with DataTypeErrorsBase {
+ override def getVariableNameForError(variableName: String): String =
+ toSQLId(Seq(context.currentScope.label, variableName))
+
override def create(
nameParts: Seq[String],
varDef: VariableDefinition,
@@ -42,7 +45,7 @@ class SqlScriptingLocalVariableManager(context: SqlScriptingExecutionContext)
throw new AnalysisException(
errorClass = "VARIABLE_ALREADY_EXISTS",
messageParameters = Map(
- "variableName" -> toSQLId(Seq(context.currentScope.label, name))))
+ "variableName" -> getVariableNameForError(name)))
}
context.currentScope.variables.put(name, varDef)
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/TestGroupState.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/TestGroupState.scala
index b92e361ef805b..17fb402457b74 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/TestGroupState.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/TestGroupState.scala
@@ -19,8 +19,8 @@ package org.apache.spark.sql.streaming
import org.apache.spark.annotation.Evolving
import org.apache.spark.api.java.Optional
-import org.apache.spark.sql.execution.streaming.GroupStateImpl
-import org.apache.spark.sql.execution.streaming.GroupStateImpl._
+import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.GroupStateImpl
+import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.GroupStateImpl._
/**
* :: Experimental ::
diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaBeanDeserializationSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaBeanDeserializationSuite.java
index a83041dc522c6..79a7d2b71cc43 100644
--- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaBeanDeserializationSuite.java
+++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaBeanDeserializationSuite.java
@@ -24,8 +24,6 @@
import java.time.LocalDate;
import java.util.*;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
import org.junit.jupiter.api.*;
import org.apache.spark.api.java.function.MapFunction;
@@ -493,17 +491,10 @@ public int hashCode() {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("shortField", shortField)
- .append("intField", intField)
- .append("longField", longField)
- .append("floatField", floatField)
- .append("doubleField", doubleField)
- .append("stringField", stringField)
- .append("booleanField", booleanField)
- .append("timestampField", timestampField)
- .append("nullIntField", nullIntField)
- .toString();
+ return "RecordSpark22000[shortField=" + shortField + ",intField=" + intField +
+ ",longField=" + longField + ",floatField=" + floatField + ",doubleField=" + doubleField +
+ ",stringField=" + stringField + ",booleanField=" + booleanField +
+ ",timestampField=" + timestampField + ",nullIntField=" + nullIntField + "]";
}
}
@@ -680,10 +671,8 @@ public int hashCode() {
@Override
public String toString() {
- return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
- .append("localDateField", localDateField)
- .append("instantField", instantField)
- .toString();
+ return "LocalDateInstantRecord[localDateField=" + localDateField +
+ ",instantField=" + instantField + "]";
}
}
diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaColumnExpressionSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaColumnExpressionSuite.java
index 9988d04220f0f..1a94867668aec 100644
--- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaColumnExpressionSuite.java
+++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaColumnExpressionSuite.java
@@ -19,8 +19,6 @@
import java.util.*;
-import com.google.common.collect.Maps;
-
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
@@ -90,6 +88,6 @@ public void isInCollectionCheckExceptionMessage() {
messageParameters.put("functionName", "`in`");
messageParameters.put("dataType", "[\"INT\", \"ARRAY\"]");
messageParameters.put("sqlExpr", "\"(a IN (b))\"");
- Assertions.assertTrue(Maps.difference(e.getMessageParameters(), messageParameters).areEqual());
+ Assertions.assertTrue(e.getMessageParameters().equals(messageParameters));
}
}
diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java
index 27137e53934d2..a780c057e9038 100644
--- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java
+++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java
@@ -27,7 +27,6 @@
import scala.collection.Seq;
import scala.jdk.javaapi.CollectionConverters;
-import com.google.common.collect.ImmutableMap;
import com.google.common.primitives.Ints;
import org.junit.jupiter.api.*;
@@ -133,7 +132,7 @@ public void testShow() {
public static class Bean implements Serializable {
private double a = 0.0;
private Integer[] b = { 0, 1 };
- private Map c = ImmutableMap.of("hello", new int[] { 1, 2 });
+ private Map c = Map.of("hello", new int[] { 1, 2 });
private List d = Arrays.asList("floppy", "disk");
private BigInteger e = new BigInteger("1234567");
private NestedBean f = new NestedBean();
@@ -312,7 +311,7 @@ public void testCovariance() {
@Test
public void testSampleBy() {
Dataset df = spark.range(0, 100, 1, 2).select(col("id").mod(3).as("key"));
- Dataset sampled = df.stat().sampleBy("key", ImmutableMap.of(0, 0.1, 1, 0.2), 0L);
+ Dataset sampled = df.stat().sampleBy("key", Map.of(0, 0.1, 1, 0.2), 0L);
List actual = sampled.groupBy("key").count().orderBy("key").collectAsList();
Assertions.assertEquals(0, actual.get(0).getLong(0));
Assertions.assertTrue(0 <= actual.get(0).getLong(1) && actual.get(0).getLong(1) <= 8);
@@ -338,7 +337,7 @@ public void testwithColumns() {
@Test
public void testSampleByColumn() {
Dataset df = spark.range(0, 100, 1, 2).select(col("id").mod(3).as("key"));
- Dataset sampled = df.stat().sampleBy(col("key"), ImmutableMap.of(0, 0.1, 1, 0.2), 0L);
+ Dataset sampled = df.stat().sampleBy(col("key"), Map.of(0, 0.1, 1, 0.2), 0L);
List actual = sampled.groupBy("key").count().orderBy("key").collectAsList();
Assertions.assertEquals(0, actual.get(0).getLong(0));
Assertions.assertTrue(0 <= actual.get(0).getLong(1) && actual.get(0).getLong(1) <= 8);
diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java
index 692b5c0ebc3a9..33afafbf25601 100644
--- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java
+++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java
@@ -33,7 +33,6 @@
import scala.Tuple4;
import scala.Tuple5;
-import com.google.common.base.Objects;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
@@ -812,6 +811,26 @@ public void testLocalTimeEncoder() {
Assertions.assertEquals(data, ds.collectAsList());
}
+ @Test
+ public void testLocalTimeFilter() {
+ Encoder encoder = Encoders.LOCALTIME();
+ List data = Arrays.asList(
+ LocalTime.of(9, 30, 45),
+ LocalTime.of(14, 10, 10),
+ LocalTime.of(22, 10, 10)
+ );
+ Dataset ds = spark.createDataset(data, encoder);
+
+ Dataset filtered = ds.filter(
+ (FilterFunction) time -> time.isAfter(LocalTime.of(12, 0, 0))
+ );
+ List expectedFiltered = Arrays.asList(
+ LocalTime.of(14, 10, 10),
+ LocalTime.of(22, 10, 10)
+ );
+ Assertions.assertEquals(expectedFiltered, filtered.collectAsList());
+ }
+
public static class KryoSerializable {
String value;
@@ -1192,12 +1211,12 @@ public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
SmallBean smallBean = (SmallBean) o;
- return b == smallBean.b && com.google.common.base.Objects.equal(a, smallBean.a);
+ return b == smallBean.b && Objects.equals(a, smallBean.a);
}
@Override
public int hashCode() {
- return Objects.hashCode(a, b);
+ return Objects.hash(a, b);
}
}
@@ -1217,7 +1236,7 @@ public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
NestedSmallBean that = (NestedSmallBean) o;
- return Objects.equal(f, that.f);
+ return Objects.equals(f, that.f);
}
@Override
@@ -1259,13 +1278,13 @@ public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
NestedSmallBeanWithNonNullField that = (NestedSmallBeanWithNonNullField) o;
- return Objects.equal(nullable_f, that.nullable_f) &&
- Objects.equal(nonNull_f, that.nonNull_f) && Objects.equal(childMap, that.childMap);
+ return Objects.equals(nullable_f, that.nullable_f) &&
+ Objects.equals(nonNull_f, that.nonNull_f) && Objects.equals(childMap, that.childMap);
}
@Override
public int hashCode() {
- return Objects.hashCode(nullable_f, nonNull_f, childMap);
+ return Objects.hash(nullable_f, nonNull_f, childMap);
}
}
@@ -1286,7 +1305,7 @@ public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
NestedSmallBean2 that = (NestedSmallBean2) o;
- return Objects.equal(f, that.f);
+ return Objects.equals(f, that.f);
}
@Override
@@ -1828,7 +1847,7 @@ public String toString() {
}
public int hashCode() {
- return Objects.hashCode(enumField, regularField);
+ return Objects.hash(enumField, regularField);
}
public boolean equals(Object other) {
@@ -2085,7 +2104,7 @@ public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
BeanWithSet that = (BeanWithSet) o;
- return Objects.equal(fields, that.fields);
+ return Objects.equals(fields, that.fields);
}
@Override
@@ -2128,14 +2147,14 @@ public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
SpecificListsBean that = (SpecificListsBean) o;
- return Objects.equal(arrayList, that.arrayList) &&
- Objects.equal(linkedList, that.linkedList) &&
- Objects.equal(list, that.list);
+ return Objects.equals(arrayList, that.arrayList) &&
+ Objects.equals(linkedList, that.linkedList) &&
+ Objects.equals(list, that.list);
}
@Override
public int hashCode() {
- return Objects.hashCode(arrayList, linkedList, list);
+ return Objects.hash(arrayList, linkedList, list);
}
}
}
diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java
index c1f48a922b727..7328249955147 100644
--- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java
+++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java
@@ -19,6 +19,7 @@
import java.io.Serializable;
import java.time.LocalDate;
+import java.time.LocalTime;
import java.util.List;
import org.apache.spark.sql.catalyst.FunctionIdentifier;
@@ -34,6 +35,7 @@
import org.apache.spark.sql.classic.SparkSession;
import org.apache.spark.sql.api.java.UDF2;
import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.TimeType;
// The test suite itself is Serializable so that anonymous Function implementations can be
// serialized, as an alternative to converting these anonymous classes to static inner classes;
@@ -136,6 +138,15 @@ public void udf7Test() {
}
}
+ @Test
+ public void udf8Test() {
+ spark.udf().register(
+ "plusTwoHours",
+ (java.time.LocalTime lt) -> lt.plusHours(2), new TimeType(6));
+ Row result = spark.sql("SELECT plusTwoHours(TIME '09:10:10')").head();
+ Assertions.assertEquals(LocalTime.of(11, 10, 10), result.get(0));
+ }
+
@Test
public void sourceTest() {
spark.udf().register("stringLengthTest", (String str) -> str.length(), DataTypes.IntegerType);
diff --git a/sql/core/src/test/resources/log4j2.properties b/sql/core/src/test/resources/log4j2.properties
index 9b0e2af355fd8..0951114470faa 100644
--- a/sql/core/src/test/resources/log4j2.properties
+++ b/sql/core/src/test/resources/log4j2.properties
@@ -42,7 +42,6 @@ appender.file.fileName = target/unit-tests.log
appender.file.layout.type = PatternLayout
appender.file.layout.pattern = %d{HH:mm:ss.SSS} %t %p %c{1}: %m%n%ex
-# Set the logger level of File Appender to WARN
appender.file.filter.threshold.type = ThresholdFilter
appender.file.filter.threshold.level = info
diff --git a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md
index da7b8f3f7709d..6e6d520efbac3 100644
--- a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md
+++ b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md
@@ -10,6 +10,7 @@
| org.apache.spark.sql.catalyst.expressions.AesDecrypt | aes_decrypt | SELECT aes_decrypt(unhex('83F16B2AA704794132802D248E6BFD4E380078182D1544813898AC97E709B28A94'), '0000111122223333') | struct |
| org.apache.spark.sql.catalyst.expressions.AesEncrypt | aes_encrypt | SELECT hex(aes_encrypt('Spark', '0000111122223333')) | struct |
| org.apache.spark.sql.catalyst.expressions.And | and | SELECT true and true | struct<(true AND true):boolean> |
+| org.apache.spark.sql.catalyst.expressions.ApproxTopKEstimate | approx_top_k_estimate | SELECT approx_top_k_estimate(approx_top_k_accumulate(expr)) FROM VALUES (0), (0), (1), (1), (2), (3), (4), (4) AS tab(expr) | struct>> |
| org.apache.spark.sql.catalyst.expressions.ArrayAggregate | aggregate | SELECT aggregate(array(1, 2, 3), 0, (acc, x) -> acc + x) | struct |
| org.apache.spark.sql.catalyst.expressions.ArrayAggregate | reduce | SELECT reduce(array(1, 2, 3), 0, (acc, x) -> acc + x) | struct |
| org.apache.spark.sql.catalyst.expressions.ArrayAppend | array_append | SELECT array_append(array('b', 'd', 'c', 'a'), 'd') | struct> |
@@ -73,6 +74,7 @@
| org.apache.spark.sql.catalyst.expressions.Cast | int | N/A | N/A |
| org.apache.spark.sql.catalyst.expressions.Cast | smallint | N/A | N/A |
| org.apache.spark.sql.catalyst.expressions.Cast | string | N/A | N/A |
+| org.apache.spark.sql.catalyst.expressions.Cast | time | N/A | N/A |
| org.apache.spark.sql.catalyst.expressions.Cast | timestamp | N/A | N/A |
| org.apache.spark.sql.catalyst.expressions.Cast | tinyint | N/A | N/A |
| org.apache.spark.sql.catalyst.expressions.Cbrt | cbrt | SELECT cbrt(27.0) | struct |
@@ -207,7 +209,7 @@
| org.apache.spark.sql.catalyst.expressions.MakeDate | make_date | SELECT make_date(2013, 7, 15) | struct |
| org.apache.spark.sql.catalyst.expressions.MakeInterval | make_interval | SELECT make_interval(100, 11, 1, 1, 12, 30, 01.001001) | struct |
| org.apache.spark.sql.catalyst.expressions.MakeTime | make_time | SELECT make_time(6, 30, 45.887) | struct |
-| org.apache.spark.sql.catalyst.expressions.MakeTimestamp | make_timestamp | SELECT make_timestamp(2014, 12, 28, 6, 30, 45.887) | struct |
+| org.apache.spark.sql.catalyst.expressions.MakeTimestampExpressionBuilder | make_timestamp | SELECT make_timestamp(2014, 12, 28, 6, 30, 45.887) | struct |
| org.apache.spark.sql.catalyst.expressions.MakeTimestampLTZExpressionBuilder | make_timestamp_ltz | SELECT make_timestamp_ltz(2014, 12, 28, 6, 30, 45.887) | struct |
| org.apache.spark.sql.catalyst.expressions.MakeTimestampNTZExpressionBuilder | make_timestamp_ntz | SELECT make_timestamp_ntz(2014, 12, 28, 6, 30, 45.887) | struct |
| org.apache.spark.sql.catalyst.expressions.MakeValidUTF8 | make_valid_utf8 | SELECT make_valid_utf8('Spark') | struct |
@@ -340,6 +342,8 @@
| org.apache.spark.sql.catalyst.expressions.Subtract | - | SELECT 2 - 1 | struct<(2 - 1):int> |
| org.apache.spark.sql.catalyst.expressions.Tan | tan | SELECT tan(0) | struct |
| org.apache.spark.sql.catalyst.expressions.Tanh | tanh | SELECT tanh(0) | struct |
+| org.apache.spark.sql.catalyst.expressions.TimeDiff | time_diff | SELECT time_diff('HOUR', TIME'20:30:29', TIME'21:30:28') | struct |
+| org.apache.spark.sql.catalyst.expressions.TimeTrunc | time_trunc | SELECT time_trunc('HOUR', TIME'09:32:05.359') | struct |
| org.apache.spark.sql.catalyst.expressions.TimeWindow | window | SELECT a, window.start, window.end, count(*) as cnt FROM VALUES ('A1', '2021-01-01 00:00:00'), ('A1', '2021-01-01 00:04:30'), ('A1', '2021-01-01 00:06:00'), ('A2', '2021-01-01 00:01:00') AS tab(a, b) GROUP by a, window(b, '5 minutes') ORDER BY a, start | struct |
| org.apache.spark.sql.catalyst.expressions.ToBinary | to_binary | SELECT to_binary('abc', 'utf-8') | struct |
| org.apache.spark.sql.catalyst.expressions.ToCharacterBuilder | to_char | SELECT to_char(454, '999') | struct |
@@ -368,6 +372,7 @@
| org.apache.spark.sql.catalyst.expressions.TryReflect | try_reflect | SELECT try_reflect('java.util.UUID', 'randomUUID') | struct |
| org.apache.spark.sql.catalyst.expressions.TrySubtract | try_subtract | SELECT try_subtract(2, 1) | struct |
| org.apache.spark.sql.catalyst.expressions.TryToBinary | try_to_binary | SELECT try_to_binary('abc', 'utf-8') | struct |
+| org.apache.spark.sql.catalyst.expressions.TryToDateExpressionBuilder | try_to_date | SELECT try_to_date('2016-12-31') | struct |
| org.apache.spark.sql.catalyst.expressions.TryToNumber | try_to_number | SELECT try_to_number('454', '999') | struct |
| org.apache.spark.sql.catalyst.expressions.TryToTimeExpressionBuilder | try_to_time | SELECT try_to_time('00:12:00.001') | struct |
| org.apache.spark.sql.catalyst.expressions.TryToTimestampExpressionBuilder | try_to_timestamp | SELECT try_to_timestamp('2016-12-31 00:12:00') | struct |
@@ -400,6 +405,8 @@
| org.apache.spark.sql.catalyst.expressions.ZeroIfNull | zeroifnull | SELECT zeroifnull(NULL) | struct |
| org.apache.spark.sql.catalyst.expressions.ZipWith | zip_with | SELECT zip_with(array(1, 2, 3), array('a', 'b', 'c'), (x, y) -> (y, x)) | struct>> |
| org.apache.spark.sql.catalyst.expressions.aggregate.AnyValue | any_value | SELECT any_value(col) FROM VALUES (10), (5), (20) AS tab(col) | struct |
+| org.apache.spark.sql.catalyst.expressions.aggregate.ApproxTopK | approx_top_k | SELECT approx_top_k(expr) FROM VALUES (0), (0), (1), (1), (2), (3), (4), (4) AS tab(expr) | struct>> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.ApproxTopKAccumulate | approx_top_k_accumulate | SELECT approx_top_k_estimate(approx_top_k_accumulate(expr)) FROM VALUES (0), (0), (1), (1), (2), (3), (4), (4) AS tab(expr) | struct>> |
| org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile | approx_percentile | SELECT approx_percentile(col, array(0.5, 0.4, 0.1), 100) FROM VALUES (0), (1), (2), (10) AS tab(col) | struct> |
| org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile | percentile_approx | SELECT percentile_approx(col, array(0.5, 0.4, 0.1), 100) FROM VALUES (0), (1), (2), (10) AS tab(col) | struct> |
| org.apache.spark.sql.catalyst.expressions.aggregate.Average | avg | SELECT avg(col) FROM VALUES (1), (2), (3) AS tab(col) | struct |
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/binary.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/binary.sql.out
index fe61e684a7ff5..b2781c56ca7da 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/binary.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/binary.sql.out
@@ -32,3 +32,28 @@ SELECT to_csv(named_struct('n', 1, 'info', X'4561736F6E2059616F20323031382D31312
-- !query analysis
Project [to_csv(named_struct(n, 1, info, 0x4561736F6E2059616F20323031382D31312D31373A31333A33333A3333), Some(America/Los_Angeles)) AS to_csv(named_struct(n, 1, info, X'4561736F6E2059616F20323031382D31312D31373A31333A33333A3333'))#x]
+- OneRowRelation
+
+
+-- !query
+select to_xml(named_struct('name', binary('Eason'), 'birth', 2018, 'org', binary('Kindergarten Cop')))
+-- !query analysis
+Project [to_xml(named_struct(name, cast(Eason as binary), birth, 2018, org, cast(Kindergarten Cop as binary)), Some(America/Los_Angeles)) AS to_xml(named_struct(name, Eason, birth, 2018, org, Kindergarten Cop))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT from_xml(
+ to_xml(named_struct('name', binary('Eason'), 'birth', 2018, 'org', binary('Kindergarten Cop'))),
+ 'name STRING, birth INT, org STRING')
+-- !query analysis
+Project [from_xml(StructField(name,StringType,true), StructField(birth,IntegerType,true), StructField(org,StringType,true), to_xml(named_struct(name, cast(Eason as binary), birth, 2018, org, cast(Kindergarten Cop as binary)), Some(America/Los_Angeles)), Some(America/Los_Angeles)) AS from_xml(to_xml(named_struct(name, Eason, birth, 2018, org, Kindergarten Cop)))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT from_xml(
+ to_xml(named_struct('name', binary('Eason'), 'birth', 2018, 'org', binary('Kindergarten Cop'))),
+ 'name binary, birth INT, org binary')
+-- !query analysis
+Project [from_xml(StructField(name,BinaryType,true), StructField(birth,IntegerType,true), StructField(org,BinaryType,true), to_xml(named_struct(name, cast(Eason as binary), birth, 2018, org, cast(Kindergarten Cop as binary)), Some(America/Los_Angeles)), Some(America/Los_Angeles)) AS from_xml(to_xml(named_struct(name, Eason, birth, 2018, org, Kindergarten Cop)))#x]
++- OneRowRelation
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/binary_base64.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/binary_base64.sql.out
index fe61e684a7ff5..b2781c56ca7da 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/binary_base64.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/binary_base64.sql.out
@@ -32,3 +32,28 @@ SELECT to_csv(named_struct('n', 1, 'info', X'4561736F6E2059616F20323031382D31312
-- !query analysis
Project [to_csv(named_struct(n, 1, info, 0x4561736F6E2059616F20323031382D31312D31373A31333A33333A3333), Some(America/Los_Angeles)) AS to_csv(named_struct(n, 1, info, X'4561736F6E2059616F20323031382D31312D31373A31333A33333A3333'))#x]
+- OneRowRelation
+
+
+-- !query
+select to_xml(named_struct('name', binary('Eason'), 'birth', 2018, 'org', binary('Kindergarten Cop')))
+-- !query analysis
+Project [to_xml(named_struct(name, cast(Eason as binary), birth, 2018, org, cast(Kindergarten Cop as binary)), Some(America/Los_Angeles)) AS to_xml(named_struct(name, Eason, birth, 2018, org, Kindergarten Cop))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT from_xml(
+ to_xml(named_struct('name', binary('Eason'), 'birth', 2018, 'org', binary('Kindergarten Cop'))),
+ 'name STRING, birth INT, org STRING')
+-- !query analysis
+Project [from_xml(StructField(name,StringType,true), StructField(birth,IntegerType,true), StructField(org,StringType,true), to_xml(named_struct(name, cast(Eason as binary), birth, 2018, org, cast(Kindergarten Cop as binary)), Some(America/Los_Angeles)), Some(America/Los_Angeles)) AS from_xml(to_xml(named_struct(name, Eason, birth, 2018, org, Kindergarten Cop)))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT from_xml(
+ to_xml(named_struct('name', binary('Eason'), 'birth', 2018, 'org', binary('Kindergarten Cop'))),
+ 'name binary, birth INT, org binary')
+-- !query analysis
+Project [from_xml(StructField(name,BinaryType,true), StructField(birth,IntegerType,true), StructField(org,BinaryType,true), to_xml(named_struct(name, cast(Eason as binary), birth, 2018, org, cast(Kindergarten Cop as binary)), Some(America/Los_Angeles)), Some(America/Los_Angeles)) AS from_xml(to_xml(named_struct(name, Eason, birth, 2018, org, Kindergarten Cop)))#x]
++- OneRowRelation
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/binary_basic.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/binary_basic.sql.out
index fe61e684a7ff5..b2781c56ca7da 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/binary_basic.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/binary_basic.sql.out
@@ -32,3 +32,28 @@ SELECT to_csv(named_struct('n', 1, 'info', X'4561736F6E2059616F20323031382D31312
-- !query analysis
Project [to_csv(named_struct(n, 1, info, 0x4561736F6E2059616F20323031382D31312D31373A31333A33333A3333), Some(America/Los_Angeles)) AS to_csv(named_struct(n, 1, info, X'4561736F6E2059616F20323031382D31312D31373A31333A33333A3333'))#x]
+- OneRowRelation
+
+
+-- !query
+select to_xml(named_struct('name', binary('Eason'), 'birth', 2018, 'org', binary('Kindergarten Cop')))
+-- !query analysis
+Project [to_xml(named_struct(name, cast(Eason as binary), birth, 2018, org, cast(Kindergarten Cop as binary)), Some(America/Los_Angeles)) AS to_xml(named_struct(name, Eason, birth, 2018, org, Kindergarten Cop))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT from_xml(
+ to_xml(named_struct('name', binary('Eason'), 'birth', 2018, 'org', binary('Kindergarten Cop'))),
+ 'name STRING, birth INT, org STRING')
+-- !query analysis
+Project [from_xml(StructField(name,StringType,true), StructField(birth,IntegerType,true), StructField(org,StringType,true), to_xml(named_struct(name, cast(Eason as binary), birth, 2018, org, cast(Kindergarten Cop as binary)), Some(America/Los_Angeles)), Some(America/Los_Angeles)) AS from_xml(to_xml(named_struct(name, Eason, birth, 2018, org, Kindergarten Cop)))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT from_xml(
+ to_xml(named_struct('name', binary('Eason'), 'birth', 2018, 'org', binary('Kindergarten Cop'))),
+ 'name binary, birth INT, org binary')
+-- !query analysis
+Project [from_xml(StructField(name,BinaryType,true), StructField(birth,IntegerType,true), StructField(org,BinaryType,true), to_xml(named_struct(name, cast(Eason as binary), birth, 2018, org, cast(Kindergarten Cop as binary)), Some(America/Los_Angeles)), Some(America/Los_Angeles)) AS from_xml(to_xml(named_struct(name, Eason, birth, 2018, org, Kindergarten Cop)))#x]
++- OneRowRelation
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/binary_hex.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/binary_hex.sql.out
index fe61e684a7ff5..b2781c56ca7da 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/binary_hex.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/binary_hex.sql.out
@@ -32,3 +32,28 @@ SELECT to_csv(named_struct('n', 1, 'info', X'4561736F6E2059616F20323031382D31312
-- !query analysis
Project [to_csv(named_struct(n, 1, info, 0x4561736F6E2059616F20323031382D31312D31373A31333A33333A3333), Some(America/Los_Angeles)) AS to_csv(named_struct(n, 1, info, X'4561736F6E2059616F20323031382D31312D31373A31333A33333A3333'))#x]
+- OneRowRelation
+
+
+-- !query
+select to_xml(named_struct('name', binary('Eason'), 'birth', 2018, 'org', binary('Kindergarten Cop')))
+-- !query analysis
+Project [to_xml(named_struct(name, cast(Eason as binary), birth, 2018, org, cast(Kindergarten Cop as binary)), Some(America/Los_Angeles)) AS to_xml(named_struct(name, Eason, birth, 2018, org, Kindergarten Cop))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT from_xml(
+ to_xml(named_struct('name', binary('Eason'), 'birth', 2018, 'org', binary('Kindergarten Cop'))),
+ 'name STRING, birth INT, org STRING')
+-- !query analysis
+Project [from_xml(StructField(name,StringType,true), StructField(birth,IntegerType,true), StructField(org,StringType,true), to_xml(named_struct(name, cast(Eason as binary), birth, 2018, org, cast(Kindergarten Cop as binary)), Some(America/Los_Angeles)), Some(America/Los_Angeles)) AS from_xml(to_xml(named_struct(name, Eason, birth, 2018, org, Kindergarten Cop)))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT from_xml(
+ to_xml(named_struct('name', binary('Eason'), 'birth', 2018, 'org', binary('Kindergarten Cop'))),
+ 'name binary, birth INT, org binary')
+-- !query analysis
+Project [from_xml(StructField(name,BinaryType,true), StructField(birth,IntegerType,true), StructField(org,BinaryType,true), to_xml(named_struct(name, cast(Eason as binary), birth, 2018, org, cast(Kindergarten Cop as binary)), Some(America/Los_Angeles)), Some(America/Los_Angeles)) AS from_xml(to_xml(named_struct(name, Eason, birth, 2018, org, Kindergarten Cop)))#x]
++- OneRowRelation
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/binary_hex_discrete.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/binary_hex_discrete.sql.out
index fe61e684a7ff5..b2781c56ca7da 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/binary_hex_discrete.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/binary_hex_discrete.sql.out
@@ -32,3 +32,28 @@ SELECT to_csv(named_struct('n', 1, 'info', X'4561736F6E2059616F20323031382D31312
-- !query analysis
Project [to_csv(named_struct(n, 1, info, 0x4561736F6E2059616F20323031382D31312D31373A31333A33333A3333), Some(America/Los_Angeles)) AS to_csv(named_struct(n, 1, info, X'4561736F6E2059616F20323031382D31312D31373A31333A33333A3333'))#x]
+- OneRowRelation
+
+
+-- !query
+select to_xml(named_struct('name', binary('Eason'), 'birth', 2018, 'org', binary('Kindergarten Cop')))
+-- !query analysis
+Project [to_xml(named_struct(name, cast(Eason as binary), birth, 2018, org, cast(Kindergarten Cop as binary)), Some(America/Los_Angeles)) AS to_xml(named_struct(name, Eason, birth, 2018, org, Kindergarten Cop))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT from_xml(
+ to_xml(named_struct('name', binary('Eason'), 'birth', 2018, 'org', binary('Kindergarten Cop'))),
+ 'name STRING, birth INT, org STRING')
+-- !query analysis
+Project [from_xml(StructField(name,StringType,true), StructField(birth,IntegerType,true), StructField(org,StringType,true), to_xml(named_struct(name, cast(Eason as binary), birth, 2018, org, cast(Kindergarten Cop as binary)), Some(America/Los_Angeles)), Some(America/Los_Angeles)) AS from_xml(to_xml(named_struct(name, Eason, birth, 2018, org, Kindergarten Cop)))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT from_xml(
+ to_xml(named_struct('name', binary('Eason'), 'birth', 2018, 'org', binary('Kindergarten Cop'))),
+ 'name binary, birth INT, org binary')
+-- !query analysis
+Project [from_xml(StructField(name,BinaryType,true), StructField(birth,IntegerType,true), StructField(org,BinaryType,true), to_xml(named_struct(name, cast(Eason as binary), birth, 2018, org, cast(Kindergarten Cop as binary)), Some(America/Los_Angeles)), Some(America/Los_Angeles)) AS from_xml(to_xml(named_struct(name, Eason, birth, 2018, org, Kindergarten Cop)))#x]
++- OneRowRelation
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cast.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cast.sql.out
index 643dfd3771ffe..053d7af3df45f 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/cast.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cast.sql.out
@@ -881,6 +881,97 @@ Project [cast(10.654321 as interval month) AS CAST(10.654321 AS INTERVAL MONTH)#
+- OneRowRelation
+-- !query
+SELECT CAST(TIME '00:01:52' AS tinyint)
+-- !query analysis
+Project [cast(00:01:52 as tinyint) AS CAST(TIME '00:01:52' AS TINYINT)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(TIME '00:01:52' AS smallint)
+-- !query analysis
+Project [cast(00:01:52 as smallint) AS CAST(TIME '00:01:52' AS SMALLINT)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(TIME '00:01:52' AS int)
+-- !query analysis
+Project [cast(00:01:52 as int) AS CAST(TIME '00:01:52' AS INT)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(TIME '00:01:52' AS bigint)
+-- !query analysis
+Project [cast(00:01:52 as bigint) AS CAST(TIME '00:01:52' AS BIGINT)#xL]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(TIME '23:59:59' AS tinyint)
+-- !query analysis
+Project [cast(23:59:59 as tinyint) AS CAST(TIME '23:59:59' AS TINYINT)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(TIME '23:59:59' AS smallint)
+-- !query analysis
+Project [cast(23:59:59 as smallint) AS CAST(TIME '23:59:59' AS SMALLINT)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(TIME '23:59:59' AS int)
+-- !query analysis
+Project [cast(23:59:59 as int) AS CAST(TIME '23:59:59' AS INT)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(TIME '23:59:59' AS bigint)
+-- !query analysis
+Project [cast(23:59:59 as bigint) AS CAST(TIME '23:59:59' AS BIGINT)#xL]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(TIME '00:00:17.5' AS tinyint)
+-- !query analysis
+Project [cast(00:00:17.5 as tinyint) AS CAST(TIME '00:00:17.5' AS TINYINT)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(TIME '00:00:17.5' AS int)
+-- !query analysis
+Project [cast(00:00:17.5 as int) AS CAST(TIME '00:00:17.5' AS INT)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(TIME '00:00:17.9' AS int)
+-- !query analysis
+Project [cast(00:00:17.9 as int) AS CAST(TIME '00:00:17.9' AS INT)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(TIME '00:00:00' AS tinyint)
+-- !query analysis
+Project [cast(00:00:00 as tinyint) AS CAST(TIME '00:00:00' AS TINYINT)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(TIME '00:00:00' AS int)
+-- !query analysis
+Project [cast(00:00:00 as int) AS CAST(TIME '00:00:00' AS INT)#x]
++- OneRowRelation
+
+
-- !query
SELECT '1.23' :: int
-- !query analysis
@@ -1033,6 +1124,34 @@ Project [cast(10.123456 as interval day to second) AS CAST(10.123456 AS INTERVAL
+- OneRowRelation
+-- !query
+SELECT TIME '00:01:52' :: tinyint
+-- !query analysis
+Project [cast(00:01:52 as tinyint) AS CAST(TIME '00:01:52' AS TINYINT)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT TIME '00:01:52' :: int
+-- !query analysis
+Project [cast(00:01:52 as int) AS CAST(TIME '00:01:52' AS INT)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT TIME '23:59:59' :: tinyint
+-- !query analysis
+Project [cast(23:59:59 as tinyint) AS CAST(TIME '23:59:59' AS TINYINT)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT TIME '23:59:59' :: int
+-- !query analysis
+Project [cast(23:59:59 as int) AS CAST(TIME '23:59:59' AS INT)#x]
++- OneRowRelation
+
+
-- !query
SELECT '1.23' :: int :: long
-- !query analysis
@@ -1113,3 +1232,122 @@ org.apache.spark.sql.catalyst.parser.ParseException
"hint" : ""
}
}
+
+
+-- !query
+SELECT CAST(time '00:00:00' AS decimal(1, 0))
+-- !query analysis
+Project [cast(00:00:00 as decimal(1,0)) AS CAST(TIME '00:00:00' AS DECIMAL(1,0))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '12:00:00' AS decimal(7, 2))
+-- !query analysis
+Project [cast(12:00:00 as decimal(7,2)) AS CAST(TIME '12:00:00' AS DECIMAL(7,2))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '01:30:45' AS decimal(8, 3))
+-- !query analysis
+Project [cast(01:30:45 as decimal(8,3)) AS CAST(TIME '01:30:45' AS DECIMAL(8,3))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '23:59:59' AS decimal(9, 4))
+-- !query analysis
+Project [cast(23:59:59 as decimal(9,4)) AS CAST(TIME '23:59:59' AS DECIMAL(9,4))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '01:02:03' AS decimal(15, 9))
+-- !query analysis
+Project [cast(01:02:03 as decimal(15,9)) AS CAST(TIME '01:02:03' AS DECIMAL(15,9))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '10:20:30' AS decimal(20, 10))
+-- !query analysis
+Project [cast(10:20:30 as decimal(20,10)) AS CAST(TIME '10:20:30' AS DECIMAL(20,10))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '23:59:59.001' AS decimal(8, 3))
+-- !query analysis
+Project [cast(23:59:59.001 as decimal(8,3)) AS CAST(TIME '23:59:59.001' AS DECIMAL(8,3))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '23:59:59.999999' AS decimal(11, 6))
+-- !query analysis
+Project [cast(23:59:59.999999 as decimal(11,6)) AS CAST(TIME '23:59:59.999999' AS DECIMAL(11,6))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '23:59:59.999999999' AS decimal(14, 9))
+-- !query analysis
+Project [cast(23:59:59.999999 as decimal(14,9)) AS CAST(TIME '23:59:59.999999' AS DECIMAL(14,9))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '23:59:59.999999999' AS decimal(20, 10))
+-- !query analysis
+Project [cast(23:59:59.999999 as decimal(20,10)) AS CAST(TIME '23:59:59.999999' AS DECIMAL(20,10))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '00:01:00' AS decimal(1, 0))
+-- !query analysis
+Project [cast(00:01:00 as decimal(1,0)) AS CAST(TIME '00:01:00' AS DECIMAL(1,0))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '01:00:00' AS decimal(3, 0))
+-- !query analysis
+Project [cast(01:00:00 as decimal(3,0)) AS CAST(TIME '01:00:00' AS DECIMAL(3,0))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '10:00:00' AS decimal(5, 2))
+-- !query analysis
+Project [cast(10:00:00 as decimal(5,2)) AS CAST(TIME '10:00:00' AS DECIMAL(5,2))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '23:59:59.9' AS decimal(6, 0))
+-- !query analysis
+Project [cast(23:59:59.9 as decimal(6,0)) AS CAST(TIME '23:59:59.9' AS DECIMAL(6,0))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '23:59:59.999' AS decimal(8, 2))
+-- !query analysis
+Project [cast(23:59:59.999 as decimal(8,2)) AS CAST(TIME '23:59:59.999' AS DECIMAL(8,2))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '23:59:59.999999' AS decimal(11, 5))
+-- !query analysis
+Project [cast(23:59:59.999999 as decimal(11,5)) AS CAST(TIME '23:59:59.999999' AS DECIMAL(11,5))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '23:59:59.999999999' AS decimal(14, 8))
+-- !query analysis
+Project [cast(23:59:59.999999 as decimal(14,8)) AS CAST(TIME '23:59:59.999999' AS DECIMAL(14,8))#x]
++- OneRowRelation
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out
index 524797015a2f6..4e864523368d7 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out
@@ -376,7 +376,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`char_tbl4`, false
insert into char_tbl4 select c, c, v, c from str_view
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/char_tbl4, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/char_tbl4], Append, `spark_catalog`.`default`.`char_tbl4`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/char_tbl4), [c7, c8, v, s]
-+- Project [static_invoke(CharVarcharCodegenUtils.charTypeWriteSideCheck(cast(c#x as string), 7)) AS c7#x, static_invoke(CharVarcharCodegenUtils.charTypeWriteSideCheck(cast(c#x as string), 8)) AS c8#x, static_invoke(CharVarcharCodegenUtils.varcharTypeWriteSideCheck(cast(v#x as string), 6)) AS v#x, cast(c#x as string) AS s#x]
++- Project [static_invoke(CharVarcharCodegenUtils.charTypeWriteSideCheck(cast(c#x as string), 7)) AS c7#x, static_invoke(CharVarcharCodegenUtils.charTypeWriteSideCheck(cast(c#x as string), 8)) AS c8#x, static_invoke(CharVarcharCodegenUtils.varcharTypeWriteSideCheck(cast(v#x as string), 6)) AS v#x, c#x AS s#x]
+- Project [c#x, c#x, v#x, c#x]
+- SubqueryAlias str_view
+- View (`str_view`, [c#x, v#x])
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/collations-aliases.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/collations-aliases.sql.out
new file mode 100644
index 0000000000000..454682e392be8
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/collations-aliases.sql.out
@@ -0,0 +1,335 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+create table t1(s string, utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet
+-- !query analysis
+CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false
+
+
+-- !query
+insert into t1 values ('Spark', 'Spark', 'SQL')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaAAaA')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaaAaA')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaaAaAaaAaaAaAaaAaaAaA')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('bbAbaAbA', 'bbAbAAbA', 'a')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('İo', 'İo', 'İo')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('İo', 'İo', 'İo ')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('İo', 'İo ', 'İo')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('İo', 'İo', 'i̇o')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('efd2', 'efd2', 'efd2')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('Hello, world! Nice day.', 'Hello, world! Nice day.', 'Hello, world! Nice day.')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('Something else. Nothing here.', 'Something else. Nothing here.', 'Something else. Nothing here.')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('kitten', 'kitten', 'sitTing')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('abc', 'abc', 'abc')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('abcdcba', 'abcdcba', 'aBcDCbA')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+select concat_ws(' ', utf8_lcase, utf8_lcase) from t1
+-- !query analysis
+Project [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select `concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)` from (
+ select concat_ws(' ', utf8_lcase, utf8_lcase) from t1
+)
+-- !query analysis
+Project [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
++- SubqueryAlias __auto_generated_subquery_name
+ +- Project [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select * from (
+ select concat_ws(' ', utf8_lcase, utf8_lcase) from t1
+)
+-- !query analysis
+Project [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
++- SubqueryAlias __auto_generated_subquery_name
+ +- Project [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select subq1.* from (
+ select concat_ws(' ', utf8_lcase, utf8_lcase) from t1
+) AS subq1
+-- !query analysis
+Project [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
++- SubqueryAlias subq1
+ +- Project [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+with cte as (
+ select concat_ws(' ', utf8_lcase, utf8_lcase) from t1
+)
+select * from cte
+-- !query analysis
+WithCTE
+:- CTERelationDef xxxx, false
+: +- SubqueryAlias cte
+: +- Project [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
+: +- SubqueryAlias spark_catalog.default.t1
+: +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
++- Project [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
+ +- SubqueryAlias cte
+ +- CTERelationRef xxxx, true, [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x], false, false
+
+
+-- !query
+select * from values (1) where exists (
+ select concat_ws(' ', utf8_lcase, utf8_lcase) from t1
+)
+-- !query analysis
+Project [col1#x]
++- Filter exists#x []
+ : +- Project [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
+ : +- SubqueryAlias spark_catalog.default.t1
+ : +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+ +- LocalRelation [col1#x]
+
+
+-- !query
+select (
+ select concat_ws(' ', utf8_lcase, utf8_lcase) from t1 limit 1
+)
+-- !query analysis
+Project [scalar-subquery#x [] AS scalarsubquery()#x]
+: +- GlobalLimit 1
+: +- LocalLimit 1
+: +- Project [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
+: +- SubqueryAlias spark_catalog.default.t1
+: +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
++- OneRowRelation
+
+
+-- !query
+select (
+ with cte as (
+ select concat_ws(' ', utf8_lcase, utf8_lcase) from t1
+ )
+ select * from cte limit 1
+)
+-- !query analysis
+Project [scalar-subquery#x [] AS scalarsubquery()#x]
+: +- WithCTE
+: :- CTERelationDef xxxx, false
+: : +- SubqueryAlias cte
+: : +- Project [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
+: : +- SubqueryAlias spark_catalog.default.t1
+: : +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+: +- GlobalLimit 1
+: +- LocalLimit 1
+: +- Project [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
+: +- SubqueryAlias cte
+: +- CTERelationRef xxxx, true, [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x], false, false
++- OneRowRelation
+
+
+-- !query
+select * from (
+ select concat_ws(' ', utf8_lcase, utf8_lcase) from t1 limit 1
+)
+where (
+ `concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)` == 'aaa'
+)
+-- !query analysis
+Project [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
++- Filter (concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x = aaa)
+ +- SubqueryAlias __auto_generated_subquery_name
+ +- GlobalLimit 1
+ +- LocalLimit 1
+ +- Project [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select lower(`concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)`) from (
+ select concat_ws(' ', utf8_lcase, utf8_lcase) from t1
+ group by 1
+ order by 1
+)
+-- !query analysis
+Project [lower(concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x) AS lower(concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase))#x]
++- SubqueryAlias __auto_generated_subquery_name
+ +- Sort [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x ASC NULLS FIRST], true
+ +- Aggregate [concat_ws( , utf8_lcase#x, utf8_lcase#x)], [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select lower(`concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)`) from (
+ select concat_ws(' ', utf8_lcase, utf8_lcase) from t1
+ group by 1
+ order by max(concat_ws(' ', utf8_lcase, utf8_lcase))
+)
+-- !query analysis
+Project [lower(concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x) AS lower(concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase))#x]
++- SubqueryAlias __auto_generated_subquery_name
+ +- Project [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
+ +- Sort [max(concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase))#x ASC NULLS FIRST], true
+ +- Aggregate [concat_ws( , utf8_lcase#x, utf8_lcase#x)], [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x, max(concat_ws( , utf8_lcase#x, utf8_lcase#x)) AS max(concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase))#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+create temporary view v1 as (
+ select concat_ws(' ', utf8_lcase, utf8_lcase) from t1
+)
+-- !query analysis
+CreateViewCommand `v1`, (
+ select concat_ws(' ', utf8_lcase, utf8_lcase) from t1
+), false, false, LocalTempView, UNSUPPORTED, true
+ +- Project [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select * from v1
+-- !query analysis
+Project [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
++- SubqueryAlias v1
+ +- View (`v1`, [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x])
+ +- Project [cast(concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x as string collate UTF8_LCASE) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
+ +- Project [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select `concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)` from v1
+-- !query analysis
+Project [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
++- SubqueryAlias v1
+ +- View (`v1`, [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x])
+ +- Project [cast(concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x as string collate UTF8_LCASE) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
+ +- Project [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+drop view v1
+-- !query analysis
+DropTempViewCommand v1
+
+
+-- !query
+drop table t1
+-- !query analysis
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t1
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/collations-basic.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/collations-basic.sql.out
new file mode 100644
index 0000000000000..03409b001c153
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/collations-basic.sql.out
@@ -0,0 +1,1164 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+create table t1(utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet
+-- !query analysis
+CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false
+
+
+-- !query
+insert into t1 values('aaa', 'aaa')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [utf8_binary, utf8_lcase]
++- Project [col1#x AS utf8_binary#x, cast(col2#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+insert into t1 values('AAA', 'AAA')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [utf8_binary, utf8_lcase]
++- Project [col1#x AS utf8_binary#x, cast(col2#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+insert into t1 values('bbb', 'bbb')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [utf8_binary, utf8_lcase]
++- Project [col1#x AS utf8_binary#x, cast(col2#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+insert into t1 values('BBB', 'BBB')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [utf8_binary, utf8_lcase]
++- Project [col1#x AS utf8_binary#x, cast(col2#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+describe table t1
+-- !query analysis
+DescribeTableCommand `spark_catalog`.`default`.`t1`, false, [col_name#x, data_type#x, comment#x]
+
+
+-- !query
+select count(*) from t1 group by utf8_binary
+-- !query analysis
+Aggregate [utf8_binary#x], [count(1) AS count(1)#xL]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select count(*) from t1 group by utf8_lcase
+-- !query analysis
+Aggregate [utf8_lcase#x], [count(1) AS count(1)#xL]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select * from t1 where utf8_binary = 'aaa'
+-- !query analysis
+Project [utf8_binary#x, utf8_lcase#x]
++- Filter (utf8_binary#x = aaa)
+ +- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select * from t1 where utf8_lcase = 'aaa' collate utf8_lcase
+-- !query analysis
+Project [utf8_binary#x, utf8_lcase#x]
++- Filter (utf8_lcase#x = collate(aaa, utf8_lcase))
+ +- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select * from t1 where utf8_binary < 'bbb'
+-- !query analysis
+Project [utf8_binary#x, utf8_lcase#x]
++- Filter (utf8_binary#x < bbb)
+ +- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select * from t1 where utf8_lcase < 'bbb' collate utf8_lcase
+-- !query analysis
+Project [utf8_binary#x, utf8_lcase#x]
++- Filter (utf8_lcase#x < collate(bbb, utf8_lcase))
+ +- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select l.utf8_binary, r.utf8_lcase from t1 l join t1 r on l.utf8_lcase = r.utf8_lcase
+-- !query analysis
+Project [utf8_binary#x, utf8_lcase#x]
++- Join Inner, (utf8_lcase#x = utf8_lcase#x)
+ :- SubqueryAlias l
+ : +- SubqueryAlias spark_catalog.default.t1
+ : +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_lcase#x] parquet
+ +- SubqueryAlias r
+ +- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+create table t2(utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet
+-- !query analysis
+CreateDataSourceTableCommand `spark_catalog`.`default`.`t2`, false
+
+
+-- !query
+insert into t2 values('aaa', 'aaa')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t2], Append, `spark_catalog`.`default`.`t2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t2), [utf8_binary, utf8_lcase]
++- Project [col1#x AS utf8_binary#x, cast(col2#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+insert into t2 values('bbb', 'bbb')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t2], Append, `spark_catalog`.`default`.`t2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t2), [utf8_binary, utf8_lcase]
++- Project [col1#x AS utf8_binary#x, cast(col2#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+select * from t1 anti join t2 on t1.utf8_lcase = t2.utf8_lcase
+-- !query analysis
+Project [utf8_binary#x, utf8_lcase#x]
++- Join LeftAnti, (utf8_lcase#x = utf8_lcase#x)
+ :- SubqueryAlias spark_catalog.default.t1
+ : +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_lcase#x] parquet
+ +- SubqueryAlias spark_catalog.default.t2
+ +- Relation spark_catalog.default.t2[utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+drop table t2
+-- !query analysis
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t2
+
+
+-- !query
+drop table t1
+-- !query analysis
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t1
+
+
+-- !query
+select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except select col1 collate utf8_lcase from values ('aaa'), ('bbb')
+-- !query analysis
+Except false
+:- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x]
+: +- LocalRelation [col1#x]
++- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except all select col1 collate utf8_lcase from values ('aaa'), ('bbb')
+-- !query analysis
+Except All true
+:- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x]
+: +- LocalRelation [col1#x]
++- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union select col1 collate utf8_lcase from values ('aaa'), ('bbb')
+-- !query analysis
+Distinct
++- Union false, false
+ :- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x]
+ : +- LocalRelation [col1#x]
+ +- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union all select col1 collate utf8_lcase from values ('aaa'), ('bbb')
+-- !query analysis
+Union false, false
+:- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x]
+: +- LocalRelation [col1#x]
++- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+select col1 collate utf8_lcase from values ('aaa'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') intersect select col1 collate utf8_lcase from values ('aaa'), ('bbb')
+-- !query analysis
+Intersect false
+:- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x]
+: +- LocalRelation [col1#x]
++- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except select col1 collate unicode_ci from values ('aaa'), ('bbb')
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "INCOMPATIBLE_COLUMN_TYPE",
+ "sqlState" : "42825",
+ "messageParameters" : {
+ "columnOrdinalNumber" : "first",
+ "dataType1" : "\"STRING COLLATE UNICODE_CI\"",
+ "dataType2" : "\"STRING COLLATE UTF8_LCASE\"",
+ "hint" : "",
+ "operator" : "EXCEPT",
+ "tableOrdinalNumber" : "second"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 1,
+ "stopIndex" : 162,
+ "fragment" : "select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except select col1 collate unicode_ci from values ('aaa'), ('bbb')"
+ } ]
+}
+
+
+-- !query
+select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except all select col1 collate unicode_ci from values ('aaa'), ('bbb')
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "INCOMPATIBLE_COLUMN_TYPE",
+ "sqlState" : "42825",
+ "messageParameters" : {
+ "columnOrdinalNumber" : "first",
+ "dataType1" : "\"STRING COLLATE UNICODE_CI\"",
+ "dataType2" : "\"STRING COLLATE UTF8_LCASE\"",
+ "hint" : "",
+ "operator" : "EXCEPT ALL",
+ "tableOrdinalNumber" : "second"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 1,
+ "stopIndex" : 166,
+ "fragment" : "select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except all select col1 collate unicode_ci from values ('aaa'), ('bbb')"
+ } ]
+}
+
+
+-- !query
+select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union select col1 collate unicode_ci from values ('aaa'), ('bbb')
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "INCOMPATIBLE_COLUMN_TYPE",
+ "sqlState" : "42825",
+ "messageParameters" : {
+ "columnOrdinalNumber" : "first",
+ "dataType1" : "\"STRING COLLATE UNICODE_CI\"",
+ "dataType2" : "\"STRING COLLATE UTF8_LCASE\"",
+ "hint" : "",
+ "operator" : "UNION",
+ "tableOrdinalNumber" : "second"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 1,
+ "stopIndex" : 161,
+ "fragment" : "select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union select col1 collate unicode_ci from values ('aaa'), ('bbb')"
+ } ]
+}
+
+
+-- !query
+select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union all select col1 collate unicode_ci from values ('aaa'), ('bbb')
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "INCOMPATIBLE_COLUMN_TYPE",
+ "sqlState" : "42825",
+ "messageParameters" : {
+ "columnOrdinalNumber" : "first",
+ "dataType1" : "\"STRING COLLATE UNICODE_CI\"",
+ "dataType2" : "\"STRING COLLATE UTF8_LCASE\"",
+ "hint" : "",
+ "operator" : "UNION",
+ "tableOrdinalNumber" : "second"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 1,
+ "stopIndex" : 165,
+ "fragment" : "select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union all select col1 collate unicode_ci from values ('aaa'), ('bbb')"
+ } ]
+}
+
+
+-- !query
+select col1 collate utf8_lcase from values ('aaa'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') intersect select col1 collate unicode_ci from values ('aaa'), ('bbb')
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "INCOMPATIBLE_COLUMN_TYPE",
+ "sqlState" : "42825",
+ "messageParameters" : {
+ "columnOrdinalNumber" : "first",
+ "dataType1" : "\"STRING COLLATE UNICODE_CI\"",
+ "dataType2" : "\"STRING COLLATE UTF8_LCASE\"",
+ "hint" : "",
+ "operator" : "INTERSECT",
+ "tableOrdinalNumber" : "second"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 1,
+ "stopIndex" : 156,
+ "fragment" : "select col1 collate utf8_lcase from values ('aaa'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') intersect select col1 collate unicode_ci from values ('aaa'), ('bbb')"
+ } ]
+}
+
+
+-- !query
+create table t1 (c1 struct) USING PARQUET
+-- !query analysis
+CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false
+
+
+-- !query
+insert into t1 values (named_struct('utf8_binary', 'aaa', 'utf8_lcase', 'aaa'))
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [c1]
++- Project [named_struct(utf8_binary, col1#x.utf8_binary, utf8_lcase, cast(col1#x.utf8_lcase as string collate UTF8_LCASE)) AS c1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+insert into t1 values (named_struct('utf8_binary', 'AAA', 'utf8_lcase', 'AAA'))
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [c1]
++- Project [named_struct(utf8_binary, col1#x.utf8_binary, utf8_lcase, cast(col1#x.utf8_lcase as string collate UTF8_LCASE)) AS c1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+select count(*) from t1 group by c1.utf8_binary
+-- !query analysis
+Aggregate [c1#x.utf8_binary], [count(1) AS count(1)#xL]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[c1#x] parquet
+
+
+-- !query
+select count(*) from t1 group by c1.utf8_lcase
+-- !query analysis
+Aggregate [c1#x.utf8_lcase], [count(1) AS count(1)#xL]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[c1#x] parquet
+
+
+-- !query
+drop table t1
+-- !query analysis
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t1
+
+
+-- !query
+select array_contains(ARRAY('aaa' collate utf8_lcase),'AAA' collate utf8_lcase)
+-- !query analysis
+Project [array_contains(array(collate(aaa, utf8_lcase)), collate(AAA, utf8_lcase)) AS array_contains(array(collate(aaa, utf8_lcase)), collate(AAA, utf8_lcase))#x]
++- OneRowRelation
+
+
+-- !query
+select array_position(ARRAY('aaa' collate utf8_lcase, 'bbb' collate utf8_lcase),'BBB' collate utf8_lcase)
+-- !query analysis
+Project [array_position(array(collate(aaa, utf8_lcase), collate(bbb, utf8_lcase)), collate(BBB, utf8_lcase)) AS array_position(array(collate(aaa, utf8_lcase), collate(bbb, utf8_lcase)), collate(BBB, utf8_lcase))#xL]
++- OneRowRelation
+
+
+-- !query
+select nullif('aaa' COLLATE utf8_lcase, 'AAA' COLLATE utf8_lcase)
+-- !query analysis
+Project [nullif(collate(aaa, utf8_lcase), collate(AAA, utf8_lcase)) AS nullif(collate(aaa, utf8_lcase), collate(AAA, utf8_lcase))#x]
++- OneRowRelation
+
+
+-- !query
+select least('aaa' COLLATE utf8_lcase, 'AAA' collate utf8_lcase, 'a' collate utf8_lcase)
+-- !query analysis
+Project [least(collate(aaa, utf8_lcase), collate(AAA, utf8_lcase), collate(a, utf8_lcase)) AS least(collate(aaa, utf8_lcase), collate(AAA, utf8_lcase), collate(a, utf8_lcase))#x]
++- OneRowRelation
+
+
+-- !query
+select arrays_overlap(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase))
+-- !query analysis
+Project [arrays_overlap(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase))) AS arrays_overlap(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase)))#x]
++- OneRowRelation
+
+
+-- !query
+select array_distinct(array('aaa' collate utf8_lcase, 'AAA' collate utf8_lcase))
+-- !query analysis
+Project [array_distinct(array(collate(aaa, utf8_lcase), collate(AAA, utf8_lcase))) AS array_distinct(array(collate(aaa, utf8_lcase), collate(AAA, utf8_lcase)))#x]
++- OneRowRelation
+
+
+-- !query
+select array_union(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase))
+-- !query analysis
+Project [array_union(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase))) AS array_union(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase)))#x]
++- OneRowRelation
+
+
+-- !query
+select array_intersect(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase))
+-- !query analysis
+Project [array_intersect(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase))) AS array_intersect(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase)))#x]
++- OneRowRelation
+
+
+-- !query
+select array_except(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase))
+-- !query analysis
+Project [array_except(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase))) AS array_except(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase)))#x]
++- OneRowRelation
+
+
+-- !query
+select 'a' collate unicode < 'A'
+-- !query analysis
+Project [(collate(a, unicode) < A) AS (collate(a, unicode) < 'A' collate UNICODE)#x]
++- OneRowRelation
+
+
+-- !query
+select 'a' collate unicode_ci = 'A'
+-- !query analysis
+Project [(collate(a, unicode_ci) = A) AS (collate(a, unicode_ci) = 'A' collate UNICODE_CI)#x]
++- OneRowRelation
+
+
+-- !query
+select 'a' collate unicode_ai = 'å'
+-- !query analysis
+Project [(collate(a, unicode_ai) = å) AS (collate(a, unicode_ai) = 'å' collate UNICODE_AI)#x]
++- OneRowRelation
+
+
+-- !query
+select 'a' collate unicode_ci_ai = 'Å'
+-- !query analysis
+Project [(collate(a, unicode_ci_ai) = Å) AS (collate(a, unicode_ci_ai) = 'Å' collate UNICODE_CI_AI)#x]
++- OneRowRelation
+
+
+-- !query
+select 'a' collate en < 'A'
+-- !query analysis
+Project [(collate(a, en) < A) AS (collate(a, en) < 'A' collate en)#x]
++- OneRowRelation
+
+
+-- !query
+select 'a' collate en_ci = 'A'
+-- !query analysis
+Project [(collate(a, en_ci) = A) AS (collate(a, en_ci) = 'A' collate en_CI)#x]
++- OneRowRelation
+
+
+-- !query
+select 'a' collate en_ai = 'å'
+-- !query analysis
+Project [(collate(a, en_ai) = å) AS (collate(a, en_ai) = 'å' collate en_AI)#x]
++- OneRowRelation
+
+
+-- !query
+select 'a' collate en_ci_ai = 'Å'
+-- !query analysis
+Project [(collate(a, en_ci_ai) = Å) AS (collate(a, en_ci_ai) = 'Å' collate en_CI_AI)#x]
++- OneRowRelation
+
+
+-- !query
+select 'Kypper' collate sv < 'Köpfe'
+-- !query analysis
+Project [(collate(Kypper, sv) < Köpfe) AS (collate(Kypper, sv) < 'Köpfe' collate sv)#x]
++- OneRowRelation
+
+
+-- !query
+select 'Kypper' collate de > 'Köpfe'
+-- !query analysis
+Project [(collate(Kypper, de) > Köpfe) AS (collate(Kypper, de) > 'Köpfe' collate de)#x]
++- OneRowRelation
+
+
+-- !query
+select 'I' collate tr_ci = 'ı'
+-- !query analysis
+Project [(collate(I, tr_ci) = ı) AS (collate(I, tr_ci) = 'ı' collate tr_CI)#x]
++- OneRowRelation
+
+
+-- !query
+create table t3 (text string collate utf8_binary, pairDelim string collate utf8_lcase, keyValueDelim string collate utf8_binary) using parquet
+-- !query analysis
+CreateDataSourceTableCommand `spark_catalog`.`default`.`t3`, false
+
+
+-- !query
+insert into t3 values('a:1,b:2,c:3', ',', ':')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t3, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t3], Append, `spark_catalog`.`default`.`t3`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t3), [text, pairDelim, keyValueDelim]
++- Project [col1#x AS text#x, cast(col2#x as string collate UTF8_LCASE) AS pairDelim#x, col3#x AS keyValueDelim#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+select str_to_map(text, pairDelim, keyValueDelim) from t3
+-- !query analysis
+Project [str_to_map(cast(text#x as string collate null), cast(pairDelim#x as string collate null), cast(keyValueDelim#x as string collate null)) AS str_to_map(text, pairDelim, keyValueDelim)#x]
++- SubqueryAlias spark_catalog.default.t3
+ +- Relation spark_catalog.default.t3[text#x,pairDelim#x,keyValueDelim#x] parquet
+
+
+-- !query
+select str_to_map(text collate utf8_binary, pairDelim collate utf8_lcase, keyValueDelim collate utf8_binary) from t3
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
+ "sqlState" : "42P21",
+ "messageParameters" : {
+ "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
+ }
+}
+
+
+-- !query
+select str_to_map(text collate utf8_binary, pairDelim collate utf8_binary, keyValueDelim collate utf8_binary) from t3
+-- !query analysis
+Project [str_to_map(collate(text#x, utf8_binary), collate(pairDelim#x, utf8_binary), collate(keyValueDelim#x, utf8_binary)) AS str_to_map(collate(text, utf8_binary), collate(pairDelim, utf8_binary), collate(keyValueDelim, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t3
+ +- Relation spark_catalog.default.t3[text#x,pairDelim#x,keyValueDelim#x] parquet
+
+
+-- !query
+select str_to_map(text collate unicode_ai, pairDelim collate unicode_ai, keyValueDelim collate unicode_ai) from t3
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"collate(text, unicode_ai)\"",
+ "inputType" : "\"STRING COLLATE UNICODE_AI\"",
+ "paramIndex" : "first",
+ "requiredType" : "\"STRING\"",
+ "sqlExpr" : "\"str_to_map(collate(text, unicode_ai), collate(pairDelim, unicode_ai), collate(keyValueDelim, unicode_ai))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 106,
+ "fragment" : "str_to_map(text collate unicode_ai, pairDelim collate unicode_ai, keyValueDelim collate unicode_ai)"
+ } ]
+}
+
+
+-- !query
+drop table t3
+-- !query analysis
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t3
+
+
+-- !query
+create table t1(s string, utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet
+-- !query analysis
+CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false
+
+
+-- !query
+insert into t1 values ('Spark', 'Spark', 'SQL')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaAAaA')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaaAaA')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaaAaAaaAaaAaAaaAaaAaA')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('bbAbaAbA', 'bbAbAAbA', 'a')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('İo', 'İo', 'İo')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('İo', 'İo', 'İo ')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('İo', 'İo ', 'İo')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('İo', 'İo', 'i̇o')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('efd2', 'efd2', 'efd2')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('Hello, world! Nice day.', 'Hello, world! Nice day.', 'Hello, world! Nice day.')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('Something else. Nothing here.', 'Something else. Nothing here.', 'Something else. Nothing here.')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('kitten', 'kitten', 'sitTing')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('abc', 'abc', 'abc')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('abcdcba', 'abcdcba', 'aBcDCbA')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+create table t2(ascii double) using parquet
+-- !query analysis
+CreateDataSourceTableCommand `spark_catalog`.`default`.`t2`, false
+
+
+-- !query
+insert into t2 values (97.52143)
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t2], Append, `spark_catalog`.`default`.`t2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t2), [ascii]
++- Project [cast(col1#x as double) AS ascii#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+insert into t2 values (66.421)
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t2], Append, `spark_catalog`.`default`.`t2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t2), [ascii]
++- Project [cast(col1#x as double) AS ascii#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+create table t3(utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet
+-- !query analysis
+CreateDataSourceTableCommand `spark_catalog`.`default`.`t3`, false
+
+
+-- !query
+insert into t3 values ('aaAaAAaA', 'aaAaaAaA')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t3, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t3], Append, `spark_catalog`.`default`.`t3`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t3), [utf8_binary, utf8_lcase]
++- Project [col1#x AS utf8_binary#x, cast(col2#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+insert into t3 values ('efd2', 'efd2')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t3, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t3], Append, `spark_catalog`.`default`.`t3`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t3), [utf8_binary, utf8_lcase]
++- Project [col1#x AS utf8_binary#x, cast(col2#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+create table t4(num long) using parquet
+-- !query analysis
+CreateDataSourceTableCommand `spark_catalog`.`default`.`t4`, false
+
+
+-- !query
+insert into t4 values (97)
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t4, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t4], Append, `spark_catalog`.`default`.`t4`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t4), [num]
++- Project [cast(col1#x as bigint) AS num#xL]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+insert into t4 values (66)
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t4, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t4], Append, `spark_catalog`.`default`.`t4`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t4), [num]
++- Project [cast(col1#x as bigint) AS num#xL]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+select elt(2, s, utf8_binary) from t1
+-- !query analysis
+Project [elt(2, s#x, utf8_binary#x, true) AS elt(2, s, utf8_binary)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select elt(2, utf8_binary, utf8_lcase, s) from t1
+-- !query analysis
+Project [elt(2, cast(utf8_binary#x as string collate null), cast(utf8_lcase#x as string collate null), cast(s#x as string collate null), true) AS elt(2, utf8_binary, utf8_lcase, s)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select elt(1, utf8_binary collate utf8_binary, utf8_lcase collate utf8_lcase) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
+ "sqlState" : "42P21",
+ "messageParameters" : {
+ "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
+ }
+}
+
+
+-- !query
+select elt(1, utf8_binary collate utf8_binary, utf8_lcase collate utf8_binary) from t1
+-- !query analysis
+Project [elt(1, collate(utf8_binary#x, utf8_binary), collate(utf8_lcase#x, utf8_binary), true) AS elt(1, collate(utf8_binary, utf8_binary), collate(utf8_lcase, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select elt(1, utf8_binary collate utf8_binary, utf8_lcase) from t1
+-- !query analysis
+Project [elt(1, collate(utf8_binary#x, utf8_binary), cast(utf8_lcase#x as string), true) AS elt(1, collate(utf8_binary, utf8_binary), utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select elt(1, utf8_binary, 'word'), elt(1, utf8_lcase, 'word') from t1
+-- !query analysis
+Project [elt(1, utf8_binary#x, word, true) AS elt(1, utf8_binary, word)#x, elt(1, utf8_lcase#x, word, true) AS elt(1, utf8_lcase, 'word' collate UTF8_LCASE)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select elt(1, utf8_binary, 'word' collate utf8_lcase), elt(1, utf8_lcase, 'word' collate utf8_binary) from t1
+-- !query analysis
+Project [elt(1, cast(utf8_binary#x as string collate UTF8_LCASE), collate(word, utf8_lcase), true) AS elt(1, utf8_binary, collate(word, utf8_lcase))#x, elt(1, cast(utf8_lcase#x as string), collate(word, utf8_binary), true) AS elt(1, utf8_lcase, collate(word, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select ascii(utf8_binary), ascii(utf8_lcase) from t1
+-- !query analysis
+Project [ascii(utf8_binary#x) AS ascii(utf8_binary)#x, ascii(utf8_lcase#x) AS ascii(utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select ascii(utf8_binary collate utf8_lcase), ascii(utf8_lcase collate utf8_binary) from t1
+-- !query analysis
+Project [ascii(collate(utf8_binary#x, utf8_lcase)) AS ascii(collate(utf8_binary, utf8_lcase))#x, ascii(collate(utf8_lcase#x, utf8_binary)) AS ascii(collate(utf8_lcase, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select unbase64(utf8_binary), unbase64(utf8_lcase) from t3
+-- !query analysis
+Project [unbase64(utf8_binary#x, false) AS unbase64(utf8_binary)#x, unbase64(utf8_lcase#x, false) AS unbase64(utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t3
+ +- Relation spark_catalog.default.t3[utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select unbase64(utf8_binary collate utf8_lcase), unbase64(utf8_lcase collate utf8_binary) from t3
+-- !query analysis
+Project [unbase64(collate(utf8_binary#x, utf8_lcase), false) AS unbase64(collate(utf8_binary, utf8_lcase))#x, unbase64(collate(utf8_lcase#x, utf8_binary), false) AS unbase64(collate(utf8_lcase, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t3
+ +- Relation spark_catalog.default.t3[utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select base64(utf8_binary), base64(utf8_lcase) from t1
+-- !query analysis
+Project [base64(cast(utf8_binary#x as binary)) AS base64(utf8_binary)#x, base64(cast(utf8_lcase#x as binary)) AS base64(utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select base64(utf8_binary collate utf8_lcase), base64(utf8_lcase collate utf8_binary) from t1
+-- !query analysis
+Project [base64(cast(collate(utf8_binary#x, utf8_lcase) as binary)) AS base64(collate(utf8_binary, utf8_lcase))#x, base64(cast(collate(utf8_lcase#x, utf8_binary) as binary)) AS base64(collate(utf8_lcase, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select decode(encode(utf8_binary, 'utf-8'), 'utf-8'), decode(encode(utf8_lcase, 'utf-8'), 'utf-8') from t1
+-- !query analysis
+Project [decode(encode(utf8_binary#x, utf-8), utf-8) AS decode(encode(utf8_binary, utf-8), utf-8)#x, decode(encode(utf8_lcase#x, utf-8), utf-8) AS decode(encode(utf8_lcase, utf-8), utf-8)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select decode(encode(utf8_binary collate utf8_lcase, 'utf-8'), 'utf-8'), decode(encode(utf8_lcase collate utf8_binary, 'utf-8'), 'utf-8') from t1
+-- !query analysis
+Project [decode(encode(collate(utf8_binary#x, utf8_lcase), utf-8), utf-8) AS decode(encode(collate(utf8_binary, utf8_lcase), utf-8), utf-8)#x, decode(encode(collate(utf8_lcase#x, utf8_binary), utf-8), utf-8) AS decode(encode(collate(utf8_lcase, utf8_binary), utf-8), utf-8)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select format_number(ascii, '###.###') from t2
+-- !query analysis
+Project [format_number(ascii#x, ###.###) AS format_number(ascii, ###.###)#x]
++- SubqueryAlias spark_catalog.default.t2
+ +- Relation spark_catalog.default.t2[ascii#x] parquet
+
+
+-- !query
+select format_number(ascii, '###.###' collate utf8_lcase) from t2
+-- !query analysis
+Project [format_number(ascii#x, collate(###.###, utf8_lcase)) AS format_number(ascii, collate(###.###, utf8_lcase))#x]
++- SubqueryAlias spark_catalog.default.t2
+ +- Relation spark_catalog.default.t2[ascii#x] parquet
+
+
+-- !query
+select encode(utf8_binary, 'utf-8'), encode(utf8_lcase, 'utf-8') from t1
+-- !query analysis
+Project [encode(utf8_binary#x, utf-8) AS encode(utf8_binary, utf-8)#x, encode(utf8_lcase#x, utf-8) AS encode(utf8_lcase, utf-8)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select encode(utf8_binary collate utf8_lcase, 'utf-8'), encode(utf8_lcase collate utf8_binary, 'utf-8') from t1
+-- !query analysis
+Project [encode(collate(utf8_binary#x, utf8_lcase), utf-8) AS encode(collate(utf8_binary, utf8_lcase), utf-8)#x, encode(collate(utf8_lcase#x, utf8_binary), utf-8) AS encode(collate(utf8_lcase, utf8_binary), utf-8)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select to_binary(utf8_binary, 'utf-8'), to_binary(utf8_lcase, 'utf-8') from t1
+-- !query analysis
+Project [to_binary(utf8_binary#x, Some(utf-8), false) AS to_binary(utf8_binary, utf-8)#x, to_binary(utf8_lcase#x, Some(utf-8), false) AS to_binary(utf8_lcase, utf-8)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select to_binary(utf8_binary collate utf8_lcase, 'utf-8'), to_binary(utf8_lcase collate utf8_binary, 'utf-8') from t1
+-- !query analysis
+Project [to_binary(collate(utf8_binary#x, utf8_lcase), Some(utf-8), false) AS to_binary(collate(utf8_binary, utf8_lcase), utf-8)#x, to_binary(collate(utf8_lcase#x, utf8_binary), Some(utf-8), false) AS to_binary(collate(utf8_lcase, utf8_binary), utf-8)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select soundex(utf8_binary), soundex(utf8_lcase) from t1
+-- !query analysis
+Project [soundex(utf8_binary#x) AS soundex(utf8_binary)#x, soundex(utf8_lcase#x) AS soundex(utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select soundex(utf8_binary collate utf8_lcase), soundex(utf8_lcase collate utf8_binary) from t1
+-- !query analysis
+Project [soundex(collate(utf8_binary#x, utf8_lcase)) AS soundex(collate(utf8_binary, utf8_lcase))#x, soundex(collate(utf8_lcase#x, utf8_binary)) AS soundex(collate(utf8_lcase, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select luhn_check(num) from t4
+-- !query analysis
+Project [luhn_check(cast(num#xL as string)) AS luhn_check(num)#x]
++- SubqueryAlias spark_catalog.default.t4
+ +- Relation spark_catalog.default.t4[num#xL] parquet
+
+
+-- !query
+select levenshtein(utf8_binary, utf8_lcase) from t1
+-- !query analysis
+Project [levenshtein(cast(utf8_binary#x as string collate null), cast(utf8_lcase#x as string collate null), None) AS levenshtein(utf8_binary, utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select levenshtein(s, utf8_binary) from t1
+-- !query analysis
+Project [levenshtein(s#x, utf8_binary#x, None) AS levenshtein(s, utf8_binary)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select levenshtein(utf8_binary collate utf8_binary, s collate utf8_lcase) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
+ "sqlState" : "42P21",
+ "messageParameters" : {
+ "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
+ }
+}
+
+
+-- !query
+select levenshtein(utf8_binary, utf8_lcase collate utf8_binary) from t1
+-- !query analysis
+Project [levenshtein(utf8_binary#x, collate(utf8_lcase#x, utf8_binary), None) AS levenshtein(utf8_binary, collate(utf8_lcase, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select levenshtein(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t1
+-- !query analysis
+Project [levenshtein(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase), None) AS levenshtein(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select levenshtein(utf8_binary, 'a'), levenshtein(utf8_lcase, 'a') from t1
+-- !query analysis
+Project [levenshtein(utf8_binary#x, a, None) AS levenshtein(utf8_binary, a)#x, levenshtein(utf8_lcase#x, a, None) AS levenshtein(utf8_lcase, 'a' collate UTF8_LCASE)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select levenshtein(utf8_binary, 'AaAA' collate utf8_lcase, 3), levenshtein(utf8_lcase, 'AAa' collate utf8_binary, 4) from t1
+-- !query analysis
+Project [levenshtein(cast(utf8_binary#x as string collate UTF8_LCASE), collate(AaAA, utf8_lcase), Some(3)) AS levenshtein(utf8_binary, collate(AaAA, utf8_lcase), 3)#x, levenshtein(cast(utf8_lcase#x as string), collate(AAa, utf8_binary), Some(4)) AS levenshtein(utf8_lcase, collate(AAa, utf8_binary), 4)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select is_valid_utf8(utf8_binary), is_valid_utf8(utf8_lcase) from t1
+-- !query analysis
+Project [is_valid_utf8(utf8_binary#x) AS is_valid_utf8(utf8_binary)#x, is_valid_utf8(utf8_lcase#x) AS is_valid_utf8(utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select is_valid_utf8(utf8_binary collate utf8_lcase), is_valid_utf8(utf8_lcase collate utf8_binary) from t1
+-- !query analysis
+Project [is_valid_utf8(collate(utf8_binary#x, utf8_lcase)) AS is_valid_utf8(collate(utf8_binary, utf8_lcase))#x, is_valid_utf8(collate(utf8_lcase#x, utf8_binary)) AS is_valid_utf8(collate(utf8_lcase, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select is_valid_utf8(utf8_binary collate utf8_lcase_rtrim), is_valid_utf8(utf8_lcase collate utf8_binary_rtrim) from t1
+-- !query analysis
+Project [is_valid_utf8(collate(utf8_binary#x, utf8_lcase_rtrim)) AS is_valid_utf8(collate(utf8_binary, utf8_lcase_rtrim))#x, is_valid_utf8(collate(utf8_lcase#x, utf8_binary_rtrim)) AS is_valid_utf8(collate(utf8_lcase, utf8_binary_rtrim))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select make_valid_utf8(utf8_binary), make_valid_utf8(utf8_lcase) from t1
+-- !query analysis
+Project [make_valid_utf8(utf8_binary#x) AS make_valid_utf8(utf8_binary)#x, make_valid_utf8(utf8_lcase#x) AS make_valid_utf8(utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select make_valid_utf8(utf8_binary collate utf8_lcase), make_valid_utf8(utf8_lcase collate utf8_binary) from t1
+-- !query analysis
+Project [make_valid_utf8(collate(utf8_binary#x, utf8_lcase)) AS make_valid_utf8(collate(utf8_binary, utf8_lcase))#x, make_valid_utf8(collate(utf8_lcase#x, utf8_binary)) AS make_valid_utf8(collate(utf8_lcase, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select make_valid_utf8(utf8_binary collate utf8_lcase_rtrim), make_valid_utf8(utf8_lcase collate utf8_binary_rtrim) from t1
+-- !query analysis
+Project [make_valid_utf8(collate(utf8_binary#x, utf8_lcase_rtrim)) AS make_valid_utf8(collate(utf8_binary, utf8_lcase_rtrim))#x, make_valid_utf8(collate(utf8_lcase#x, utf8_binary_rtrim)) AS make_valid_utf8(collate(utf8_lcase, utf8_binary_rtrim))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select validate_utf8(utf8_binary), validate_utf8(utf8_lcase) from t1
+-- !query analysis
+Project [validate_utf8(utf8_binary#x) AS validate_utf8(utf8_binary)#x, validate_utf8(utf8_lcase#x) AS validate_utf8(utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select validate_utf8(utf8_binary collate utf8_lcase), validate_utf8(utf8_lcase collate utf8_binary) from t1
+-- !query analysis
+Project [validate_utf8(collate(utf8_binary#x, utf8_lcase)) AS validate_utf8(collate(utf8_binary, utf8_lcase))#x, validate_utf8(collate(utf8_lcase#x, utf8_binary)) AS validate_utf8(collate(utf8_lcase, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select validate_utf8(utf8_binary collate utf8_lcase_rtrim), validate_utf8(utf8_lcase collate utf8_binary_rtrim) from t1
+-- !query analysis
+Project [validate_utf8(collate(utf8_binary#x, utf8_lcase_rtrim)) AS validate_utf8(collate(utf8_binary, utf8_lcase_rtrim))#x, validate_utf8(collate(utf8_lcase#x, utf8_binary_rtrim)) AS validate_utf8(collate(utf8_lcase, utf8_binary_rtrim))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select try_validate_utf8(utf8_binary), try_validate_utf8(utf8_lcase) from t1
+-- !query analysis
+Project [try_validate_utf8(utf8_binary#x) AS try_validate_utf8(utf8_binary)#x, try_validate_utf8(utf8_lcase#x) AS try_validate_utf8(utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select try_validate_utf8(utf8_binary collate utf8_lcase), try_validate_utf8(utf8_lcase collate utf8_binary) from t1
+-- !query analysis
+Project [try_validate_utf8(collate(utf8_binary#x, utf8_lcase)) AS try_validate_utf8(collate(utf8_binary, utf8_lcase))#x, try_validate_utf8(collate(utf8_lcase#x, utf8_binary)) AS try_validate_utf8(collate(utf8_lcase, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select try_validate_utf8(utf8_binary collate utf8_lcase_rtrim), try_validate_utf8(utf8_lcase collate utf8_binary_rtrim) from t1
+-- !query analysis
+Project [try_validate_utf8(collate(utf8_binary#x, utf8_lcase_rtrim)) AS try_validate_utf8(collate(utf8_binary, utf8_lcase_rtrim))#x, try_validate_utf8(collate(utf8_lcase#x, utf8_binary_rtrim)) AS try_validate_utf8(collate(utf8_lcase, utf8_binary_rtrim))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+drop table t1
+-- !query analysis
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t1
+
+
+-- !query
+drop table t2
+-- !query analysis
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t2
+
+
+-- !query
+drop table t3
+-- !query analysis
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t3
+
+
+-- !query
+drop table t4
+-- !query analysis
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t4
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/collations-padding-trim.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/collations-padding-trim.sql.out
new file mode 100644
index 0000000000000..626c4e57e14bd
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/collations-padding-trim.sql.out
@@ -0,0 +1,690 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+create table t1(s string, utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet
+-- !query analysis
+CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false
+
+
+-- !query
+insert into t1 values ('Spark', 'Spark', 'SQL')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaAAaA')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaaAaA')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaaAaAaaAaaAaAaaAaaAaA')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('bbAbaAbA', 'bbAbAAbA', 'a')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('İo', 'İo', 'İo')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('İo', 'İo', 'İo ')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('İo', 'İo ', 'İo')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('İo', 'İo', 'i̇o')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('efd2', 'efd2', 'efd2')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('Hello, world! Nice day.', 'Hello, world! Nice day.', 'Hello, world! Nice day.')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('Something else. Nothing here.', 'Something else. Nothing here.', 'Something else. Nothing here.')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('kitten', 'kitten', 'sitTing')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('abc', 'abc', 'abc')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('abcdcba', 'abcdcba', 'aBcDCbA')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+select rpad(utf8_binary, 8, utf8_lcase) from t1
+-- !query analysis
+Project [rpad(cast(utf8_binary#x as string collate null), 8, cast(utf8_lcase#x as string collate null)) AS rpad(utf8_binary, 8, utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select rpad(s, 8, utf8_binary) from t1
+-- !query analysis
+Project [rpad(s#x, 8, utf8_binary#x) AS rpad(s, 8, utf8_binary)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select rpad(utf8_binary collate utf8_binary, 8, s collate utf8_lcase) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
+ "sqlState" : "42P21",
+ "messageParameters" : {
+ "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
+ }
+}
+
+
+-- !query
+select rpad(utf8_binary, 8, utf8_lcase collate utf8_binary) from t1
+-- !query analysis
+Project [rpad(utf8_binary#x, 8, collate(utf8_lcase#x, utf8_binary)) AS rpad(utf8_binary, 8, collate(utf8_lcase, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select rpad(utf8_binary collate utf8_lcase, 8, utf8_lcase collate utf8_lcase) from t1
+-- !query analysis
+Project [rpad(collate(utf8_binary#x, utf8_lcase), 8, collate(utf8_lcase#x, utf8_lcase)) AS rpad(collate(utf8_binary, utf8_lcase), 8, collate(utf8_lcase, utf8_lcase))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select lpad(utf8_binary collate utf8_binary_rtrim, 8, utf8_lcase collate utf8_binary_rtrim) from t1
+-- !query analysis
+Project [lpad(collate(utf8_binary#x, utf8_binary_rtrim), 8, collate(utf8_lcase#x, utf8_binary_rtrim)) AS lpad(collate(utf8_binary, utf8_binary_rtrim), 8, collate(utf8_lcase, utf8_binary_rtrim))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select rpad(utf8_binary, 8, 'a'), rpad(utf8_lcase, 8, 'a') from t1
+-- !query analysis
+Project [rpad(utf8_binary#x, 8, a) AS rpad(utf8_binary, 8, a)#x, rpad(utf8_lcase#x, 8, a) AS rpad(utf8_lcase, 8, 'a' collate UTF8_LCASE)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select rpad(utf8_binary, 8, 'AaAA' collate utf8_lcase), rpad(utf8_lcase, 8, 'AAa' collate utf8_binary) from t1
+-- !query analysis
+Project [rpad(cast(utf8_binary#x as string collate UTF8_LCASE), 8, collate(AaAA, utf8_lcase)) AS rpad(utf8_binary, 8, collate(AaAA, utf8_lcase))#x, rpad(cast(utf8_lcase#x as string), 8, collate(AAa, utf8_binary)) AS rpad(utf8_lcase, 8, collate(AAa, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select lpad(utf8_binary, 8, utf8_lcase) from t1
+-- !query analysis
+Project [lpad(cast(utf8_binary#x as string collate null), 8, cast(utf8_lcase#x as string collate null)) AS lpad(utf8_binary, 8, utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select lpad(s, 8, utf8_binary) from t1
+-- !query analysis
+Project [lpad(s#x, 8, utf8_binary#x) AS lpad(s, 8, utf8_binary)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select lpad(utf8_binary collate utf8_binary, 8, s collate utf8_lcase) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
+ "sqlState" : "42P21",
+ "messageParameters" : {
+ "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
+ }
+}
+
+
+-- !query
+select lpad(utf8_binary, 8, utf8_lcase collate utf8_binary) from t1
+-- !query analysis
+Project [lpad(utf8_binary#x, 8, collate(utf8_lcase#x, utf8_binary)) AS lpad(utf8_binary, 8, collate(utf8_lcase, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select lpad(utf8_binary collate utf8_lcase, 8, utf8_lcase collate utf8_lcase) from t1
+-- !query analysis
+Project [lpad(collate(utf8_binary#x, utf8_lcase), 8, collate(utf8_lcase#x, utf8_lcase)) AS lpad(collate(utf8_binary, utf8_lcase), 8, collate(utf8_lcase, utf8_lcase))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select lpad(utf8_binary collate utf8_binary_rtrim, 8, utf8_lcase collate utf8_binary_rtrim) from t1
+-- !query analysis
+Project [lpad(collate(utf8_binary#x, utf8_binary_rtrim), 8, collate(utf8_lcase#x, utf8_binary_rtrim)) AS lpad(collate(utf8_binary, utf8_binary_rtrim), 8, collate(utf8_lcase, utf8_binary_rtrim))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select lpad(utf8_binary, 8, 'a'), lpad(utf8_lcase, 8, 'a') from t1
+-- !query analysis
+Project [lpad(utf8_binary#x, 8, a) AS lpad(utf8_binary, 8, a)#x, lpad(utf8_lcase#x, 8, a) AS lpad(utf8_lcase, 8, 'a' collate UTF8_LCASE)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select lpad(utf8_binary, 8, 'AaAA' collate utf8_lcase), lpad(utf8_lcase, 8, 'AAa' collate utf8_binary) from t1
+-- !query analysis
+Project [lpad(cast(utf8_binary#x as string collate UTF8_LCASE), 8, collate(AaAA, utf8_lcase)) AS lpad(utf8_binary, 8, collate(AaAA, utf8_lcase))#x, lpad(cast(utf8_lcase#x as string), 8, collate(AAa, utf8_binary)) AS lpad(utf8_lcase, 8, collate(AAa, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select TRIM(utf8_binary, utf8_lcase) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION",
+ "sqlState" : "42P22",
+ "messageParameters" : {
+ "expr" : "\"TRIM(BOTH utf8_binary FROM utf8_lcase)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 36,
+ "fragment" : "TRIM(utf8_binary, utf8_lcase)"
+ } ]
+}
+
+
+-- !query
+select TRIM(s, utf8_binary) from t1
+-- !query analysis
+Project [trim(utf8_binary#x, Some(s#x)) AS TRIM(BOTH s FROM utf8_binary)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select TRIM(utf8_binary collate utf8_binary, s collate utf8_lcase) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
+ "sqlState" : "42P21",
+ "messageParameters" : {
+ "explicitTypes" : "\"STRING COLLATE UTF8_LCASE\", \"STRING\""
+ }
+}
+
+
+-- !query
+select TRIM(utf8_binary, utf8_lcase collate utf8_binary) from t1
+-- !query analysis
+Project [trim(collate(utf8_lcase#x, utf8_binary), Some(utf8_binary#x)) AS TRIM(BOTH utf8_binary FROM collate(utf8_lcase, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select TRIM(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t1
+-- !query analysis
+Project [trim(collate(utf8_lcase#x, utf8_lcase), Some(collate(utf8_binary#x, utf8_lcase))) AS TRIM(BOTH collate(utf8_binary, utf8_lcase) FROM collate(utf8_lcase, utf8_lcase))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select TRIM(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t1
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"collate(utf8_lcase, unicode_ai)\"",
+ "inputType" : "\"STRING COLLATE UNICODE_AI\"",
+ "paramIndex" : "first",
+ "requiredType" : "\"STRING\"",
+ "sqlExpr" : "\"TRIM(BOTH collate(utf8_binary, unicode_ai) FROM collate(utf8_lcase, unicode_ai))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 74,
+ "fragment" : "TRIM(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai)"
+ } ]
+}
+
+
+-- !query
+select TRIM(utf8_binary collate utf8_binary_rtrim, utf8_lcase collate utf8_binary_rtrim) from t1
+-- !query analysis
+Project [trim(collate(utf8_lcase#x, utf8_binary_rtrim), Some(collate(utf8_binary#x, utf8_binary_rtrim))) AS TRIM(BOTH collate(utf8_binary, utf8_binary_rtrim) FROM collate(utf8_lcase, utf8_binary_rtrim))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select TRIM('ABc', utf8_binary), TRIM('ABc', utf8_lcase) from t1
+-- !query analysis
+Project [trim(utf8_binary#x, Some(ABc)) AS TRIM(BOTH ABc FROM utf8_binary)#x, trim(utf8_lcase#x, Some(ABc)) AS TRIM(BOTH 'ABc' collate UTF8_LCASE FROM utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select TRIM('ABc' collate utf8_lcase, utf8_binary), TRIM('AAa' collate utf8_binary, utf8_lcase) from t1
+-- !query analysis
+Project [trim(cast(utf8_binary#x as string collate UTF8_LCASE), Some(collate(ABc, utf8_lcase))) AS TRIM(BOTH collate(ABc, utf8_lcase) FROM utf8_binary)#x, trim(cast(utf8_lcase#x as string), Some(collate(AAa, utf8_binary))) AS TRIM(BOTH collate(AAa, utf8_binary) FROM utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select BTRIM(utf8_binary, utf8_lcase) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION",
+ "sqlState" : "42P22",
+ "messageParameters" : {
+ "expr" : "\"TRIM(BOTH utf8_lcase FROM utf8_binary)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 37,
+ "fragment" : "BTRIM(utf8_binary, utf8_lcase)"
+ } ]
+}
+
+
+-- !query
+select BTRIM(s, utf8_binary) from t1
+-- !query analysis
+Project [btrim(s#x, utf8_binary#x) AS btrim(s, utf8_binary)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select BTRIM(utf8_binary collate utf8_binary, s collate utf8_lcase) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
+ "sqlState" : "42P21",
+ "messageParameters" : {
+ "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
+ }
+}
+
+
+-- !query
+select BTRIM(utf8_binary, utf8_lcase collate utf8_binary) from t1
+-- !query analysis
+Project [btrim(utf8_binary#x, collate(utf8_lcase#x, utf8_binary)) AS btrim(utf8_binary, collate(utf8_lcase, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select BTRIM(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t1
+-- !query analysis
+Project [btrim(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase)) AS btrim(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select BTRIM(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t1
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"collate(utf8_binary, unicode_ai)\"",
+ "inputType" : "\"STRING COLLATE UNICODE_AI\"",
+ "paramIndex" : "first",
+ "requiredType" : "\"STRING\"",
+ "sqlExpr" : "\"TRIM(BOTH collate(utf8_lcase, unicode_ai) FROM collate(utf8_binary, unicode_ai))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 75,
+ "fragment" : "BTRIM(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai)"
+ } ]
+}
+
+
+-- !query
+select BTRIM(utf8_binary collate utf8_binary_rtrim, utf8_lcase collate utf8_binary_rtrim) from t1
+-- !query analysis
+Project [btrim(collate(utf8_binary#x, utf8_binary_rtrim), collate(utf8_lcase#x, utf8_binary_rtrim)) AS btrim(collate(utf8_binary, utf8_binary_rtrim), collate(utf8_lcase, utf8_binary_rtrim))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select BTRIM('ABc', utf8_binary), BTRIM('ABc', utf8_lcase) from t1
+-- !query analysis
+Project [btrim(ABc, utf8_binary#x) AS btrim(ABc, utf8_binary)#x, btrim(ABc, utf8_lcase#x) AS btrim(ABc, utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select BTRIM('ABc' collate utf8_lcase, utf8_binary), BTRIM('AAa' collate utf8_binary, utf8_lcase) from t1
+-- !query analysis
+Project [btrim(collate(ABc, utf8_lcase), utf8_binary#x) AS btrim(collate(ABc, utf8_lcase), utf8_binary)#x, btrim(collate(AAa, utf8_binary), utf8_lcase#x) AS btrim(collate(AAa, utf8_binary), utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select LTRIM(utf8_binary, utf8_lcase) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION",
+ "sqlState" : "42P22",
+ "messageParameters" : {
+ "expr" : "\"TRIM(LEADING utf8_binary FROM utf8_lcase)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 37,
+ "fragment" : "LTRIM(utf8_binary, utf8_lcase)"
+ } ]
+}
+
+
+-- !query
+select LTRIM(s, utf8_binary) from t1
+-- !query analysis
+Project [ltrim(utf8_binary#x, Some(s#x)) AS TRIM(LEADING s FROM utf8_binary)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select LTRIM(utf8_binary collate utf8_binary, s collate utf8_lcase) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
+ "sqlState" : "42P21",
+ "messageParameters" : {
+ "explicitTypes" : "\"STRING COLLATE UTF8_LCASE\", \"STRING\""
+ }
+}
+
+
+-- !query
+select LTRIM(utf8_binary, utf8_lcase collate utf8_binary) from t1
+-- !query analysis
+Project [ltrim(collate(utf8_lcase#x, utf8_binary), Some(utf8_binary#x)) AS TRIM(LEADING utf8_binary FROM collate(utf8_lcase, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select LTRIM(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t1
+-- !query analysis
+Project [ltrim(collate(utf8_lcase#x, utf8_lcase), Some(collate(utf8_binary#x, utf8_lcase))) AS TRIM(LEADING collate(utf8_binary, utf8_lcase) FROM collate(utf8_lcase, utf8_lcase))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select LTRIM(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t1
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"collate(utf8_lcase, unicode_ai)\"",
+ "inputType" : "\"STRING COLLATE UNICODE_AI\"",
+ "paramIndex" : "first",
+ "requiredType" : "\"STRING\"",
+ "sqlExpr" : "\"TRIM(LEADING collate(utf8_binary, unicode_ai) FROM collate(utf8_lcase, unicode_ai))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 75,
+ "fragment" : "LTRIM(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai)"
+ } ]
+}
+
+
+-- !query
+select LTRIM(utf8_binary collate utf8_binary_rtrim, utf8_lcase collate utf8_binary_rtrim) from t1
+-- !query analysis
+Project [ltrim(collate(utf8_lcase#x, utf8_binary_rtrim), Some(collate(utf8_binary#x, utf8_binary_rtrim))) AS TRIM(LEADING collate(utf8_binary, utf8_binary_rtrim) FROM collate(utf8_lcase, utf8_binary_rtrim))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select LTRIM('ABc', utf8_binary), LTRIM('ABc', utf8_lcase) from t1
+-- !query analysis
+Project [ltrim(utf8_binary#x, Some(ABc)) AS TRIM(LEADING ABc FROM utf8_binary)#x, ltrim(utf8_lcase#x, Some(ABc)) AS TRIM(LEADING 'ABc' collate UTF8_LCASE FROM utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select LTRIM('ABc' collate utf8_lcase, utf8_binary), LTRIM('AAa' collate utf8_binary, utf8_lcase) from t1
+-- !query analysis
+Project [ltrim(cast(utf8_binary#x as string collate UTF8_LCASE), Some(collate(ABc, utf8_lcase))) AS TRIM(LEADING collate(ABc, utf8_lcase) FROM utf8_binary)#x, ltrim(cast(utf8_lcase#x as string), Some(collate(AAa, utf8_binary))) AS TRIM(LEADING collate(AAa, utf8_binary) FROM utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select RTRIM(utf8_binary, utf8_lcase) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION",
+ "sqlState" : "42P22",
+ "messageParameters" : {
+ "expr" : "\"TRIM(TRAILING utf8_binary FROM utf8_lcase)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 37,
+ "fragment" : "RTRIM(utf8_binary, utf8_lcase)"
+ } ]
+}
+
+
+-- !query
+select RTRIM(s, utf8_binary) from t1
+-- !query analysis
+Project [rtrim(utf8_binary#x, Some(s#x)) AS TRIM(TRAILING s FROM utf8_binary)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select RTRIM(utf8_binary collate utf8_binary, s collate utf8_lcase) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
+ "sqlState" : "42P21",
+ "messageParameters" : {
+ "explicitTypes" : "\"STRING COLLATE UTF8_LCASE\", \"STRING\""
+ }
+}
+
+
+-- !query
+select RTRIM(utf8_binary, utf8_lcase collate utf8_binary) from t1
+-- !query analysis
+Project [rtrim(collate(utf8_lcase#x, utf8_binary), Some(utf8_binary#x)) AS TRIM(TRAILING utf8_binary FROM collate(utf8_lcase, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select RTRIM(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t1
+-- !query analysis
+Project [rtrim(collate(utf8_lcase#x, utf8_lcase), Some(collate(utf8_binary#x, utf8_lcase))) AS TRIM(TRAILING collate(utf8_binary, utf8_lcase) FROM collate(utf8_lcase, utf8_lcase))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select RTRIM(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t1
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"collate(utf8_lcase, unicode_ai)\"",
+ "inputType" : "\"STRING COLLATE UNICODE_AI\"",
+ "paramIndex" : "first",
+ "requiredType" : "\"STRING\"",
+ "sqlExpr" : "\"TRIM(TRAILING collate(utf8_binary, unicode_ai) FROM collate(utf8_lcase, unicode_ai))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 75,
+ "fragment" : "RTRIM(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai)"
+ } ]
+}
+
+
+-- !query
+select RTRIM(utf8_binary collate utf8_binary_rtrim, utf8_lcase collate utf8_binary_rtrim) from t1
+-- !query analysis
+Project [rtrim(collate(utf8_lcase#x, utf8_binary_rtrim), Some(collate(utf8_binary#x, utf8_binary_rtrim))) AS TRIM(TRAILING collate(utf8_binary, utf8_binary_rtrim) FROM collate(utf8_lcase, utf8_binary_rtrim))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select RTRIM('ABc', utf8_binary), RTRIM('ABc', utf8_lcase) from t1
+-- !query analysis
+Project [rtrim(utf8_binary#x, Some(ABc)) AS TRIM(TRAILING ABc FROM utf8_binary)#x, rtrim(utf8_lcase#x, Some(ABc)) AS TRIM(TRAILING 'ABc' collate UTF8_LCASE FROM utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select RTRIM('ABc' collate utf8_lcase, utf8_binary), RTRIM('AAa' collate utf8_binary, utf8_lcase) from t1
+-- !query analysis
+Project [rtrim(cast(utf8_binary#x as string collate UTF8_LCASE), Some(collate(ABc, utf8_lcase))) AS TRIM(TRAILING collate(ABc, utf8_lcase) FROM utf8_binary)#x, rtrim(cast(utf8_lcase#x as string), Some(collate(AAa, utf8_binary))) AS TRIM(TRAILING collate(AAa, utf8_binary) FROM utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+drop table t1
+-- !query analysis
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t1
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/collations-string-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/collations-string-functions.sql.out
new file mode 100644
index 0000000000000..f7d2d595a5817
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/collations-string-functions.sql.out
@@ -0,0 +1,1510 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+create table t1(s string, utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet
+-- !query analysis
+CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false
+
+
+-- !query
+insert into t1 values ('Spark', 'Spark', 'SQL')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaAAaA')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaaAaA')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaaAaAaaAaaAaAaaAaaAaA')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('bbAbaAbA', 'bbAbAAbA', 'a')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('İo', 'İo', 'İo')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('İo', 'İo', 'İo ')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('İo', 'İo ', 'İo')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('İo', 'İo', 'i̇o')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('efd2', 'efd2', 'efd2')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('Hello, world! Nice day.', 'Hello, world! Nice day.', 'Hello, world! Nice day.')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('Something else. Nothing here.', 'Something else. Nothing here.', 'Something else. Nothing here.')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('kitten', 'kitten', 'sitTing')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('abc', 'abc', 'abc')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+insert into t1 values ('abcdcba', 'abcdcba', 'aBcDCbA')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [s, utf8_binary, utf8_lcase]
++- Project [col1#x AS s#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+create table t2(ascii long) using parquet
+-- !query analysis
+CreateDataSourceTableCommand `spark_catalog`.`default`.`t2`, false
+
+
+-- !query
+insert into t2 values (97)
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t2], Append, `spark_catalog`.`default`.`t2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t2), [ascii]
++- Project [cast(col1#x as bigint) AS ascii#xL]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+insert into t2 values (66)
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t2], Append, `spark_catalog`.`default`.`t2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t2), [ascii]
++- Project [cast(col1#x as bigint) AS ascii#xL]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+create table t3(format string collate utf8_binary, utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet
+-- !query analysis
+CreateDataSourceTableCommand `spark_catalog`.`default`.`t3`, false
+
+
+-- !query
+insert into t3 values ('%s%s', 'abCdE', 'abCdE')
+-- !query analysis
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t3, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t3], Append, `spark_catalog`.`default`.`t3`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t3), [format, utf8_binary, utf8_lcase]
++- Project [col1#x AS format#x, col2#x AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+select concat_ws(' ', utf8_lcase, utf8_lcase) from t1
+-- !query analysis
+Project [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select concat_ws(' ', utf8_binary, utf8_lcase) from t1
+-- !query analysis
+Project [concat_ws( , cast(utf8_binary#x as string collate null), cast(utf8_lcase#x as string collate null)) AS concat_ws(' ' collate null, utf8_binary, utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select concat_ws(' ' collate utf8_binary, utf8_binary, 'SQL' collate utf8_lcase) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
+ "sqlState" : "42P21",
+ "messageParameters" : {
+ "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
+ }
+}
+
+
+-- !query
+select concat_ws(' ' collate utf8_lcase, utf8_binary, 'SQL' collate utf8_lcase) from t1
+-- !query analysis
+Project [concat_ws(collate( , utf8_lcase), cast(utf8_binary#x as string collate UTF8_LCASE), collate(SQL, utf8_lcase)) AS concat_ws(collate( , utf8_lcase), utf8_binary, collate(SQL, utf8_lcase))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select concat_ws(',', utf8_lcase, 'word'), concat_ws(',', utf8_binary, 'word') from t1
+-- !query analysis
+Project [concat_ws(,, utf8_lcase#x, word) AS concat_ws(',' collate UTF8_LCASE, utf8_lcase, 'word' collate UTF8_LCASE)#x, concat_ws(,, utf8_binary#x, word) AS concat_ws(,, utf8_binary, word)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select concat_ws(',', utf8_lcase, 'word' collate utf8_binary), concat_ws(',', utf8_binary, 'word' collate utf8_lcase) from t1
+-- !query analysis
+Project [concat_ws(,, cast(utf8_lcase#x as string), collate(word, utf8_binary)) AS concat_ws(,, utf8_lcase, collate(word, utf8_binary))#x, concat_ws(,, cast(utf8_binary#x as string collate UTF8_LCASE), collate(word, utf8_lcase)) AS concat_ws(',' collate UTF8_LCASE, utf8_binary, collate(word, utf8_lcase))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select split_part(utf8_binary, utf8_lcase, 3) from t1
+-- !query analysis
+Project [split_part(cast(utf8_binary#x as string collate null), cast(utf8_lcase#x as string collate null), 3) AS split_part(utf8_binary, utf8_lcase, 3)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select split_part(s, utf8_binary, 1) from t1
+-- !query analysis
+Project [split_part(s#x, utf8_binary#x, 1) AS split_part(s, utf8_binary, 1)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select split_part(utf8_binary collate utf8_binary, s collate utf8_lcase, 1) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
+ "sqlState" : "42P21",
+ "messageParameters" : {
+ "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
+ }
+}
+
+
+-- !query
+select split_part(utf8_binary, utf8_lcase collate utf8_binary, 2) from t1
+-- !query analysis
+Project [split_part(utf8_binary#x, collate(utf8_lcase#x, utf8_binary), 2) AS split_part(utf8_binary, collate(utf8_lcase, utf8_binary), 2)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select split_part(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 2) from t1
+-- !query analysis
+Project [split_part(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase), 2) AS split_part(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase), 2)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select split_part(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai, 2) from t1
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"collate(utf8_binary, unicode_ai)\"",
+ "inputType" : "\"STRING COLLATE UNICODE_AI\"",
+ "paramIndex" : "first",
+ "requiredType" : "\"STRING\"",
+ "sqlExpr" : "\"split_part(collate(utf8_binary, unicode_ai), collate(utf8_lcase, unicode_ai), 2)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 83,
+ "fragment" : "split_part(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai, 2)"
+ } ]
+}
+
+
+-- !query
+select split_part(utf8_binary, 'a', 3), split_part(utf8_lcase, 'a', 3) from t1
+-- !query analysis
+Project [split_part(utf8_binary#x, a, 3) AS split_part(utf8_binary, a, 3)#x, split_part(utf8_lcase#x, a, 3) AS split_part(utf8_lcase, 'a' collate UTF8_LCASE, 3)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select split_part(utf8_binary, 'a' collate utf8_lcase, 3), split_part(utf8_lcase, 'a' collate utf8_binary, 3) from t1
+-- !query analysis
+Project [split_part(cast(utf8_binary#x as string collate UTF8_LCASE), collate(a, utf8_lcase), 3) AS split_part(utf8_binary, collate(a, utf8_lcase), 3)#x, split_part(cast(utf8_lcase#x as string), collate(a, utf8_binary), 3) AS split_part(utf8_lcase, collate(a, utf8_binary), 3)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select split_part(utf8_binary, 'a ' collate utf8_lcase_rtrim, 3), split_part(utf8_lcase, 'a' collate utf8_binary, 3) from t1
+-- !query analysis
+Project [split_part(cast(utf8_binary#x as string collate UTF8_LCASE_RTRIM), collate(a , utf8_lcase_rtrim), 3) AS split_part(utf8_binary, collate(a , utf8_lcase_rtrim), 3)#x, split_part(cast(utf8_lcase#x as string), collate(a, utf8_binary), 3) AS split_part(utf8_lcase, collate(a, utf8_binary), 3)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select contains(utf8_binary, utf8_lcase) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION",
+ "sqlState" : "42P22",
+ "messageParameters" : {
+ "expr" : "\"contains(utf8_binary, utf8_lcase)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 40,
+ "fragment" : "contains(utf8_binary, utf8_lcase)"
+ } ]
+}
+
+
+-- !query
+select contains(s, utf8_binary) from t1
+-- !query analysis
+Project [Contains(s#x, utf8_binary#x) AS contains(s, utf8_binary)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select contains(utf8_binary collate utf8_binary, s collate utf8_lcase) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
+ "sqlState" : "42P21",
+ "messageParameters" : {
+ "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
+ }
+}
+
+
+-- !query
+select contains(utf8_binary, utf8_lcase collate utf8_binary) from t1
+-- !query analysis
+Project [Contains(utf8_binary#x, collate(utf8_lcase#x, utf8_binary)) AS contains(utf8_binary, collate(utf8_lcase, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select contains(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t1
+-- !query analysis
+Project [Contains(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase)) AS contains(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select contains(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t1
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"collate(utf8_binary, unicode_ai)\"",
+ "inputType" : "\"STRING COLLATE UNICODE_AI\"",
+ "paramIndex" : "first",
+ "requiredType" : "\"STRING\"",
+ "sqlExpr" : "\"contains(collate(utf8_binary, unicode_ai), collate(utf8_lcase, unicode_ai))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 78,
+ "fragment" : "contains(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai)"
+ } ]
+}
+
+
+-- !query
+select contains(utf8_binary, 'a'), contains(utf8_lcase, 'a') from t1
+-- !query analysis
+Project [Contains(utf8_binary#x, a) AS contains(utf8_binary, a)#x, Contains(utf8_lcase#x, a) AS contains(utf8_lcase, 'a' collate UTF8_LCASE)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select contains(utf8_binary, 'AaAA' collate utf8_lcase), contains(utf8_lcase, 'AAa' collate utf8_binary) from t1
+-- !query analysis
+Project [Contains(cast(utf8_binary#x as string collate UTF8_LCASE), collate(AaAA, utf8_lcase)) AS contains(utf8_binary, collate(AaAA, utf8_lcase))#x, Contains(cast(utf8_lcase#x as string), collate(AAa, utf8_binary)) AS contains(utf8_lcase, collate(AAa, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select contains(utf8_binary, 'AaAA ' collate utf8_lcase_rtrim), contains(utf8_lcase, 'AAa ' collate utf8_binary_rtrim) from t1
+-- !query analysis
+Project [Contains(cast(utf8_binary#x as string collate UTF8_LCASE_RTRIM), collate(AaAA , utf8_lcase_rtrim)) AS contains(utf8_binary, collate(AaAA , utf8_lcase_rtrim))#x, Contains(cast(utf8_lcase#x as string collate UTF8_BINARY_RTRIM), collate(AAa , utf8_binary_rtrim)) AS contains(utf8_lcase, collate(AAa , utf8_binary_rtrim))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select substring_index(utf8_binary, utf8_lcase, 2) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION",
+ "sqlState" : "42P22",
+ "messageParameters" : {
+ "expr" : "\"substring_index(utf8_binary, utf8_lcase, 2)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 50,
+ "fragment" : "substring_index(utf8_binary, utf8_lcase, 2)"
+ } ]
+}
+
+
+-- !query
+select substring_index(s, utf8_binary,1) from t1
+-- !query analysis
+Project [substring_index(s#x, utf8_binary#x, 1) AS substring_index(s, utf8_binary, 1)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select substring_index(utf8_binary collate utf8_binary, s collate utf8_lcase, 3) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
+ "sqlState" : "42P21",
+ "messageParameters" : {
+ "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
+ }
+}
+
+
+-- !query
+select substring_index(utf8_binary, utf8_lcase collate utf8_binary, 2) from t1
+-- !query analysis
+Project [substring_index(utf8_binary#x, collate(utf8_lcase#x, utf8_binary), 2) AS substring_index(utf8_binary, collate(utf8_lcase, utf8_binary), 2)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select substring_index(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 2) from t1
+-- !query analysis
+Project [substring_index(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase), 2) AS substring_index(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase), 2)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select substring_index(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai, 2) from t1
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"collate(utf8_binary, unicode_ai)\"",
+ "inputType" : "\"STRING COLLATE UNICODE_AI\"",
+ "paramIndex" : "first",
+ "requiredType" : "\"STRING\"",
+ "sqlExpr" : "\"substring_index(collate(utf8_binary, unicode_ai), collate(utf8_lcase, unicode_ai), 2)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 88,
+ "fragment" : "substring_index(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai, 2)"
+ } ]
+}
+
+
+-- !query
+select substring_index(utf8_binary, 'a', 2), substring_index(utf8_lcase, 'a', 2) from t1
+-- !query analysis
+Project [substring_index(utf8_binary#x, a, 2) AS substring_index(utf8_binary, a, 2)#x, substring_index(utf8_lcase#x, a, 2) AS substring_index(utf8_lcase, 'a' collate UTF8_LCASE, 2)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select substring_index(utf8_binary, 'AaAA' collate utf8_lcase, 2), substring_index(utf8_lcase, 'AAa' collate utf8_binary, 2) from t1
+-- !query analysis
+Project [substring_index(cast(utf8_binary#x as string collate UTF8_LCASE), collate(AaAA, utf8_lcase), 2) AS substring_index(utf8_binary, collate(AaAA, utf8_lcase), 2)#x, substring_index(cast(utf8_lcase#x as string), collate(AAa, utf8_binary), 2) AS substring_index(utf8_lcase, collate(AAa, utf8_binary), 2)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select substring_index(utf8_binary, 'AaAA ' collate utf8_lcase_rtrim, 2), substring_index(utf8_lcase, 'AAa' collate utf8_binary, 2) from t1
+-- !query analysis
+Project [substring_index(cast(utf8_binary#x as string collate UTF8_LCASE_RTRIM), collate(AaAA , utf8_lcase_rtrim), 2) AS substring_index(utf8_binary, collate(AaAA , utf8_lcase_rtrim), 2)#x, substring_index(cast(utf8_lcase#x as string), collate(AAa, utf8_binary), 2) AS substring_index(utf8_lcase, collate(AAa, utf8_binary), 2)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select instr(utf8_binary, utf8_lcase) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION",
+ "sqlState" : "42P22",
+ "messageParameters" : {
+ "expr" : "\"instr(utf8_binary, utf8_lcase)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 37,
+ "fragment" : "instr(utf8_binary, utf8_lcase)"
+ } ]
+}
+
+
+-- !query
+select instr(s, utf8_binary) from t1
+-- !query analysis
+Project [instr(s#x, utf8_binary#x) AS instr(s, utf8_binary)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select instr(utf8_binary collate utf8_binary, s collate utf8_lcase) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
+ "sqlState" : "42P21",
+ "messageParameters" : {
+ "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
+ }
+}
+
+
+-- !query
+select instr(utf8_binary, utf8_lcase collate utf8_binary) from t1
+-- !query analysis
+Project [instr(utf8_binary#x, collate(utf8_lcase#x, utf8_binary)) AS instr(utf8_binary, collate(utf8_lcase, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select instr(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t1
+-- !query analysis
+Project [instr(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase)) AS instr(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select instr(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t1
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"collate(utf8_binary, unicode_ai)\"",
+ "inputType" : "\"STRING COLLATE UNICODE_AI\"",
+ "paramIndex" : "first",
+ "requiredType" : "\"STRING\"",
+ "sqlExpr" : "\"instr(collate(utf8_binary, unicode_ai), collate(utf8_lcase, unicode_ai))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 75,
+ "fragment" : "instr(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai)"
+ } ]
+}
+
+
+-- !query
+select instr(utf8_binary, 'a'), instr(utf8_lcase, 'a') from t1
+-- !query analysis
+Project [instr(utf8_binary#x, a) AS instr(utf8_binary, a)#x, instr(utf8_lcase#x, a) AS instr(utf8_lcase, 'a' collate UTF8_LCASE)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select instr(utf8_binary, 'AaAA' collate utf8_lcase), instr(utf8_lcase, 'AAa' collate utf8_binary) from t1
+-- !query analysis
+Project [instr(cast(utf8_binary#x as string collate UTF8_LCASE), collate(AaAA, utf8_lcase)) AS instr(utf8_binary, collate(AaAA, utf8_lcase))#x, instr(cast(utf8_lcase#x as string), collate(AAa, utf8_binary)) AS instr(utf8_lcase, collate(AAa, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select find_in_set(utf8_binary, utf8_lcase) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION",
+ "sqlState" : "42P22",
+ "messageParameters" : {
+ "expr" : "\"find_in_set(utf8_binary, utf8_lcase)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 43,
+ "fragment" : "find_in_set(utf8_binary, utf8_lcase)"
+ } ]
+}
+
+
+-- !query
+select find_in_set(s, utf8_binary) from t1
+-- !query analysis
+Project [find_in_set(s#x, utf8_binary#x) AS find_in_set(s, utf8_binary)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select find_in_set(utf8_binary collate utf8_binary, s collate utf8_lcase) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
+ "sqlState" : "42P21",
+ "messageParameters" : {
+ "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
+ }
+}
+
+
+-- !query
+select find_in_set(utf8_binary, utf8_lcase collate utf8_binary) from t1
+-- !query analysis
+Project [find_in_set(utf8_binary#x, collate(utf8_lcase#x, utf8_binary)) AS find_in_set(utf8_binary, collate(utf8_lcase, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select find_in_set(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t1
+-- !query analysis
+Project [find_in_set(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase)) AS find_in_set(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select find_in_set(utf8_binary, 'aaAaaAaA,i̇o'), find_in_set(utf8_lcase, 'aaAaaAaA,i̇o') from t1
+-- !query analysis
+Project [find_in_set(utf8_binary#x, aaAaaAaA,i̇o) AS find_in_set(utf8_binary, aaAaaAaA,i̇o)#x, find_in_set(utf8_lcase#x, aaAaaAaA,i̇o) AS find_in_set(utf8_lcase, 'aaAaaAaA,i̇o' collate UTF8_LCASE)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select find_in_set(utf8_binary, 'aaAaaAaA,i̇o' collate utf8_lcase), find_in_set(utf8_lcase, 'aaAaaAaA,i̇o' collate utf8_binary) from t1
+-- !query analysis
+Project [find_in_set(cast(utf8_binary#x as string collate UTF8_LCASE), collate(aaAaaAaA,i̇o, utf8_lcase)) AS find_in_set(utf8_binary, collate(aaAaaAaA,i̇o, utf8_lcase))#x, find_in_set(cast(utf8_lcase#x as string), collate(aaAaaAaA,i̇o, utf8_binary)) AS find_in_set(utf8_lcase, collate(aaAaaAaA,i̇o, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select find_in_set(utf8_binary, 'aaAaaAaA,i̇o ' collate utf8_lcase_rtrim), find_in_set(utf8_lcase, 'aaAaaAaA,i̇o' collate utf8_binary) from t1
+-- !query analysis
+Project [find_in_set(cast(utf8_binary#x as string collate UTF8_LCASE_RTRIM), collate(aaAaaAaA,i̇o , utf8_lcase_rtrim)) AS find_in_set(utf8_binary, collate(aaAaaAaA,i̇o , utf8_lcase_rtrim))#x, find_in_set(cast(utf8_lcase#x as string), collate(aaAaaAaA,i̇o, utf8_binary)) AS find_in_set(utf8_lcase, collate(aaAaaAaA,i̇o, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select startswith(utf8_binary, utf8_lcase) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION",
+ "sqlState" : "42P22",
+ "messageParameters" : {
+ "expr" : "\"startswith(utf8_binary, utf8_lcase)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 42,
+ "fragment" : "startswith(utf8_binary, utf8_lcase)"
+ } ]
+}
+
+
+-- !query
+select startswith(s, utf8_binary) from t1
+-- !query analysis
+Project [StartsWith(s#x, utf8_binary#x) AS startswith(s, utf8_binary)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select startswith(utf8_binary collate utf8_binary, s collate utf8_lcase) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
+ "sqlState" : "42P21",
+ "messageParameters" : {
+ "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
+ }
+}
+
+
+-- !query
+select startswith(utf8_binary, utf8_lcase collate utf8_binary) from t1
+-- !query analysis
+Project [StartsWith(utf8_binary#x, collate(utf8_lcase#x, utf8_binary)) AS startswith(utf8_binary, collate(utf8_lcase, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select startswith(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t1
+-- !query analysis
+Project [StartsWith(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase)) AS startswith(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select startswith(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t1
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"collate(utf8_binary, unicode_ai)\"",
+ "inputType" : "\"STRING COLLATE UNICODE_AI\"",
+ "paramIndex" : "first",
+ "requiredType" : "\"STRING\"",
+ "sqlExpr" : "\"startswith(collate(utf8_binary, unicode_ai), collate(utf8_lcase, unicode_ai))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 80,
+ "fragment" : "startswith(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai)"
+ } ]
+}
+
+
+-- !query
+select startswith(utf8_binary, 'aaAaaAaA'), startswith(utf8_lcase, 'aaAaaAaA') from t1
+-- !query analysis
+Project [StartsWith(utf8_binary#x, aaAaaAaA) AS startswith(utf8_binary, aaAaaAaA)#x, StartsWith(utf8_lcase#x, aaAaaAaA) AS startswith(utf8_lcase, 'aaAaaAaA' collate UTF8_LCASE)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select startswith(utf8_binary, 'aaAaaAaA' collate utf8_lcase), startswith(utf8_lcase, 'aaAaaAaA' collate utf8_binary) from t1
+-- !query analysis
+Project [StartsWith(cast(utf8_binary#x as string collate UTF8_LCASE), collate(aaAaaAaA, utf8_lcase)) AS startswith(utf8_binary, collate(aaAaaAaA, utf8_lcase))#x, StartsWith(cast(utf8_lcase#x as string), collate(aaAaaAaA, utf8_binary)) AS startswith(utf8_lcase, collate(aaAaaAaA, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select startswith(utf8_binary, 'aaAaaAaA ' collate utf8_lcase_rtrim), startswith(utf8_lcase, 'aaAaaAaA' collate utf8_binary) from t1
+-- !query analysis
+Project [StartsWith(cast(utf8_binary#x as string collate UTF8_LCASE_RTRIM), collate(aaAaaAaA , utf8_lcase_rtrim)) AS startswith(utf8_binary, collate(aaAaaAaA , utf8_lcase_rtrim))#x, StartsWith(cast(utf8_lcase#x as string), collate(aaAaaAaA, utf8_binary)) AS startswith(utf8_lcase, collate(aaAaaAaA, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select translate(utf8_lcase, utf8_lcase, '12345') from t1
+-- !query analysis
+Project [translate(utf8_lcase#x, utf8_lcase#x, 12345) AS translate(utf8_lcase, utf8_lcase, '12345' collate UTF8_LCASE)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select translate(utf8_binary, utf8_lcase, '12345') from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION",
+ "sqlState" : "42P22",
+ "messageParameters" : {
+ "expr" : "\"translate(utf8_binary, utf8_lcase, 12345)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 50,
+ "fragment" : "translate(utf8_binary, utf8_lcase, '12345')"
+ } ]
+}
+
+
+-- !query
+select translate(utf8_binary, 'aBc' collate utf8_lcase, '12345' collate utf8_binary) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
+ "sqlState" : "42P21",
+ "messageParameters" : {
+ "explicitTypes" : "\"STRING COLLATE UTF8_LCASE\", \"STRING\""
+ }
+}
+
+
+-- !query
+select translate(utf8_binary, 'SQL' collate utf8_lcase, '12345' collate utf8_lcase) from t1
+-- !query analysis
+Project [translate(cast(utf8_binary#x as string collate UTF8_LCASE), collate(SQL, utf8_lcase), collate(12345, utf8_lcase)) AS translate(utf8_binary, collate(SQL, utf8_lcase), collate(12345, utf8_lcase))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select translate(utf8_binary, 'SQL' collate unicode_ai, '12345' collate unicode_ai) from t1
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"utf8_binary\"",
+ "inputType" : "\"STRING COLLATE UNICODE_AI\"",
+ "paramIndex" : "first",
+ "requiredType" : "\"STRING\"",
+ "sqlExpr" : "\"translate(utf8_binary, collate(SQL, unicode_ai), collate(12345, unicode_ai))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 83,
+ "fragment" : "translate(utf8_binary, 'SQL' collate unicode_ai, '12345' collate unicode_ai)"
+ } ]
+}
+
+
+-- !query
+select translate(utf8_lcase, 'aaAaaAaA', '12345'), translate(utf8_binary, 'aaAaaAaA', '12345') from t1
+-- !query analysis
+Project [translate(utf8_lcase#x, aaAaaAaA, 12345) AS translate(utf8_lcase, 'aaAaaAaA' collate UTF8_LCASE, '12345' collate UTF8_LCASE)#x, translate(utf8_binary#x, aaAaaAaA, 12345) AS translate(utf8_binary, aaAaaAaA, 12345)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select translate(utf8_lcase, 'aBc' collate utf8_binary, '12345'), translate(utf8_binary, 'aBc' collate utf8_lcase, '12345') from t1
+-- !query analysis
+Project [translate(cast(utf8_lcase#x as string), collate(aBc, utf8_binary), 12345) AS translate(utf8_lcase, collate(aBc, utf8_binary), 12345)#x, translate(cast(utf8_binary#x as string collate UTF8_LCASE), collate(aBc, utf8_lcase), 12345) AS translate(utf8_binary, collate(aBc, utf8_lcase), '12345' collate UTF8_LCASE)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select translate(utf8_lcase, 'aBc ' collate utf8_binary_rtrim, '12345'), translate(utf8_binary, 'aBc' collate utf8_lcase, '12345') from t1
+-- !query analysis
+Project [translate(cast(utf8_lcase#x as string collate UTF8_BINARY_RTRIM), collate(aBc , utf8_binary_rtrim), 12345) AS translate(utf8_lcase, collate(aBc , utf8_binary_rtrim), '12345' collate UTF8_BINARY_RTRIM)#x, translate(cast(utf8_binary#x as string collate UTF8_LCASE), collate(aBc, utf8_lcase), 12345) AS translate(utf8_binary, collate(aBc, utf8_lcase), '12345' collate UTF8_LCASE)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select replace(utf8_binary, utf8_lcase, 'abc') from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION",
+ "sqlState" : "42P22",
+ "messageParameters" : {
+ "expr" : "\"replace(utf8_binary, utf8_lcase, abc)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 46,
+ "fragment" : "replace(utf8_binary, utf8_lcase, 'abc')"
+ } ]
+}
+
+
+-- !query
+select replace(s, utf8_binary, 'abc') from t1
+-- !query analysis
+Project [replace(s#x, utf8_binary#x, abc) AS replace(s, utf8_binary, abc)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select replace(utf8_binary collate utf8_binary, s collate utf8_lcase, 'abc') from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
+ "sqlState" : "42P21",
+ "messageParameters" : {
+ "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
+ }
+}
+
+
+-- !query
+select replace(utf8_binary, utf8_lcase collate utf8_binary, 'abc') from t1
+-- !query analysis
+Project [replace(utf8_binary#x, collate(utf8_lcase#x, utf8_binary), abc) AS replace(utf8_binary, collate(utf8_lcase, utf8_binary), abc)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select replace(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 'abc') from t1
+-- !query analysis
+Project [replace(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase), abc) AS replace(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase), 'abc' collate UTF8_LCASE)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select replace(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai, 'abc') from t1
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"collate(utf8_binary, unicode_ai)\"",
+ "inputType" : "\"STRING COLLATE UNICODE_AI\"",
+ "paramIndex" : "first",
+ "requiredType" : "\"STRING\"",
+ "sqlExpr" : "\"replace(collate(utf8_binary, unicode_ai), collate(utf8_lcase, unicode_ai), 'abc' collate UNICODE_AI)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 84,
+ "fragment" : "replace(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai, 'abc')"
+ } ]
+}
+
+
+-- !query
+select replace(utf8_binary, 'aaAaaAaA', 'abc'), replace(utf8_lcase, 'aaAaaAaA', 'abc') from t1
+-- !query analysis
+Project [replace(utf8_binary#x, aaAaaAaA, abc) AS replace(utf8_binary, aaAaaAaA, abc)#x, replace(utf8_lcase#x, aaAaaAaA, abc) AS replace(utf8_lcase, 'aaAaaAaA' collate UTF8_LCASE, 'abc' collate UTF8_LCASE)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select replace(utf8_binary, 'aaAaaAaA' collate utf8_lcase, 'abc'), replace(utf8_lcase, 'aaAaaAaA' collate utf8_binary, 'abc') from t1
+-- !query analysis
+Project [replace(cast(utf8_binary#x as string collate UTF8_LCASE), collate(aaAaaAaA, utf8_lcase), abc) AS replace(utf8_binary, collate(aaAaaAaA, utf8_lcase), 'abc' collate UTF8_LCASE)#x, replace(cast(utf8_lcase#x as string), collate(aaAaaAaA, utf8_binary), abc) AS replace(utf8_lcase, collate(aaAaaAaA, utf8_binary), abc)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select replace(utf8_binary, 'aaAaaAaA ' collate utf8_lcase_rtrim, 'abc'), replace(utf8_lcase, 'aaAaaAaA' collate utf8_binary, 'abc') from t1
+-- !query analysis
+Project [replace(cast(utf8_binary#x as string collate UTF8_LCASE_RTRIM), collate(aaAaaAaA , utf8_lcase_rtrim), abc) AS replace(utf8_binary, collate(aaAaaAaA , utf8_lcase_rtrim), 'abc' collate UTF8_LCASE_RTRIM)#x, replace(cast(utf8_lcase#x as string), collate(aaAaaAaA, utf8_binary), abc) AS replace(utf8_lcase, collate(aaAaaAaA, utf8_binary), abc)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select endswith(utf8_binary, utf8_lcase) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION",
+ "sqlState" : "42P22",
+ "messageParameters" : {
+ "expr" : "\"endswith(utf8_binary, utf8_lcase)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 40,
+ "fragment" : "endswith(utf8_binary, utf8_lcase)"
+ } ]
+}
+
+
+-- !query
+select endswith(s, utf8_binary) from t1
+-- !query analysis
+Project [EndsWith(s#x, utf8_binary#x) AS endswith(s, utf8_binary)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select endswith(utf8_binary collate utf8_binary, s collate utf8_lcase) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
+ "sqlState" : "42P21",
+ "messageParameters" : {
+ "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
+ }
+}
+
+
+-- !query
+select endswith(utf8_binary, utf8_lcase collate utf8_binary) from t1
+-- !query analysis
+Project [EndsWith(utf8_binary#x, collate(utf8_lcase#x, utf8_binary)) AS endswith(utf8_binary, collate(utf8_lcase, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select endswith(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t1
+-- !query analysis
+Project [EndsWith(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase)) AS endswith(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select endswith(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t1
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"collate(utf8_binary, unicode_ai)\"",
+ "inputType" : "\"STRING COLLATE UNICODE_AI\"",
+ "paramIndex" : "first",
+ "requiredType" : "\"STRING\"",
+ "sqlExpr" : "\"endswith(collate(utf8_binary, unicode_ai), collate(utf8_lcase, unicode_ai))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 78,
+ "fragment" : "endswith(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai)"
+ } ]
+}
+
+
+-- !query
+select endswith(utf8_binary, 'aaAaaAaA'), endswith(utf8_lcase, 'aaAaaAaA') from t1
+-- !query analysis
+Project [EndsWith(utf8_binary#x, aaAaaAaA) AS endswith(utf8_binary, aaAaaAaA)#x, EndsWith(utf8_lcase#x, aaAaaAaA) AS endswith(utf8_lcase, 'aaAaaAaA' collate UTF8_LCASE)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select endswith(utf8_binary, 'aaAaaAaA' collate utf8_lcase), endswith(utf8_lcase, 'aaAaaAaA' collate utf8_binary) from t1
+-- !query analysis
+Project [EndsWith(cast(utf8_binary#x as string collate UTF8_LCASE), collate(aaAaaAaA, utf8_lcase)) AS endswith(utf8_binary, collate(aaAaaAaA, utf8_lcase))#x, EndsWith(cast(utf8_lcase#x as string), collate(aaAaaAaA, utf8_binary)) AS endswith(utf8_lcase, collate(aaAaaAaA, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select endswith(utf8_binary, 'aaAaaAaA ' collate utf8_lcase_rtrim), endswith(utf8_lcase, 'aaAaaAaA' collate utf8_binary) from t1
+-- !query analysis
+Project [EndsWith(cast(utf8_binary#x as string collate UTF8_LCASE_RTRIM), collate(aaAaaAaA , utf8_lcase_rtrim)) AS endswith(utf8_binary, collate(aaAaaAaA , utf8_lcase_rtrim))#x, EndsWith(cast(utf8_lcase#x as string), collate(aaAaaAaA, utf8_binary)) AS endswith(utf8_lcase, collate(aaAaaAaA, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select repeat(utf8_binary, 3), repeat(utf8_lcase, 2) from t1
+-- !query analysis
+Project [repeat(utf8_binary#x, 3) AS repeat(utf8_binary, 3)#x, repeat(utf8_lcase#x, 2) AS repeat(utf8_lcase, 2)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select repeat(utf8_binary collate utf8_lcase, 3), repeat(utf8_lcase collate utf8_binary, 2) from t1
+-- !query analysis
+Project [repeat(collate(utf8_binary#x, utf8_lcase), 3) AS repeat(collate(utf8_binary, utf8_lcase), 3)#x, repeat(collate(utf8_lcase#x, utf8_binary), 2) AS repeat(collate(utf8_lcase, utf8_binary), 2)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select chr(ascii) from t2
+-- !query analysis
+Project [chr(ascii#xL) AS chr(ascii)#x]
++- SubqueryAlias spark_catalog.default.t2
+ +- Relation spark_catalog.default.t2[ascii#xL] parquet
+
+
+-- !query
+select sentences(utf8_binary), sentences(utf8_lcase) from t1
+-- !query analysis
+Project [sentences(utf8_binary#x, , ) AS sentences(utf8_binary, , )#x, sentences(utf8_lcase#x, , ) AS sentences(utf8_lcase, , )#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select sentences(utf8_binary collate utf8_lcase), sentences(utf8_lcase collate utf8_binary) from t1
+-- !query analysis
+Project [sentences(collate(utf8_binary#x, utf8_lcase), , ) AS sentences(collate(utf8_binary, utf8_lcase), , )#x, sentences(collate(utf8_lcase#x, utf8_binary), , ) AS sentences(collate(utf8_lcase, utf8_binary), , )#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select upper(utf8_binary), upper(utf8_lcase) from t1
+-- !query analysis
+Project [upper(utf8_binary#x) AS upper(utf8_binary)#x, upper(utf8_lcase#x) AS upper(utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select upper(utf8_binary collate utf8_lcase), upper(utf8_lcase collate utf8_binary) from t1
+-- !query analysis
+Project [upper(collate(utf8_binary#x, utf8_lcase)) AS upper(collate(utf8_binary, utf8_lcase))#x, upper(collate(utf8_lcase#x, utf8_binary)) AS upper(collate(utf8_lcase, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select lower(utf8_binary), lower(utf8_lcase) from t1
+-- !query analysis
+Project [lower(utf8_binary#x) AS lower(utf8_binary)#x, lower(utf8_lcase#x) AS lower(utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select lower(utf8_binary collate utf8_lcase), lower(utf8_lcase collate utf8_binary) from t1
+-- !query analysis
+Project [lower(collate(utf8_binary#x, utf8_lcase)) AS lower(collate(utf8_binary, utf8_lcase))#x, lower(collate(utf8_lcase#x, utf8_binary)) AS lower(collate(utf8_lcase, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select initcap(utf8_binary), initcap(utf8_lcase) from t1
+-- !query analysis
+Project [initcap(utf8_binary#x) AS initcap(utf8_binary)#x, initcap(utf8_lcase#x) AS initcap(utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select initcap(utf8_binary collate utf8_lcase), initcap(utf8_lcase collate utf8_binary) from t1
+-- !query analysis
+Project [initcap(collate(utf8_binary#x, utf8_lcase)) AS initcap(collate(utf8_binary, utf8_lcase))#x, initcap(collate(utf8_lcase#x, utf8_binary)) AS initcap(collate(utf8_lcase, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select overlay(utf8_binary, utf8_lcase, 2) from t1
+-- !query analysis
+Project [overlay(cast(utf8_binary#x as string collate null), cast(utf8_lcase#x as string collate null), 2, -1) AS overlay(utf8_binary, utf8_lcase, 2, -1)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select overlay(s, utf8_binary,1) from t1
+-- !query analysis
+Project [overlay(s#x, utf8_binary#x, 1, -1) AS overlay(s, utf8_binary, 1, -1)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select overlay(utf8_binary collate utf8_binary, s collate utf8_lcase, 3) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
+ "sqlState" : "42P21",
+ "messageParameters" : {
+ "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
+ }
+}
+
+
+-- !query
+select overlay(utf8_binary, utf8_lcase collate utf8_binary, 2) from t1
+-- !query analysis
+Project [overlay(utf8_binary#x, collate(utf8_lcase#x, utf8_binary), 2, -1) AS overlay(utf8_binary, collate(utf8_lcase, utf8_binary), 2, -1)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select overlay(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 2) from t1
+-- !query analysis
+Project [overlay(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase), 2, -1) AS overlay(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase), 2, -1)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select overlay(utf8_binary, 'a', 2), overlay(utf8_lcase, 'a', 2) from t1
+-- !query analysis
+Project [overlay(utf8_binary#x, a, 2, -1) AS overlay(utf8_binary, a, 2, -1)#x, overlay(utf8_lcase#x, a, 2, -1) AS overlay(utf8_lcase, 'a' collate UTF8_LCASE, 2, -1)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select overlay(utf8_binary, 'AaAA' collate utf8_lcase, 2), overlay(utf8_lcase, 'AAa' collate utf8_binary, 2) from t1
+-- !query analysis
+Project [overlay(cast(utf8_binary#x as string collate UTF8_LCASE), collate(AaAA, utf8_lcase), 2, -1) AS overlay(utf8_binary, collate(AaAA, utf8_lcase), 2, -1)#x, overlay(cast(utf8_lcase#x as string), collate(AAa, utf8_binary), 2, -1) AS overlay(utf8_lcase, collate(AAa, utf8_binary), 2, -1)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select format_string(format, utf8_binary, utf8_lcase) from t3
+-- !query analysis
+Project [format_string(format#x, utf8_binary#x, utf8_lcase#x) AS format_string(format, utf8_binary, utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t3
+ +- Relation spark_catalog.default.t3[format#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select format_string(format collate utf8_lcase, utf8_lcase, utf8_binary collate utf8_lcase, 3), format_string(format, utf8_lcase collate utf8_binary, utf8_binary) from t3
+-- !query analysis
+Project [format_string(collate(format#x, utf8_lcase), utf8_lcase#x, collate(utf8_binary#x, utf8_lcase), 3) AS format_string(collate(format, utf8_lcase), utf8_lcase, collate(utf8_binary, utf8_lcase), 3)#x, format_string(format#x, collate(utf8_lcase#x, utf8_binary), utf8_binary#x) AS format_string(format, collate(utf8_lcase, utf8_binary), utf8_binary)#x]
++- SubqueryAlias spark_catalog.default.t3
+ +- Relation spark_catalog.default.t3[format#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select format_string(format, utf8_binary, utf8_lcase) from t3
+-- !query analysis
+Project [format_string(format#x, utf8_binary#x, utf8_lcase#x) AS format_string(format, utf8_binary, utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t3
+ +- Relation spark_catalog.default.t3[format#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select length(utf8_binary), length(utf8_lcase) from t1
+-- !query analysis
+Project [length(utf8_binary#x) AS length(utf8_binary)#x, length(utf8_lcase#x) AS length(utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select length(utf8_binary collate utf8_lcase), length(utf8_lcase collate utf8_binary) from t1
+-- !query analysis
+Project [length(collate(utf8_binary#x, utf8_lcase)) AS length(collate(utf8_binary, utf8_lcase))#x, length(collate(utf8_lcase#x, utf8_binary)) AS length(collate(utf8_lcase, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select bit_length(utf8_binary), bit_length(utf8_lcase) from t1
+-- !query analysis
+Project [bit_length(utf8_binary#x) AS bit_length(utf8_binary)#x, bit_length(utf8_lcase#x) AS bit_length(utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select bit_length(utf8_binary collate utf8_lcase), bit_length(utf8_lcase collate utf8_binary) from t1
+-- !query analysis
+Project [bit_length(collate(utf8_binary#x, utf8_lcase)) AS bit_length(collate(utf8_binary, utf8_lcase))#x, bit_length(collate(utf8_lcase#x, utf8_binary)) AS bit_length(collate(utf8_lcase, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select octet_length(utf8_binary), octet_length(utf8_lcase) from t1
+-- !query analysis
+Project [octet_length(utf8_binary#x) AS octet_length(utf8_binary)#x, octet_length(utf8_lcase#x) AS octet_length(utf8_lcase)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select octet_length(utf8_binary collate utf8_lcase), octet_length(utf8_lcase collate utf8_binary) from t1
+-- !query analysis
+Project [octet_length(collate(utf8_binary#x, utf8_lcase)) AS octet_length(collate(utf8_binary, utf8_lcase))#x, octet_length(collate(utf8_lcase#x, utf8_binary)) AS octet_length(collate(utf8_lcase, utf8_binary))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select octet_length(utf8_binary collate utf8_lcase_rtrim), octet_length(utf8_lcase collate utf8_binary_rtrim) from t1
+-- !query analysis
+Project [octet_length(collate(utf8_binary#x, utf8_lcase_rtrim)) AS octet_length(collate(utf8_binary, utf8_lcase_rtrim))#x, octet_length(collate(utf8_lcase#x, utf8_binary_rtrim)) AS octet_length(collate(utf8_lcase, utf8_binary_rtrim))#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select substr(utf8_binary, 2, 2), substr(utf8_lcase, 2, 2) from t1
+-- !query analysis
+Project [substr(utf8_binary#x, 2, 2) AS substr(utf8_binary, 2, 2)#x, substr(utf8_lcase#x, 2, 2) AS substr(utf8_lcase, 2, 2)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select substr(utf8_binary collate utf8_lcase, 2, 2), substr(utf8_lcase collate utf8_binary, 2, 2) from t1
+-- !query analysis
+Project [substr(collate(utf8_binary#x, utf8_lcase), 2, 2) AS substr(collate(utf8_binary, utf8_lcase), 2, 2)#x, substr(collate(utf8_lcase#x, utf8_binary), 2, 2) AS substr(collate(utf8_lcase, utf8_binary), 2, 2)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select right(utf8_binary, 2), right(utf8_lcase, 2) from t1
+-- !query analysis
+Project [right(utf8_binary#x, 2) AS right(utf8_binary, 2)#x, right(utf8_lcase#x, 2) AS right(utf8_lcase, 2)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select right(utf8_binary collate utf8_lcase, 2), right(utf8_lcase collate utf8_binary, 2) from t1
+-- !query analysis
+Project [right(collate(utf8_binary#x, utf8_lcase), 2) AS right(collate(utf8_binary, utf8_lcase), 2)#x, right(collate(utf8_lcase#x, utf8_binary), 2) AS right(collate(utf8_lcase, utf8_binary), 2)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select left(utf8_binary, '2' collate utf8_lcase), left(utf8_lcase, 2) from t1
+-- !query analysis
+Project [left(utf8_binary#x, cast(collate(2, utf8_lcase) as int)) AS left(utf8_binary, collate(2, utf8_lcase))#x, left(utf8_lcase#x, 2) AS left(utf8_lcase, 2)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select left(utf8_binary collate utf8_lcase, 2), left(utf8_lcase collate utf8_binary, 2) from t1
+-- !query analysis
+Project [left(collate(utf8_binary#x, utf8_lcase), 2) AS left(collate(utf8_binary, utf8_lcase), 2)#x, left(collate(utf8_lcase#x, utf8_binary), 2) AS left(collate(utf8_lcase, utf8_binary), 2)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select locate(utf8_binary, utf8_lcase) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION",
+ "sqlState" : "42P22",
+ "messageParameters" : {
+ "expr" : "\"locate(utf8_binary, utf8_lcase, 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 38,
+ "fragment" : "locate(utf8_binary, utf8_lcase)"
+ } ]
+}
+
+
+-- !query
+select locate(s, utf8_binary) from t1
+-- !query analysis
+Project [locate(s#x, utf8_binary#x, 1) AS locate(s, utf8_binary, 1)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select locate(utf8_binary collate utf8_binary, s collate utf8_lcase) from t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
+ "sqlState" : "42P21",
+ "messageParameters" : {
+ "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
+ }
+}
+
+
+-- !query
+select locate(utf8_binary, utf8_lcase collate utf8_binary) from t1
+-- !query analysis
+Project [locate(utf8_binary#x, collate(utf8_lcase#x, utf8_binary), 1) AS locate(utf8_binary, collate(utf8_lcase, utf8_binary), 1)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select locate(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 3) from t1
+-- !query analysis
+Project [locate(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase), 3) AS locate(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase), 3)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select locate(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai, 3) from t1
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"collate(utf8_binary, unicode_ai)\"",
+ "inputType" : "\"STRING COLLATE UNICODE_AI\"",
+ "paramIndex" : "first",
+ "requiredType" : "\"STRING\"",
+ "sqlExpr" : "\"locate(collate(utf8_binary, unicode_ai), collate(utf8_lcase, unicode_ai), 3)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 79,
+ "fragment" : "locate(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai, 3)"
+ } ]
+}
+
+
+-- !query
+select locate(utf8_binary, 'a'), locate(utf8_lcase, 'a') from t1
+-- !query analysis
+Project [locate(utf8_binary#x, a, 1) AS locate(utf8_binary, a, 1)#x, locate(utf8_lcase#x, a, 1) AS locate(utf8_lcase, 'a' collate UTF8_LCASE, 1)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select locate(utf8_binary, 'AaAA' collate utf8_lcase, 4), locate(utf8_lcase, 'AAa' collate utf8_binary, 4) from t1
+-- !query analysis
+Project [locate(cast(utf8_binary#x as string collate UTF8_LCASE), collate(AaAA, utf8_lcase), 4) AS locate(utf8_binary, collate(AaAA, utf8_lcase), 4)#x, locate(cast(utf8_lcase#x as string), collate(AAa, utf8_binary), 4) AS locate(utf8_lcase, collate(AAa, utf8_binary), 4)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+select locate(utf8_binary, 'AaAA ' collate utf8_binary_rtrim, 4), locate(utf8_lcase, 'AAa ' collate utf8_binary, 4) from t1
+-- !query analysis
+Project [locate(cast(utf8_binary#x as string collate UTF8_BINARY_RTRIM), collate(AaAA , utf8_binary_rtrim), 4) AS locate(utf8_binary, collate(AaAA , utf8_binary_rtrim), 4)#x, locate(cast(utf8_lcase#x as string), collate(AAa , utf8_binary), 4) AS locate(utf8_lcase, collate(AAa , utf8_binary), 4)#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[s#x,utf8_binary#x,utf8_lcase#x] parquet
+
+
+-- !query
+drop table t1
+-- !query analysis
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t1
+
+
+-- !query
+drop table t2
+-- !query analysis
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t2
+
+
+-- !query
+drop table t3
+-- !query analysis
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t3
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out
deleted file mode 100644
index 957bcabb078fe..0000000000000
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out
+++ /dev/null
@@ -1,3303 +0,0 @@
--- Automatically generated by SQLQueryTestSuite
--- !query
-create table t1(utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet
--- !query analysis
-CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false
-
-
--- !query
-insert into t1 values('aaa', 'aaa')
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [utf8_binary, utf8_lcase]
-+- Project [cast(col1#x as string) AS utf8_binary#x, cast(col2#x as string collate UTF8_LCASE) AS utf8_lcase#x]
- +- LocalRelation [col1#x, col2#x]
-
-
--- !query
-insert into t1 values('AAA', 'AAA')
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [utf8_binary, utf8_lcase]
-+- Project [cast(col1#x as string) AS utf8_binary#x, cast(col2#x as string collate UTF8_LCASE) AS utf8_lcase#x]
- +- LocalRelation [col1#x, col2#x]
-
-
--- !query
-insert into t1 values('bbb', 'bbb')
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [utf8_binary, utf8_lcase]
-+- Project [cast(col1#x as string) AS utf8_binary#x, cast(col2#x as string collate UTF8_LCASE) AS utf8_lcase#x]
- +- LocalRelation [col1#x, col2#x]
-
-
--- !query
-insert into t1 values('BBB', 'BBB')
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [utf8_binary, utf8_lcase]
-+- Project [cast(col1#x as string) AS utf8_binary#x, cast(col2#x as string collate UTF8_LCASE) AS utf8_lcase#x]
- +- LocalRelation [col1#x, col2#x]
-
-
--- !query
-describe table t1
--- !query analysis
-DescribeTableCommand `spark_catalog`.`default`.`t1`, false, [col_name#x, data_type#x, comment#x]
-
-
--- !query
-select count(*) from t1 group by utf8_binary
--- !query analysis
-Aggregate [utf8_binary#x], [count(1) AS count(1)#xL]
-+- SubqueryAlias spark_catalog.default.t1
- +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select count(*) from t1 group by utf8_lcase
--- !query analysis
-Aggregate [utf8_lcase#x], [count(1) AS count(1)#xL]
-+- SubqueryAlias spark_catalog.default.t1
- +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select * from t1 where utf8_binary = 'aaa'
--- !query analysis
-Project [utf8_binary#x, utf8_lcase#x]
-+- Filter (utf8_binary#x = aaa)
- +- SubqueryAlias spark_catalog.default.t1
- +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select * from t1 where utf8_lcase = 'aaa' collate utf8_lcase
--- !query analysis
-Project [utf8_binary#x, utf8_lcase#x]
-+- Filter (utf8_lcase#x = collate(aaa, utf8_lcase))
- +- SubqueryAlias spark_catalog.default.t1
- +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select * from t1 where utf8_binary < 'bbb'
--- !query analysis
-Project [utf8_binary#x, utf8_lcase#x]
-+- Filter (utf8_binary#x < bbb)
- +- SubqueryAlias spark_catalog.default.t1
- +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select * from t1 where utf8_lcase < 'bbb' collate utf8_lcase
--- !query analysis
-Project [utf8_binary#x, utf8_lcase#x]
-+- Filter (utf8_lcase#x < collate(bbb, utf8_lcase))
- +- SubqueryAlias spark_catalog.default.t1
- +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select l.utf8_binary, r.utf8_lcase from t1 l join t1 r on l.utf8_lcase = r.utf8_lcase
--- !query analysis
-Project [utf8_binary#x, utf8_lcase#x]
-+- Join Inner, (utf8_lcase#x = utf8_lcase#x)
- :- SubqueryAlias l
- : +- SubqueryAlias spark_catalog.default.t1
- : +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_lcase#x] parquet
- +- SubqueryAlias r
- +- SubqueryAlias spark_catalog.default.t1
- +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-create table t2(utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet
--- !query analysis
-CreateDataSourceTableCommand `spark_catalog`.`default`.`t2`, false
-
-
--- !query
-insert into t2 values('aaa', 'aaa')
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t2], Append, `spark_catalog`.`default`.`t2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t2), [utf8_binary, utf8_lcase]
-+- Project [cast(col1#x as string) AS utf8_binary#x, cast(col2#x as string collate UTF8_LCASE) AS utf8_lcase#x]
- +- LocalRelation [col1#x, col2#x]
-
-
--- !query
-insert into t2 values('bbb', 'bbb')
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t2], Append, `spark_catalog`.`default`.`t2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t2), [utf8_binary, utf8_lcase]
-+- Project [cast(col1#x as string) AS utf8_binary#x, cast(col2#x as string collate UTF8_LCASE) AS utf8_lcase#x]
- +- LocalRelation [col1#x, col2#x]
-
-
--- !query
-select * from t1 anti join t2 on t1.utf8_lcase = t2.utf8_lcase
--- !query analysis
-Project [utf8_binary#x, utf8_lcase#x]
-+- Join LeftAnti, (utf8_lcase#x = utf8_lcase#x)
- :- SubqueryAlias spark_catalog.default.t1
- : +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_lcase#x] parquet
- +- SubqueryAlias spark_catalog.default.t2
- +- Relation spark_catalog.default.t2[utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-drop table t2
--- !query analysis
-DropTable false, false
-+- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t2
-
-
--- !query
-drop table t1
--- !query analysis
-DropTable false, false
-+- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t1
-
-
--- !query
-select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except select col1 collate utf8_lcase from values ('aaa'), ('bbb')
--- !query analysis
-Except false
-:- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x]
-: +- LocalRelation [col1#x]
-+- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x]
- +- LocalRelation [col1#x]
-
-
--- !query
-select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except all select col1 collate utf8_lcase from values ('aaa'), ('bbb')
--- !query analysis
-Except All true
-:- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x]
-: +- LocalRelation [col1#x]
-+- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x]
- +- LocalRelation [col1#x]
-
-
--- !query
-select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union select col1 collate utf8_lcase from values ('aaa'), ('bbb')
--- !query analysis
-Distinct
-+- Union false, false
- :- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x]
- : +- LocalRelation [col1#x]
- +- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x]
- +- LocalRelation [col1#x]
-
-
--- !query
-select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union all select col1 collate utf8_lcase from values ('aaa'), ('bbb')
--- !query analysis
-Union false, false
-:- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x]
-: +- LocalRelation [col1#x]
-+- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x]
- +- LocalRelation [col1#x]
-
-
--- !query
-select col1 collate utf8_lcase from values ('aaa'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') intersect select col1 collate utf8_lcase from values ('aaa'), ('bbb')
--- !query analysis
-Intersect false
-:- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x]
-: +- LocalRelation [col1#x]
-+- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x]
- +- LocalRelation [col1#x]
-
-
--- !query
-select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except select col1 collate unicode_ci from values ('aaa'), ('bbb')
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "INCOMPATIBLE_COLUMN_TYPE",
- "sqlState" : "42825",
- "messageParameters" : {
- "columnOrdinalNumber" : "first",
- "dataType1" : "\"STRING COLLATE UNICODE_CI\"",
- "dataType2" : "\"STRING COLLATE UTF8_LCASE\"",
- "hint" : "",
- "operator" : "EXCEPT",
- "tableOrdinalNumber" : "second"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 1,
- "stopIndex" : 162,
- "fragment" : "select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except select col1 collate unicode_ci from values ('aaa'), ('bbb')"
- } ]
-}
-
-
--- !query
-select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except all select col1 collate unicode_ci from values ('aaa'), ('bbb')
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "INCOMPATIBLE_COLUMN_TYPE",
- "sqlState" : "42825",
- "messageParameters" : {
- "columnOrdinalNumber" : "first",
- "dataType1" : "\"STRING COLLATE UNICODE_CI\"",
- "dataType2" : "\"STRING COLLATE UTF8_LCASE\"",
- "hint" : "",
- "operator" : "EXCEPT ALL",
- "tableOrdinalNumber" : "second"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 1,
- "stopIndex" : 166,
- "fragment" : "select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except all select col1 collate unicode_ci from values ('aaa'), ('bbb')"
- } ]
-}
-
-
--- !query
-select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union select col1 collate unicode_ci from values ('aaa'), ('bbb')
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "INCOMPATIBLE_COLUMN_TYPE",
- "sqlState" : "42825",
- "messageParameters" : {
- "columnOrdinalNumber" : "first",
- "dataType1" : "\"STRING COLLATE UNICODE_CI\"",
- "dataType2" : "\"STRING COLLATE UTF8_LCASE\"",
- "hint" : "",
- "operator" : "UNION",
- "tableOrdinalNumber" : "second"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 1,
- "stopIndex" : 161,
- "fragment" : "select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union select col1 collate unicode_ci from values ('aaa'), ('bbb')"
- } ]
-}
-
-
--- !query
-select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union all select col1 collate unicode_ci from values ('aaa'), ('bbb')
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "INCOMPATIBLE_COLUMN_TYPE",
- "sqlState" : "42825",
- "messageParameters" : {
- "columnOrdinalNumber" : "first",
- "dataType1" : "\"STRING COLLATE UNICODE_CI\"",
- "dataType2" : "\"STRING COLLATE UTF8_LCASE\"",
- "hint" : "",
- "operator" : "UNION",
- "tableOrdinalNumber" : "second"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 1,
- "stopIndex" : 165,
- "fragment" : "select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union all select col1 collate unicode_ci from values ('aaa'), ('bbb')"
- } ]
-}
-
-
--- !query
-select col1 collate utf8_lcase from values ('aaa'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') intersect select col1 collate unicode_ci from values ('aaa'), ('bbb')
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "INCOMPATIBLE_COLUMN_TYPE",
- "sqlState" : "42825",
- "messageParameters" : {
- "columnOrdinalNumber" : "first",
- "dataType1" : "\"STRING COLLATE UNICODE_CI\"",
- "dataType2" : "\"STRING COLLATE UTF8_LCASE\"",
- "hint" : "",
- "operator" : "INTERSECT",
- "tableOrdinalNumber" : "second"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 1,
- "stopIndex" : 156,
- "fragment" : "select col1 collate utf8_lcase from values ('aaa'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') intersect select col1 collate unicode_ci from values ('aaa'), ('bbb')"
- } ]
-}
-
-
--- !query
-create table t1 (c1 struct) USING PARQUET
--- !query analysis
-CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false
-
-
--- !query
-insert into t1 values (named_struct('utf8_binary', 'aaa', 'utf8_lcase', 'aaa'))
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [c1]
-+- Project [named_struct(utf8_binary, col1#x.utf8_binary, utf8_lcase, cast(col1#x.utf8_lcase as string collate UTF8_LCASE)) AS c1#x]
- +- LocalRelation [col1#x]
-
-
--- !query
-insert into t1 values (named_struct('utf8_binary', 'AAA', 'utf8_lcase', 'AAA'))
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [c1]
-+- Project [named_struct(utf8_binary, col1#x.utf8_binary, utf8_lcase, cast(col1#x.utf8_lcase as string collate UTF8_LCASE)) AS c1#x]
- +- LocalRelation [col1#x]
-
-
--- !query
-select count(*) from t1 group by c1.utf8_binary
--- !query analysis
-Aggregate [c1#x.utf8_binary], [count(1) AS count(1)#xL]
-+- SubqueryAlias spark_catalog.default.t1
- +- Relation spark_catalog.default.t1[c1#x] parquet
-
-
--- !query
-select count(*) from t1 group by c1.utf8_lcase
--- !query analysis
-Aggregate [c1#x.utf8_lcase], [count(1) AS count(1)#xL]
-+- SubqueryAlias spark_catalog.default.t1
- +- Relation spark_catalog.default.t1[c1#x] parquet
-
-
--- !query
-drop table t1
--- !query analysis
-DropTable false, false
-+- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t1
-
-
--- !query
-select array_contains(ARRAY('aaa' collate utf8_lcase),'AAA' collate utf8_lcase)
--- !query analysis
-Project [array_contains(array(collate(aaa, utf8_lcase)), collate(AAA, utf8_lcase)) AS array_contains(array(collate(aaa, utf8_lcase)), collate(AAA, utf8_lcase))#x]
-+- OneRowRelation
-
-
--- !query
-select array_position(ARRAY('aaa' collate utf8_lcase, 'bbb' collate utf8_lcase),'BBB' collate utf8_lcase)
--- !query analysis
-Project [array_position(array(collate(aaa, utf8_lcase), collate(bbb, utf8_lcase)), collate(BBB, utf8_lcase)) AS array_position(array(collate(aaa, utf8_lcase), collate(bbb, utf8_lcase)), collate(BBB, utf8_lcase))#xL]
-+- OneRowRelation
-
-
--- !query
-select nullif('aaa' COLLATE utf8_lcase, 'AAA' COLLATE utf8_lcase)
--- !query analysis
-Project [nullif(collate(aaa, utf8_lcase), collate(AAA, utf8_lcase)) AS nullif(collate(aaa, utf8_lcase), collate(AAA, utf8_lcase))#x]
-+- OneRowRelation
-
-
--- !query
-select least('aaa' COLLATE utf8_lcase, 'AAA' collate utf8_lcase, 'a' collate utf8_lcase)
--- !query analysis
-Project [least(collate(aaa, utf8_lcase), collate(AAA, utf8_lcase), collate(a, utf8_lcase)) AS least(collate(aaa, utf8_lcase), collate(AAA, utf8_lcase), collate(a, utf8_lcase))#x]
-+- OneRowRelation
-
-
--- !query
-select arrays_overlap(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase))
--- !query analysis
-Project [arrays_overlap(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase))) AS arrays_overlap(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase)))#x]
-+- OneRowRelation
-
-
--- !query
-select array_distinct(array('aaa' collate utf8_lcase, 'AAA' collate utf8_lcase))
--- !query analysis
-Project [array_distinct(array(collate(aaa, utf8_lcase), collate(AAA, utf8_lcase))) AS array_distinct(array(collate(aaa, utf8_lcase), collate(AAA, utf8_lcase)))#x]
-+- OneRowRelation
-
-
--- !query
-select array_union(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase))
--- !query analysis
-Project [array_union(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase))) AS array_union(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase)))#x]
-+- OneRowRelation
-
-
--- !query
-select array_intersect(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase))
--- !query analysis
-Project [array_intersect(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase))) AS array_intersect(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase)))#x]
-+- OneRowRelation
-
-
--- !query
-select array_except(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase))
--- !query analysis
-Project [array_except(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase))) AS array_except(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase)))#x]
-+- OneRowRelation
-
-
--- !query
-select 'a' collate unicode < 'A'
--- !query analysis
-Project [(collate(a, unicode) < A) AS (collate(a, unicode) < 'A' collate UNICODE)#x]
-+- OneRowRelation
-
-
--- !query
-select 'a' collate unicode_ci = 'A'
--- !query analysis
-Project [(collate(a, unicode_ci) = A) AS (collate(a, unicode_ci) = 'A' collate UNICODE_CI)#x]
-+- OneRowRelation
-
-
--- !query
-select 'a' collate unicode_ai = 'å'
--- !query analysis
-Project [(collate(a, unicode_ai) = å) AS (collate(a, unicode_ai) = 'å' collate UNICODE_AI)#x]
-+- OneRowRelation
-
-
--- !query
-select 'a' collate unicode_ci_ai = 'Å'
--- !query analysis
-Project [(collate(a, unicode_ci_ai) = Å) AS (collate(a, unicode_ci_ai) = 'Å' collate UNICODE_CI_AI)#x]
-+- OneRowRelation
-
-
--- !query
-select 'a' collate en < 'A'
--- !query analysis
-Project [(collate(a, en) < A) AS (collate(a, en) < 'A' collate en)#x]
-+- OneRowRelation
-
-
--- !query
-select 'a' collate en_ci = 'A'
--- !query analysis
-Project [(collate(a, en_ci) = A) AS (collate(a, en_ci) = 'A' collate en_CI)#x]
-+- OneRowRelation
-
-
--- !query
-select 'a' collate en_ai = 'å'
--- !query analysis
-Project [(collate(a, en_ai) = å) AS (collate(a, en_ai) = 'å' collate en_AI)#x]
-+- OneRowRelation
-
-
--- !query
-select 'a' collate en_ci_ai = 'Å'
--- !query analysis
-Project [(collate(a, en_ci_ai) = Å) AS (collate(a, en_ci_ai) = 'Å' collate en_CI_AI)#x]
-+- OneRowRelation
-
-
--- !query
-select 'Kypper' collate sv < 'Köpfe'
--- !query analysis
-Project [(collate(Kypper, sv) < Köpfe) AS (collate(Kypper, sv) < 'Köpfe' collate sv)#x]
-+- OneRowRelation
-
-
--- !query
-select 'Kypper' collate de > 'Köpfe'
--- !query analysis
-Project [(collate(Kypper, de) > Köpfe) AS (collate(Kypper, de) > 'Köpfe' collate de)#x]
-+- OneRowRelation
-
-
--- !query
-select 'I' collate tr_ci = 'ı'
--- !query analysis
-Project [(collate(I, tr_ci) = ı) AS (collate(I, tr_ci) = 'ı' collate tr_CI)#x]
-+- OneRowRelation
-
-
--- !query
-create table t4 (text string collate utf8_binary, pairDelim string collate utf8_lcase, keyValueDelim string collate utf8_binary) using parquet
--- !query analysis
-CreateDataSourceTableCommand `spark_catalog`.`default`.`t4`, false
-
-
--- !query
-insert into t4 values('a:1,b:2,c:3', ',', ':')
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t4, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t4], Append, `spark_catalog`.`default`.`t4`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t4), [text, pairDelim, keyValueDelim]
-+- Project [cast(col1#x as string) AS text#x, cast(col2#x as string collate UTF8_LCASE) AS pairDelim#x, cast(col3#x as string) AS keyValueDelim#x]
- +- LocalRelation [col1#x, col2#x, col3#x]
-
-
--- !query
-select str_to_map(text, pairDelim, keyValueDelim) from t4
--- !query analysis
-Project [str_to_map(cast(text#x as string collate null), cast(pairDelim#x as string collate null), cast(keyValueDelim#x as string collate null)) AS str_to_map(text, pairDelim, keyValueDelim)#x]
-+- SubqueryAlias spark_catalog.default.t4
- +- Relation spark_catalog.default.t4[text#x,pairDelim#x,keyValueDelim#x] parquet
-
-
--- !query
-select str_to_map(text collate utf8_binary, pairDelim collate utf8_lcase, keyValueDelim collate utf8_binary) from t4
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
- "sqlState" : "42P21",
- "messageParameters" : {
- "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
- }
-}
-
-
--- !query
-select str_to_map(text collate utf8_binary, pairDelim collate utf8_binary, keyValueDelim collate utf8_binary) from t4
--- !query analysis
-Project [str_to_map(collate(text#x, utf8_binary), collate(pairDelim#x, utf8_binary), collate(keyValueDelim#x, utf8_binary)) AS str_to_map(collate(text, utf8_binary), collate(pairDelim, utf8_binary), collate(keyValueDelim, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t4
- +- Relation spark_catalog.default.t4[text#x,pairDelim#x,keyValueDelim#x] parquet
-
-
--- !query
-select str_to_map(text collate unicode_ai, pairDelim collate unicode_ai, keyValueDelim collate unicode_ai) from t4
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"collate(text, unicode_ai)\"",
- "inputType" : "\"STRING COLLATE UNICODE_AI\"",
- "paramIndex" : "first",
- "requiredType" : "\"STRING\"",
- "sqlExpr" : "\"str_to_map(collate(text, unicode_ai), collate(pairDelim, unicode_ai), collate(keyValueDelim, unicode_ai))\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 106,
- "fragment" : "str_to_map(text collate unicode_ai, pairDelim collate unicode_ai, keyValueDelim collate unicode_ai)"
- } ]
-}
-
-
--- !query
-drop table t4
--- !query analysis
-DropTable false, false
-+- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t4
-
-
--- !query
-create table t5(s string, utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet
--- !query analysis
-CreateDataSourceTableCommand `spark_catalog`.`default`.`t5`, false
-
-
--- !query
-insert into t5 values ('Spark', 'Spark', 'SQL')
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t5], Append, `spark_catalog`.`default`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t5), [s, utf8_binary, utf8_lcase]
-+- Project [cast(col1#x as string) AS s#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
- +- LocalRelation [col1#x, col2#x, col3#x]
-
-
--- !query
-insert into t5 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaAAaA')
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t5], Append, `spark_catalog`.`default`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t5), [s, utf8_binary, utf8_lcase]
-+- Project [cast(col1#x as string) AS s#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
- +- LocalRelation [col1#x, col2#x, col3#x]
-
-
--- !query
-insert into t5 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaaAaA')
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t5], Append, `spark_catalog`.`default`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t5), [s, utf8_binary, utf8_lcase]
-+- Project [cast(col1#x as string) AS s#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
- +- LocalRelation [col1#x, col2#x, col3#x]
-
-
--- !query
-insert into t5 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaaAaAaaAaaAaAaaAaaAaA')
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t5], Append, `spark_catalog`.`default`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t5), [s, utf8_binary, utf8_lcase]
-+- Project [cast(col1#x as string) AS s#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
- +- LocalRelation [col1#x, col2#x, col3#x]
-
-
--- !query
-insert into t5 values ('bbAbaAbA', 'bbAbAAbA', 'a')
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t5], Append, `spark_catalog`.`default`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t5), [s, utf8_binary, utf8_lcase]
-+- Project [cast(col1#x as string) AS s#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
- +- LocalRelation [col1#x, col2#x, col3#x]
-
-
--- !query
-insert into t5 values ('İo', 'İo', 'İo')
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t5], Append, `spark_catalog`.`default`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t5), [s, utf8_binary, utf8_lcase]
-+- Project [cast(col1#x as string) AS s#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
- +- LocalRelation [col1#x, col2#x, col3#x]
-
-
--- !query
-insert into t5 values ('İo', 'İo', 'İo ')
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t5], Append, `spark_catalog`.`default`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t5), [s, utf8_binary, utf8_lcase]
-+- Project [cast(col1#x as string) AS s#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
- +- LocalRelation [col1#x, col2#x, col3#x]
-
-
--- !query
-insert into t5 values ('İo', 'İo ', 'İo')
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t5], Append, `spark_catalog`.`default`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t5), [s, utf8_binary, utf8_lcase]
-+- Project [cast(col1#x as string) AS s#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
- +- LocalRelation [col1#x, col2#x, col3#x]
-
-
--- !query
-insert into t5 values ('İo', 'İo', 'i̇o')
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t5], Append, `spark_catalog`.`default`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t5), [s, utf8_binary, utf8_lcase]
-+- Project [cast(col1#x as string) AS s#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
- +- LocalRelation [col1#x, col2#x, col3#x]
-
-
--- !query
-insert into t5 values ('efd2', 'efd2', 'efd2')
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t5], Append, `spark_catalog`.`default`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t5), [s, utf8_binary, utf8_lcase]
-+- Project [cast(col1#x as string) AS s#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
- +- LocalRelation [col1#x, col2#x, col3#x]
-
-
--- !query
-insert into t5 values ('Hello, world! Nice day.', 'Hello, world! Nice day.', 'Hello, world! Nice day.')
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t5], Append, `spark_catalog`.`default`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t5), [s, utf8_binary, utf8_lcase]
-+- Project [cast(col1#x as string) AS s#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
- +- LocalRelation [col1#x, col2#x, col3#x]
-
-
--- !query
-insert into t5 values ('Something else. Nothing here.', 'Something else. Nothing here.', 'Something else. Nothing here.')
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t5], Append, `spark_catalog`.`default`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t5), [s, utf8_binary, utf8_lcase]
-+- Project [cast(col1#x as string) AS s#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
- +- LocalRelation [col1#x, col2#x, col3#x]
-
-
--- !query
-insert into t5 values ('kitten', 'kitten', 'sitTing')
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t5], Append, `spark_catalog`.`default`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t5), [s, utf8_binary, utf8_lcase]
-+- Project [cast(col1#x as string) AS s#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
- +- LocalRelation [col1#x, col2#x, col3#x]
-
-
--- !query
-insert into t5 values ('abc', 'abc', 'abc')
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t5], Append, `spark_catalog`.`default`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t5), [s, utf8_binary, utf8_lcase]
-+- Project [cast(col1#x as string) AS s#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
- +- LocalRelation [col1#x, col2#x, col3#x]
-
-
--- !query
-insert into t5 values ('abcdcba', 'abcdcba', 'aBcDCbA')
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t5], Append, `spark_catalog`.`default`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t5), [s, utf8_binary, utf8_lcase]
-+- Project [cast(col1#x as string) AS s#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
- +- LocalRelation [col1#x, col2#x, col3#x]
-
-
--- !query
-create table t6(ascii long) using parquet
--- !query analysis
-CreateDataSourceTableCommand `spark_catalog`.`default`.`t6`, false
-
-
--- !query
-insert into t6 values (97)
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t6, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t6], Append, `spark_catalog`.`default`.`t6`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t6), [ascii]
-+- Project [cast(col1#x as bigint) AS ascii#xL]
- +- LocalRelation [col1#x]
-
-
--- !query
-insert into t6 values (66)
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t6, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t6], Append, `spark_catalog`.`default`.`t6`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t6), [ascii]
-+- Project [cast(col1#x as bigint) AS ascii#xL]
- +- LocalRelation [col1#x]
-
-
--- !query
-create table t7(ascii double) using parquet
--- !query analysis
-CreateDataSourceTableCommand `spark_catalog`.`default`.`t7`, false
-
-
--- !query
-insert into t7 values (97.52143)
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t7, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t7], Append, `spark_catalog`.`default`.`t7`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t7), [ascii]
-+- Project [cast(col1#x as double) AS ascii#x]
- +- LocalRelation [col1#x]
-
-
--- !query
-insert into t7 values (66.421)
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t7, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t7], Append, `spark_catalog`.`default`.`t7`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t7), [ascii]
-+- Project [cast(col1#x as double) AS ascii#x]
- +- LocalRelation [col1#x]
-
-
--- !query
-create table t8(format string collate utf8_binary, utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet
--- !query analysis
-CreateDataSourceTableCommand `spark_catalog`.`default`.`t8`, false
-
-
--- !query
-insert into t8 values ('%s%s', 'abCdE', 'abCdE')
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t8, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t8], Append, `spark_catalog`.`default`.`t8`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t8), [format, utf8_binary, utf8_lcase]
-+- Project [cast(col1#x as string) AS format#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x]
- +- LocalRelation [col1#x, col2#x, col3#x]
-
-
--- !query
-create table t9(num long) using parquet
--- !query analysis
-CreateDataSourceTableCommand `spark_catalog`.`default`.`t9`, false
-
-
--- !query
-insert into t9 values (97)
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t9, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t9], Append, `spark_catalog`.`default`.`t9`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t9), [num]
-+- Project [cast(col1#x as bigint) AS num#xL]
- +- LocalRelation [col1#x]
-
-
--- !query
-insert into t9 values (66)
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t9, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t9], Append, `spark_catalog`.`default`.`t9`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t9), [num]
-+- Project [cast(col1#x as bigint) AS num#xL]
- +- LocalRelation [col1#x]
-
-
--- !query
-create table t10(utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet
--- !query analysis
-CreateDataSourceTableCommand `spark_catalog`.`default`.`t10`, false
-
-
--- !query
-insert into t10 values ('aaAaAAaA', 'aaAaaAaA')
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t10, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t10], Append, `spark_catalog`.`default`.`t10`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t10), [utf8_binary, utf8_lcase]
-+- Project [cast(col1#x as string) AS utf8_binary#x, cast(col2#x as string collate UTF8_LCASE) AS utf8_lcase#x]
- +- LocalRelation [col1#x, col2#x]
-
-
--- !query
-insert into t10 values ('efd2', 'efd2')
--- !query analysis
-InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t10, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t10], Append, `spark_catalog`.`default`.`t10`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t10), [utf8_binary, utf8_lcase]
-+- Project [cast(col1#x as string) AS utf8_binary#x, cast(col2#x as string collate UTF8_LCASE) AS utf8_lcase#x]
- +- LocalRelation [col1#x, col2#x]
-
-
--- !query
-select concat_ws(' ', utf8_lcase, utf8_lcase) from t5
--- !query analysis
-Project [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select concat_ws(' ', utf8_binary, utf8_lcase) from t5
--- !query analysis
-Project [concat_ws( , cast(utf8_binary#x as string collate null), cast(utf8_lcase#x as string collate null)) AS concat_ws(' ' collate null, utf8_binary, utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select concat_ws(' ' collate utf8_binary, utf8_binary, 'SQL' collate utf8_lcase) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
- "sqlState" : "42P21",
- "messageParameters" : {
- "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
- }
-}
-
-
--- !query
-select concat_ws(' ' collate utf8_lcase, utf8_binary, 'SQL' collate utf8_lcase) from t5
--- !query analysis
-Project [concat_ws(collate( , utf8_lcase), cast(utf8_binary#x as string collate UTF8_LCASE), collate(SQL, utf8_lcase)) AS concat_ws(collate( , utf8_lcase), utf8_binary, collate(SQL, utf8_lcase))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select concat_ws(',', utf8_lcase, 'word'), concat_ws(',', utf8_binary, 'word') from t5
--- !query analysis
-Project [concat_ws(,, utf8_lcase#x, word) AS concat_ws(',' collate UTF8_LCASE, utf8_lcase, 'word' collate UTF8_LCASE)#x, concat_ws(,, utf8_binary#x, word) AS concat_ws(,, utf8_binary, word)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select concat_ws(',', utf8_lcase, 'word' collate utf8_binary), concat_ws(',', utf8_binary, 'word' collate utf8_lcase) from t5
--- !query analysis
-Project [concat_ws(,, cast(utf8_lcase#x as string), collate(word, utf8_binary)) AS concat_ws(,, utf8_lcase, collate(word, utf8_binary))#x, concat_ws(,, cast(utf8_binary#x as string collate UTF8_LCASE), collate(word, utf8_lcase)) AS concat_ws(',' collate UTF8_LCASE, utf8_binary, collate(word, utf8_lcase))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select elt(2, s, utf8_binary) from t5
--- !query analysis
-Project [elt(2, s#x, utf8_binary#x, true) AS elt(2, s, utf8_binary)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select elt(2, utf8_binary, utf8_lcase, s) from t5
--- !query analysis
-Project [elt(2, cast(utf8_binary#x as string collate null), cast(utf8_lcase#x as string collate null), cast(s#x as string collate null), true) AS elt(2, utf8_binary, utf8_lcase, s)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select elt(1, utf8_binary collate utf8_binary, utf8_lcase collate utf8_lcase) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
- "sqlState" : "42P21",
- "messageParameters" : {
- "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
- }
-}
-
-
--- !query
-select elt(1, utf8_binary collate utf8_binary, utf8_lcase collate utf8_binary) from t5
--- !query analysis
-Project [elt(1, collate(utf8_binary#x, utf8_binary), collate(utf8_lcase#x, utf8_binary), true) AS elt(1, collate(utf8_binary, utf8_binary), collate(utf8_lcase, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select elt(1, utf8_binary collate utf8_binary, utf8_lcase) from t5
--- !query analysis
-Project [elt(1, collate(utf8_binary#x, utf8_binary), cast(utf8_lcase#x as string), true) AS elt(1, collate(utf8_binary, utf8_binary), utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select elt(1, utf8_binary, 'word'), elt(1, utf8_lcase, 'word') from t5
--- !query analysis
-Project [elt(1, utf8_binary#x, word, true) AS elt(1, utf8_binary, word)#x, elt(1, utf8_lcase#x, word, true) AS elt(1, utf8_lcase, 'word' collate UTF8_LCASE)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select elt(1, utf8_binary, 'word' collate utf8_lcase), elt(1, utf8_lcase, 'word' collate utf8_binary) from t5
--- !query analysis
-Project [elt(1, cast(utf8_binary#x as string collate UTF8_LCASE), collate(word, utf8_lcase), true) AS elt(1, utf8_binary, collate(word, utf8_lcase))#x, elt(1, cast(utf8_lcase#x as string), collate(word, utf8_binary), true) AS elt(1, utf8_lcase, collate(word, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select split_part(utf8_binary, utf8_lcase, 3) from t5
--- !query analysis
-Project [split_part(cast(utf8_binary#x as string collate null), cast(utf8_lcase#x as string collate null), 3) AS split_part(utf8_binary, utf8_lcase, 3)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select split_part(s, utf8_binary, 1) from t5
--- !query analysis
-Project [split_part(s#x, utf8_binary#x, 1) AS split_part(s, utf8_binary, 1)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select split_part(utf8_binary collate utf8_binary, s collate utf8_lcase, 1) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
- "sqlState" : "42P21",
- "messageParameters" : {
- "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
- }
-}
-
-
--- !query
-select split_part(utf8_binary, utf8_lcase collate utf8_binary, 2) from t5
--- !query analysis
-Project [split_part(utf8_binary#x, collate(utf8_lcase#x, utf8_binary), 2) AS split_part(utf8_binary, collate(utf8_lcase, utf8_binary), 2)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select split_part(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 2) from t5
--- !query analysis
-Project [split_part(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase), 2) AS split_part(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase), 2)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select split_part(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai, 2) from t5
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"collate(utf8_binary, unicode_ai)\"",
- "inputType" : "\"STRING COLLATE UNICODE_AI\"",
- "paramIndex" : "first",
- "requiredType" : "\"STRING\"",
- "sqlExpr" : "\"split_part(collate(utf8_binary, unicode_ai), collate(utf8_lcase, unicode_ai), 2)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 83,
- "fragment" : "split_part(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai, 2)"
- } ]
-}
-
-
--- !query
-select split_part(utf8_binary, 'a', 3), split_part(utf8_lcase, 'a', 3) from t5
--- !query analysis
-Project [split_part(utf8_binary#x, a, 3) AS split_part(utf8_binary, a, 3)#x, split_part(utf8_lcase#x, a, 3) AS split_part(utf8_lcase, 'a' collate UTF8_LCASE, 3)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select split_part(utf8_binary, 'a' collate utf8_lcase, 3), split_part(utf8_lcase, 'a' collate utf8_binary, 3) from t5
--- !query analysis
-Project [split_part(cast(utf8_binary#x as string collate UTF8_LCASE), collate(a, utf8_lcase), 3) AS split_part(utf8_binary, collate(a, utf8_lcase), 3)#x, split_part(cast(utf8_lcase#x as string), collate(a, utf8_binary), 3) AS split_part(utf8_lcase, collate(a, utf8_binary), 3)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select split_part(utf8_binary, 'a ' collate utf8_lcase_rtrim, 3), split_part(utf8_lcase, 'a' collate utf8_binary, 3) from t5
--- !query analysis
-Project [split_part(cast(utf8_binary#x as string collate UTF8_LCASE_RTRIM), collate(a , utf8_lcase_rtrim), 3) AS split_part(utf8_binary, collate(a , utf8_lcase_rtrim), 3)#x, split_part(cast(utf8_lcase#x as string), collate(a, utf8_binary), 3) AS split_part(utf8_lcase, collate(a, utf8_binary), 3)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select contains(utf8_binary, utf8_lcase) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION",
- "sqlState" : "42P22",
- "messageParameters" : {
- "expr" : "\"contains(utf8_binary, utf8_lcase)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 40,
- "fragment" : "contains(utf8_binary, utf8_lcase)"
- } ]
-}
-
-
--- !query
-select contains(s, utf8_binary) from t5
--- !query analysis
-Project [Contains(s#x, utf8_binary#x) AS contains(s, utf8_binary)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select contains(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
- "sqlState" : "42P21",
- "messageParameters" : {
- "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
- }
-}
-
-
--- !query
-select contains(utf8_binary, utf8_lcase collate utf8_binary) from t5
--- !query analysis
-Project [Contains(utf8_binary#x, collate(utf8_lcase#x, utf8_binary)) AS contains(utf8_binary, collate(utf8_lcase, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select contains(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5
--- !query analysis
-Project [Contains(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase)) AS contains(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select contains(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t5
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"collate(utf8_binary, unicode_ai)\"",
- "inputType" : "\"STRING COLLATE UNICODE_AI\"",
- "paramIndex" : "first",
- "requiredType" : "\"STRING\"",
- "sqlExpr" : "\"contains(collate(utf8_binary, unicode_ai), collate(utf8_lcase, unicode_ai))\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 78,
- "fragment" : "contains(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai)"
- } ]
-}
-
-
--- !query
-select contains(utf8_binary, 'a'), contains(utf8_lcase, 'a') from t5
--- !query analysis
-Project [Contains(utf8_binary#x, a) AS contains(utf8_binary, a)#x, Contains(utf8_lcase#x, a) AS contains(utf8_lcase, 'a' collate UTF8_LCASE)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select contains(utf8_binary, 'AaAA' collate utf8_lcase), contains(utf8_lcase, 'AAa' collate utf8_binary) from t5
--- !query analysis
-Project [Contains(cast(utf8_binary#x as string collate UTF8_LCASE), collate(AaAA, utf8_lcase)) AS contains(utf8_binary, collate(AaAA, utf8_lcase))#x, Contains(cast(utf8_lcase#x as string), collate(AAa, utf8_binary)) AS contains(utf8_lcase, collate(AAa, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select contains(utf8_binary, 'AaAA ' collate utf8_lcase_rtrim), contains(utf8_lcase, 'AAa ' collate utf8_binary_rtrim) from t5
--- !query analysis
-Project [Contains(cast(utf8_binary#x as string collate UTF8_LCASE_RTRIM), collate(AaAA , utf8_lcase_rtrim)) AS contains(utf8_binary, collate(AaAA , utf8_lcase_rtrim))#x, Contains(cast(utf8_lcase#x as string collate UTF8_BINARY_RTRIM), collate(AAa , utf8_binary_rtrim)) AS contains(utf8_lcase, collate(AAa , utf8_binary_rtrim))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select substring_index(utf8_binary, utf8_lcase, 2) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION",
- "sqlState" : "42P22",
- "messageParameters" : {
- "expr" : "\"substring_index(utf8_binary, utf8_lcase, 2)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 50,
- "fragment" : "substring_index(utf8_binary, utf8_lcase, 2)"
- } ]
-}
-
-
--- !query
-select substring_index(s, utf8_binary,1) from t5
--- !query analysis
-Project [substring_index(s#x, utf8_binary#x, 1) AS substring_index(s, utf8_binary, 1)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select substring_index(utf8_binary collate utf8_binary, s collate utf8_lcase, 3) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
- "sqlState" : "42P21",
- "messageParameters" : {
- "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
- }
-}
-
-
--- !query
-select substring_index(utf8_binary, utf8_lcase collate utf8_binary, 2) from t5
--- !query analysis
-Project [substring_index(utf8_binary#x, collate(utf8_lcase#x, utf8_binary), 2) AS substring_index(utf8_binary, collate(utf8_lcase, utf8_binary), 2)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select substring_index(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 2) from t5
--- !query analysis
-Project [substring_index(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase), 2) AS substring_index(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase), 2)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select substring_index(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai, 2) from t5
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"collate(utf8_binary, unicode_ai)\"",
- "inputType" : "\"STRING COLLATE UNICODE_AI\"",
- "paramIndex" : "first",
- "requiredType" : "\"STRING\"",
- "sqlExpr" : "\"substring_index(collate(utf8_binary, unicode_ai), collate(utf8_lcase, unicode_ai), 2)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 88,
- "fragment" : "substring_index(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai, 2)"
- } ]
-}
-
-
--- !query
-select substring_index(utf8_binary, 'a', 2), substring_index(utf8_lcase, 'a', 2) from t5
--- !query analysis
-Project [substring_index(utf8_binary#x, a, 2) AS substring_index(utf8_binary, a, 2)#x, substring_index(utf8_lcase#x, a, 2) AS substring_index(utf8_lcase, 'a' collate UTF8_LCASE, 2)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select substring_index(utf8_binary, 'AaAA' collate utf8_lcase, 2), substring_index(utf8_lcase, 'AAa' collate utf8_binary, 2) from t5
--- !query analysis
-Project [substring_index(cast(utf8_binary#x as string collate UTF8_LCASE), collate(AaAA, utf8_lcase), 2) AS substring_index(utf8_binary, collate(AaAA, utf8_lcase), 2)#x, substring_index(cast(utf8_lcase#x as string), collate(AAa, utf8_binary), 2) AS substring_index(utf8_lcase, collate(AAa, utf8_binary), 2)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select substring_index(utf8_binary, 'AaAA ' collate utf8_lcase_rtrim, 2), substring_index(utf8_lcase, 'AAa' collate utf8_binary, 2) from t5
--- !query analysis
-Project [substring_index(cast(utf8_binary#x as string collate UTF8_LCASE_RTRIM), collate(AaAA , utf8_lcase_rtrim), 2) AS substring_index(utf8_binary, collate(AaAA , utf8_lcase_rtrim), 2)#x, substring_index(cast(utf8_lcase#x as string), collate(AAa, utf8_binary), 2) AS substring_index(utf8_lcase, collate(AAa, utf8_binary), 2)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select instr(utf8_binary, utf8_lcase) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION",
- "sqlState" : "42P22",
- "messageParameters" : {
- "expr" : "\"instr(utf8_binary, utf8_lcase)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 37,
- "fragment" : "instr(utf8_binary, utf8_lcase)"
- } ]
-}
-
-
--- !query
-select instr(s, utf8_binary) from t5
--- !query analysis
-Project [instr(s#x, utf8_binary#x) AS instr(s, utf8_binary)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select instr(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
- "sqlState" : "42P21",
- "messageParameters" : {
- "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
- }
-}
-
-
--- !query
-select instr(utf8_binary, utf8_lcase collate utf8_binary) from t5
--- !query analysis
-Project [instr(utf8_binary#x, collate(utf8_lcase#x, utf8_binary)) AS instr(utf8_binary, collate(utf8_lcase, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select instr(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5
--- !query analysis
-Project [instr(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase)) AS instr(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select instr(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t5
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"collate(utf8_binary, unicode_ai)\"",
- "inputType" : "\"STRING COLLATE UNICODE_AI\"",
- "paramIndex" : "first",
- "requiredType" : "\"STRING\"",
- "sqlExpr" : "\"instr(collate(utf8_binary, unicode_ai), collate(utf8_lcase, unicode_ai))\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 75,
- "fragment" : "instr(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai)"
- } ]
-}
-
-
--- !query
-select instr(utf8_binary, 'a'), instr(utf8_lcase, 'a') from t5
--- !query analysis
-Project [instr(utf8_binary#x, a) AS instr(utf8_binary, a)#x, instr(utf8_lcase#x, a) AS instr(utf8_lcase, 'a' collate UTF8_LCASE)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select instr(utf8_binary, 'AaAA' collate utf8_lcase), instr(utf8_lcase, 'AAa' collate utf8_binary) from t5
--- !query analysis
-Project [instr(cast(utf8_binary#x as string collate UTF8_LCASE), collate(AaAA, utf8_lcase)) AS instr(utf8_binary, collate(AaAA, utf8_lcase))#x, instr(cast(utf8_lcase#x as string), collate(AAa, utf8_binary)) AS instr(utf8_lcase, collate(AAa, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select find_in_set(utf8_binary, utf8_lcase) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION",
- "sqlState" : "42P22",
- "messageParameters" : {
- "expr" : "\"find_in_set(utf8_binary, utf8_lcase)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 43,
- "fragment" : "find_in_set(utf8_binary, utf8_lcase)"
- } ]
-}
-
-
--- !query
-select find_in_set(s, utf8_binary) from t5
--- !query analysis
-Project [find_in_set(s#x, utf8_binary#x) AS find_in_set(s, utf8_binary)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select find_in_set(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
- "sqlState" : "42P21",
- "messageParameters" : {
- "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
- }
-}
-
-
--- !query
-select find_in_set(utf8_binary, utf8_lcase collate utf8_binary) from t5
--- !query analysis
-Project [find_in_set(utf8_binary#x, collate(utf8_lcase#x, utf8_binary)) AS find_in_set(utf8_binary, collate(utf8_lcase, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select find_in_set(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5
--- !query analysis
-Project [find_in_set(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase)) AS find_in_set(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select find_in_set(utf8_binary, 'aaAaaAaA,i̇o'), find_in_set(utf8_lcase, 'aaAaaAaA,i̇o') from t5
--- !query analysis
-Project [find_in_set(utf8_binary#x, aaAaaAaA,i̇o) AS find_in_set(utf8_binary, aaAaaAaA,i̇o)#x, find_in_set(utf8_lcase#x, aaAaaAaA,i̇o) AS find_in_set(utf8_lcase, 'aaAaaAaA,i̇o' collate UTF8_LCASE)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select find_in_set(utf8_binary, 'aaAaaAaA,i̇o' collate utf8_lcase), find_in_set(utf8_lcase, 'aaAaaAaA,i̇o' collate utf8_binary) from t5
--- !query analysis
-Project [find_in_set(cast(utf8_binary#x as string collate UTF8_LCASE), collate(aaAaaAaA,i̇o, utf8_lcase)) AS find_in_set(utf8_binary, collate(aaAaaAaA,i̇o, utf8_lcase))#x, find_in_set(cast(utf8_lcase#x as string), collate(aaAaaAaA,i̇o, utf8_binary)) AS find_in_set(utf8_lcase, collate(aaAaaAaA,i̇o, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select find_in_set(utf8_binary, 'aaAaaAaA,i̇o ' collate utf8_lcase_rtrim), find_in_set(utf8_lcase, 'aaAaaAaA,i̇o' collate utf8_binary) from t5
--- !query analysis
-Project [find_in_set(cast(utf8_binary#x as string collate UTF8_LCASE_RTRIM), collate(aaAaaAaA,i̇o , utf8_lcase_rtrim)) AS find_in_set(utf8_binary, collate(aaAaaAaA,i̇o , utf8_lcase_rtrim))#x, find_in_set(cast(utf8_lcase#x as string), collate(aaAaaAaA,i̇o, utf8_binary)) AS find_in_set(utf8_lcase, collate(aaAaaAaA,i̇o, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select startswith(utf8_binary, utf8_lcase) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION",
- "sqlState" : "42P22",
- "messageParameters" : {
- "expr" : "\"startswith(utf8_binary, utf8_lcase)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 42,
- "fragment" : "startswith(utf8_binary, utf8_lcase)"
- } ]
-}
-
-
--- !query
-select startswith(s, utf8_binary) from t5
--- !query analysis
-Project [StartsWith(s#x, utf8_binary#x) AS startswith(s, utf8_binary)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select startswith(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
- "sqlState" : "42P21",
- "messageParameters" : {
- "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
- }
-}
-
-
--- !query
-select startswith(utf8_binary, utf8_lcase collate utf8_binary) from t5
--- !query analysis
-Project [StartsWith(utf8_binary#x, collate(utf8_lcase#x, utf8_binary)) AS startswith(utf8_binary, collate(utf8_lcase, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select startswith(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5
--- !query analysis
-Project [StartsWith(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase)) AS startswith(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select startswith(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t5
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"collate(utf8_binary, unicode_ai)\"",
- "inputType" : "\"STRING COLLATE UNICODE_AI\"",
- "paramIndex" : "first",
- "requiredType" : "\"STRING\"",
- "sqlExpr" : "\"startswith(collate(utf8_binary, unicode_ai), collate(utf8_lcase, unicode_ai))\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 80,
- "fragment" : "startswith(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai)"
- } ]
-}
-
-
--- !query
-select startswith(utf8_binary, 'aaAaaAaA'), startswith(utf8_lcase, 'aaAaaAaA') from t5
--- !query analysis
-Project [StartsWith(utf8_binary#x, aaAaaAaA) AS startswith(utf8_binary, aaAaaAaA)#x, StartsWith(utf8_lcase#x, aaAaaAaA) AS startswith(utf8_lcase, 'aaAaaAaA' collate UTF8_LCASE)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select startswith(utf8_binary, 'aaAaaAaA' collate utf8_lcase), startswith(utf8_lcase, 'aaAaaAaA' collate utf8_binary) from t5
--- !query analysis
-Project [StartsWith(cast(utf8_binary#x as string collate UTF8_LCASE), collate(aaAaaAaA, utf8_lcase)) AS startswith(utf8_binary, collate(aaAaaAaA, utf8_lcase))#x, StartsWith(cast(utf8_lcase#x as string), collate(aaAaaAaA, utf8_binary)) AS startswith(utf8_lcase, collate(aaAaaAaA, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select startswith(utf8_binary, 'aaAaaAaA ' collate utf8_lcase_rtrim), startswith(utf8_lcase, 'aaAaaAaA' collate utf8_binary) from t5
--- !query analysis
-Project [StartsWith(cast(utf8_binary#x as string collate UTF8_LCASE_RTRIM), collate(aaAaaAaA , utf8_lcase_rtrim)) AS startswith(utf8_binary, collate(aaAaaAaA , utf8_lcase_rtrim))#x, StartsWith(cast(utf8_lcase#x as string), collate(aaAaaAaA, utf8_binary)) AS startswith(utf8_lcase, collate(aaAaaAaA, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select translate(utf8_lcase, utf8_lcase, '12345') from t5
--- !query analysis
-Project [translate(utf8_lcase#x, utf8_lcase#x, 12345) AS translate(utf8_lcase, utf8_lcase, '12345' collate UTF8_LCASE)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select translate(utf8_binary, utf8_lcase, '12345') from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION",
- "sqlState" : "42P22",
- "messageParameters" : {
- "expr" : "\"translate(utf8_binary, utf8_lcase, 12345)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 50,
- "fragment" : "translate(utf8_binary, utf8_lcase, '12345')"
- } ]
-}
-
-
--- !query
-select translate(utf8_binary, 'aBc' collate utf8_lcase, '12345' collate utf8_binary) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
- "sqlState" : "42P21",
- "messageParameters" : {
- "explicitTypes" : "\"STRING COLLATE UTF8_LCASE\", \"STRING\""
- }
-}
-
-
--- !query
-select translate(utf8_binary, 'SQL' collate utf8_lcase, '12345' collate utf8_lcase) from t5
--- !query analysis
-Project [translate(cast(utf8_binary#x as string collate UTF8_LCASE), collate(SQL, utf8_lcase), collate(12345, utf8_lcase)) AS translate(utf8_binary, collate(SQL, utf8_lcase), collate(12345, utf8_lcase))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select translate(utf8_binary, 'SQL' collate unicode_ai, '12345' collate unicode_ai) from t5
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"utf8_binary\"",
- "inputType" : "\"STRING COLLATE UNICODE_AI\"",
- "paramIndex" : "first",
- "requiredType" : "\"STRING\"",
- "sqlExpr" : "\"translate(utf8_binary, collate(SQL, unicode_ai), collate(12345, unicode_ai))\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 83,
- "fragment" : "translate(utf8_binary, 'SQL' collate unicode_ai, '12345' collate unicode_ai)"
- } ]
-}
-
-
--- !query
-select translate(utf8_lcase, 'aaAaaAaA', '12345'), translate(utf8_binary, 'aaAaaAaA', '12345') from t5
--- !query analysis
-Project [translate(utf8_lcase#x, aaAaaAaA, 12345) AS translate(utf8_lcase, 'aaAaaAaA' collate UTF8_LCASE, '12345' collate UTF8_LCASE)#x, translate(utf8_binary#x, aaAaaAaA, 12345) AS translate(utf8_binary, aaAaaAaA, 12345)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select translate(utf8_lcase, 'aBc' collate utf8_binary, '12345'), translate(utf8_binary, 'aBc' collate utf8_lcase, '12345') from t5
--- !query analysis
-Project [translate(cast(utf8_lcase#x as string), collate(aBc, utf8_binary), 12345) AS translate(utf8_lcase, collate(aBc, utf8_binary), 12345)#x, translate(cast(utf8_binary#x as string collate UTF8_LCASE), collate(aBc, utf8_lcase), 12345) AS translate(utf8_binary, collate(aBc, utf8_lcase), '12345' collate UTF8_LCASE)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select translate(utf8_lcase, 'aBc ' collate utf8_binary_rtrim, '12345'), translate(utf8_binary, 'aBc' collate utf8_lcase, '12345') from t5
--- !query analysis
-Project [translate(cast(utf8_lcase#x as string collate UTF8_BINARY_RTRIM), collate(aBc , utf8_binary_rtrim), 12345) AS translate(utf8_lcase, collate(aBc , utf8_binary_rtrim), '12345' collate UTF8_BINARY_RTRIM)#x, translate(cast(utf8_binary#x as string collate UTF8_LCASE), collate(aBc, utf8_lcase), 12345) AS translate(utf8_binary, collate(aBc, utf8_lcase), '12345' collate UTF8_LCASE)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select replace(utf8_binary, utf8_lcase, 'abc') from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION",
- "sqlState" : "42P22",
- "messageParameters" : {
- "expr" : "\"replace(utf8_binary, utf8_lcase, abc)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 46,
- "fragment" : "replace(utf8_binary, utf8_lcase, 'abc')"
- } ]
-}
-
-
--- !query
-select replace(s, utf8_binary, 'abc') from t5
--- !query analysis
-Project [replace(s#x, utf8_binary#x, abc) AS replace(s, utf8_binary, abc)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select replace(utf8_binary collate utf8_binary, s collate utf8_lcase, 'abc') from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
- "sqlState" : "42P21",
- "messageParameters" : {
- "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
- }
-}
-
-
--- !query
-select replace(utf8_binary, utf8_lcase collate utf8_binary, 'abc') from t5
--- !query analysis
-Project [replace(utf8_binary#x, collate(utf8_lcase#x, utf8_binary), abc) AS replace(utf8_binary, collate(utf8_lcase, utf8_binary), abc)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select replace(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 'abc') from t5
--- !query analysis
-Project [replace(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase), abc) AS replace(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase), 'abc' collate UTF8_LCASE)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select replace(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai, 'abc') from t5
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"collate(utf8_binary, unicode_ai)\"",
- "inputType" : "\"STRING COLLATE UNICODE_AI\"",
- "paramIndex" : "first",
- "requiredType" : "\"STRING\"",
- "sqlExpr" : "\"replace(collate(utf8_binary, unicode_ai), collate(utf8_lcase, unicode_ai), 'abc' collate UNICODE_AI)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 84,
- "fragment" : "replace(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai, 'abc')"
- } ]
-}
-
-
--- !query
-select replace(utf8_binary, 'aaAaaAaA', 'abc'), replace(utf8_lcase, 'aaAaaAaA', 'abc') from t5
--- !query analysis
-Project [replace(utf8_binary#x, aaAaaAaA, abc) AS replace(utf8_binary, aaAaaAaA, abc)#x, replace(utf8_lcase#x, aaAaaAaA, abc) AS replace(utf8_lcase, 'aaAaaAaA' collate UTF8_LCASE, 'abc' collate UTF8_LCASE)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select replace(utf8_binary, 'aaAaaAaA' collate utf8_lcase, 'abc'), replace(utf8_lcase, 'aaAaaAaA' collate utf8_binary, 'abc') from t5
--- !query analysis
-Project [replace(cast(utf8_binary#x as string collate UTF8_LCASE), collate(aaAaaAaA, utf8_lcase), abc) AS replace(utf8_binary, collate(aaAaaAaA, utf8_lcase), 'abc' collate UTF8_LCASE)#x, replace(cast(utf8_lcase#x as string), collate(aaAaaAaA, utf8_binary), abc) AS replace(utf8_lcase, collate(aaAaaAaA, utf8_binary), abc)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select replace(utf8_binary, 'aaAaaAaA ' collate utf8_lcase_rtrim, 'abc'), replace(utf8_lcase, 'aaAaaAaA' collate utf8_binary, 'abc') from t5
--- !query analysis
-Project [replace(cast(utf8_binary#x as string collate UTF8_LCASE_RTRIM), collate(aaAaaAaA , utf8_lcase_rtrim), abc) AS replace(utf8_binary, collate(aaAaaAaA , utf8_lcase_rtrim), 'abc' collate UTF8_LCASE_RTRIM)#x, replace(cast(utf8_lcase#x as string), collate(aaAaaAaA, utf8_binary), abc) AS replace(utf8_lcase, collate(aaAaaAaA, utf8_binary), abc)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select endswith(utf8_binary, utf8_lcase) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION",
- "sqlState" : "42P22",
- "messageParameters" : {
- "expr" : "\"endswith(utf8_binary, utf8_lcase)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 40,
- "fragment" : "endswith(utf8_binary, utf8_lcase)"
- } ]
-}
-
-
--- !query
-select endswith(s, utf8_binary) from t5
--- !query analysis
-Project [EndsWith(s#x, utf8_binary#x) AS endswith(s, utf8_binary)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select endswith(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
- "sqlState" : "42P21",
- "messageParameters" : {
- "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
- }
-}
-
-
--- !query
-select endswith(utf8_binary, utf8_lcase collate utf8_binary) from t5
--- !query analysis
-Project [EndsWith(utf8_binary#x, collate(utf8_lcase#x, utf8_binary)) AS endswith(utf8_binary, collate(utf8_lcase, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select endswith(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5
--- !query analysis
-Project [EndsWith(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase)) AS endswith(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select endswith(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t5
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"collate(utf8_binary, unicode_ai)\"",
- "inputType" : "\"STRING COLLATE UNICODE_AI\"",
- "paramIndex" : "first",
- "requiredType" : "\"STRING\"",
- "sqlExpr" : "\"endswith(collate(utf8_binary, unicode_ai), collate(utf8_lcase, unicode_ai))\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 78,
- "fragment" : "endswith(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai)"
- } ]
-}
-
-
--- !query
-select endswith(utf8_binary, 'aaAaaAaA'), endswith(utf8_lcase, 'aaAaaAaA') from t5
--- !query analysis
-Project [EndsWith(utf8_binary#x, aaAaaAaA) AS endswith(utf8_binary, aaAaaAaA)#x, EndsWith(utf8_lcase#x, aaAaaAaA) AS endswith(utf8_lcase, 'aaAaaAaA' collate UTF8_LCASE)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select endswith(utf8_binary, 'aaAaaAaA' collate utf8_lcase), endswith(utf8_lcase, 'aaAaaAaA' collate utf8_binary) from t5
--- !query analysis
-Project [EndsWith(cast(utf8_binary#x as string collate UTF8_LCASE), collate(aaAaaAaA, utf8_lcase)) AS endswith(utf8_binary, collate(aaAaaAaA, utf8_lcase))#x, EndsWith(cast(utf8_lcase#x as string), collate(aaAaaAaA, utf8_binary)) AS endswith(utf8_lcase, collate(aaAaaAaA, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select endswith(utf8_binary, 'aaAaaAaA ' collate utf8_lcase_rtrim), endswith(utf8_lcase, 'aaAaaAaA' collate utf8_binary) from t5
--- !query analysis
-Project [EndsWith(cast(utf8_binary#x as string collate UTF8_LCASE_RTRIM), collate(aaAaaAaA , utf8_lcase_rtrim)) AS endswith(utf8_binary, collate(aaAaaAaA , utf8_lcase_rtrim))#x, EndsWith(cast(utf8_lcase#x as string), collate(aaAaaAaA, utf8_binary)) AS endswith(utf8_lcase, collate(aaAaaAaA, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select repeat(utf8_binary, 3), repeat(utf8_lcase, 2) from t5
--- !query analysis
-Project [repeat(utf8_binary#x, 3) AS repeat(utf8_binary, 3)#x, repeat(utf8_lcase#x, 2) AS repeat(utf8_lcase, 2)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select repeat(utf8_binary collate utf8_lcase, 3), repeat(utf8_lcase collate utf8_binary, 2) from t5
--- !query analysis
-Project [repeat(collate(utf8_binary#x, utf8_lcase), 3) AS repeat(collate(utf8_binary, utf8_lcase), 3)#x, repeat(collate(utf8_lcase#x, utf8_binary), 2) AS repeat(collate(utf8_lcase, utf8_binary), 2)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select ascii(utf8_binary), ascii(utf8_lcase) from t5
--- !query analysis
-Project [ascii(utf8_binary#x) AS ascii(utf8_binary)#x, ascii(utf8_lcase#x) AS ascii(utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select ascii(utf8_binary collate utf8_lcase), ascii(utf8_lcase collate utf8_binary) from t5
--- !query analysis
-Project [ascii(collate(utf8_binary#x, utf8_lcase)) AS ascii(collate(utf8_binary, utf8_lcase))#x, ascii(collate(utf8_lcase#x, utf8_binary)) AS ascii(collate(utf8_lcase, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select unbase64(utf8_binary), unbase64(utf8_lcase) from t10
--- !query analysis
-Project [unbase64(utf8_binary#x, false) AS unbase64(utf8_binary)#x, unbase64(utf8_lcase#x, false) AS unbase64(utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t10
- +- Relation spark_catalog.default.t10[utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select unbase64(utf8_binary collate utf8_lcase), unbase64(utf8_lcase collate utf8_binary) from t10
--- !query analysis
-Project [unbase64(collate(utf8_binary#x, utf8_lcase), false) AS unbase64(collate(utf8_binary, utf8_lcase))#x, unbase64(collate(utf8_lcase#x, utf8_binary), false) AS unbase64(collate(utf8_lcase, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t10
- +- Relation spark_catalog.default.t10[utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select chr(ascii) from t6
--- !query analysis
-Project [chr(ascii#xL) AS chr(ascii)#x]
-+- SubqueryAlias spark_catalog.default.t6
- +- Relation spark_catalog.default.t6[ascii#xL] parquet
-
-
--- !query
-select base64(utf8_binary), base64(utf8_lcase) from t5
--- !query analysis
-Project [base64(cast(utf8_binary#x as binary)) AS base64(utf8_binary)#x, base64(cast(utf8_lcase#x as binary)) AS base64(utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select base64(utf8_binary collate utf8_lcase), base64(utf8_lcase collate utf8_binary) from t5
--- !query analysis
-Project [base64(cast(collate(utf8_binary#x, utf8_lcase) as binary)) AS base64(collate(utf8_binary, utf8_lcase))#x, base64(cast(collate(utf8_lcase#x, utf8_binary) as binary)) AS base64(collate(utf8_lcase, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select decode(encode(utf8_binary, 'utf-8'), 'utf-8'), decode(encode(utf8_lcase, 'utf-8'), 'utf-8') from t5
--- !query analysis
-Project [decode(encode(utf8_binary#x, utf-8), utf-8) AS decode(encode(utf8_binary, utf-8), utf-8)#x, decode(encode(utf8_lcase#x, utf-8), utf-8) AS decode(encode(utf8_lcase, utf-8), utf-8)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select decode(encode(utf8_binary collate utf8_lcase, 'utf-8'), 'utf-8'), decode(encode(utf8_lcase collate utf8_binary, 'utf-8'), 'utf-8') from t5
--- !query analysis
-Project [decode(encode(collate(utf8_binary#x, utf8_lcase), utf-8), utf-8) AS decode(encode(collate(utf8_binary, utf8_lcase), utf-8), utf-8)#x, decode(encode(collate(utf8_lcase#x, utf8_binary), utf-8), utf-8) AS decode(encode(collate(utf8_lcase, utf8_binary), utf-8), utf-8)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select format_number(ascii, '###.###') from t7
--- !query analysis
-Project [format_number(ascii#x, ###.###) AS format_number(ascii, ###.###)#x]
-+- SubqueryAlias spark_catalog.default.t7
- +- Relation spark_catalog.default.t7[ascii#x] parquet
-
-
--- !query
-select format_number(ascii, '###.###' collate utf8_lcase) from t7
--- !query analysis
-Project [format_number(ascii#x, collate(###.###, utf8_lcase)) AS format_number(ascii, collate(###.###, utf8_lcase))#x]
-+- SubqueryAlias spark_catalog.default.t7
- +- Relation spark_catalog.default.t7[ascii#x] parquet
-
-
--- !query
-select encode(utf8_binary, 'utf-8'), encode(utf8_lcase, 'utf-8') from t5
--- !query analysis
-Project [encode(utf8_binary#x, utf-8) AS encode(utf8_binary, utf-8)#x, encode(utf8_lcase#x, utf-8) AS encode(utf8_lcase, utf-8)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select encode(utf8_binary collate utf8_lcase, 'utf-8'), encode(utf8_lcase collate utf8_binary, 'utf-8') from t5
--- !query analysis
-Project [encode(collate(utf8_binary#x, utf8_lcase), utf-8) AS encode(collate(utf8_binary, utf8_lcase), utf-8)#x, encode(collate(utf8_lcase#x, utf8_binary), utf-8) AS encode(collate(utf8_lcase, utf8_binary), utf-8)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select to_binary(utf8_binary, 'utf-8'), to_binary(utf8_lcase, 'utf-8') from t5
--- !query analysis
-Project [to_binary(utf8_binary#x, Some(utf-8), false) AS to_binary(utf8_binary, utf-8)#x, to_binary(utf8_lcase#x, Some(utf-8), false) AS to_binary(utf8_lcase, utf-8)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select to_binary(utf8_binary collate utf8_lcase, 'utf-8'), to_binary(utf8_lcase collate utf8_binary, 'utf-8') from t5
--- !query analysis
-Project [to_binary(collate(utf8_binary#x, utf8_lcase), Some(utf-8), false) AS to_binary(collate(utf8_binary, utf8_lcase), utf-8)#x, to_binary(collate(utf8_lcase#x, utf8_binary), Some(utf-8), false) AS to_binary(collate(utf8_lcase, utf8_binary), utf-8)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select sentences(utf8_binary), sentences(utf8_lcase) from t5
--- !query analysis
-Project [sentences(utf8_binary#x, , ) AS sentences(utf8_binary, , )#x, sentences(utf8_lcase#x, , ) AS sentences(utf8_lcase, , )#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select sentences(utf8_binary collate utf8_lcase), sentences(utf8_lcase collate utf8_binary) from t5
--- !query analysis
-Project [sentences(collate(utf8_binary#x, utf8_lcase), , ) AS sentences(collate(utf8_binary, utf8_lcase), , )#x, sentences(collate(utf8_lcase#x, utf8_binary), , ) AS sentences(collate(utf8_lcase, utf8_binary), , )#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select upper(utf8_binary), upper(utf8_lcase) from t5
--- !query analysis
-Project [upper(utf8_binary#x) AS upper(utf8_binary)#x, upper(utf8_lcase#x) AS upper(utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select upper(utf8_binary collate utf8_lcase), upper(utf8_lcase collate utf8_binary) from t5
--- !query analysis
-Project [upper(collate(utf8_binary#x, utf8_lcase)) AS upper(collate(utf8_binary, utf8_lcase))#x, upper(collate(utf8_lcase#x, utf8_binary)) AS upper(collate(utf8_lcase, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select lower(utf8_binary), lower(utf8_lcase) from t5
--- !query analysis
-Project [lower(utf8_binary#x) AS lower(utf8_binary)#x, lower(utf8_lcase#x) AS lower(utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select lower(utf8_binary collate utf8_lcase), lower(utf8_lcase collate utf8_binary) from t5
--- !query analysis
-Project [lower(collate(utf8_binary#x, utf8_lcase)) AS lower(collate(utf8_binary, utf8_lcase))#x, lower(collate(utf8_lcase#x, utf8_binary)) AS lower(collate(utf8_lcase, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select initcap(utf8_binary), initcap(utf8_lcase) from t5
--- !query analysis
-Project [initcap(utf8_binary#x) AS initcap(utf8_binary)#x, initcap(utf8_lcase#x) AS initcap(utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select initcap(utf8_binary collate utf8_lcase), initcap(utf8_lcase collate utf8_binary) from t5
--- !query analysis
-Project [initcap(collate(utf8_binary#x, utf8_lcase)) AS initcap(collate(utf8_binary, utf8_lcase))#x, initcap(collate(utf8_lcase#x, utf8_binary)) AS initcap(collate(utf8_lcase, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select overlay(utf8_binary, utf8_lcase, 2) from t5
--- !query analysis
-Project [overlay(cast(utf8_binary#x as string collate null), cast(utf8_lcase#x as string collate null), 2, -1) AS overlay(utf8_binary, utf8_lcase, 2, -1)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select overlay(s, utf8_binary,1) from t5
--- !query analysis
-Project [overlay(s#x, utf8_binary#x, 1, -1) AS overlay(s, utf8_binary, 1, -1)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select overlay(utf8_binary collate utf8_binary, s collate utf8_lcase, 3) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
- "sqlState" : "42P21",
- "messageParameters" : {
- "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
- }
-}
-
-
--- !query
-select overlay(utf8_binary, utf8_lcase collate utf8_binary, 2) from t5
--- !query analysis
-Project [overlay(utf8_binary#x, collate(utf8_lcase#x, utf8_binary), 2, -1) AS overlay(utf8_binary, collate(utf8_lcase, utf8_binary), 2, -1)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select overlay(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 2) from t5
--- !query analysis
-Project [overlay(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase), 2, -1) AS overlay(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase), 2, -1)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select overlay(utf8_binary, 'a', 2), overlay(utf8_lcase, 'a', 2) from t5
--- !query analysis
-Project [overlay(utf8_binary#x, a, 2, -1) AS overlay(utf8_binary, a, 2, -1)#x, overlay(utf8_lcase#x, a, 2, -1) AS overlay(utf8_lcase, 'a' collate UTF8_LCASE, 2, -1)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select overlay(utf8_binary, 'AaAA' collate utf8_lcase, 2), overlay(utf8_lcase, 'AAa' collate utf8_binary, 2) from t5
--- !query analysis
-Project [overlay(cast(utf8_binary#x as string collate UTF8_LCASE), collate(AaAA, utf8_lcase), 2, -1) AS overlay(utf8_binary, collate(AaAA, utf8_lcase), 2, -1)#x, overlay(cast(utf8_lcase#x as string), collate(AAa, utf8_binary), 2, -1) AS overlay(utf8_lcase, collate(AAa, utf8_binary), 2, -1)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select format_string(format, utf8_binary, utf8_lcase) from t8
--- !query analysis
-Project [format_string(format#x, utf8_binary#x, utf8_lcase#x) AS format_string(format, utf8_binary, utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t8
- +- Relation spark_catalog.default.t8[format#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select format_string(format collate utf8_lcase, utf8_lcase, utf8_binary collate utf8_lcase, 3), format_string(format, utf8_lcase collate utf8_binary, utf8_binary) from t8
--- !query analysis
-Project [format_string(collate(format#x, utf8_lcase), utf8_lcase#x, collate(utf8_binary#x, utf8_lcase), 3) AS format_string(collate(format, utf8_lcase), utf8_lcase, collate(utf8_binary, utf8_lcase), 3)#x, format_string(format#x, collate(utf8_lcase#x, utf8_binary), utf8_binary#x) AS format_string(format, collate(utf8_lcase, utf8_binary), utf8_binary)#x]
-+- SubqueryAlias spark_catalog.default.t8
- +- Relation spark_catalog.default.t8[format#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select format_string(format, utf8_binary, utf8_lcase) from t8
--- !query analysis
-Project [format_string(format#x, utf8_binary#x, utf8_lcase#x) AS format_string(format, utf8_binary, utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t8
- +- Relation spark_catalog.default.t8[format#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select soundex(utf8_binary), soundex(utf8_lcase) from t5
--- !query analysis
-Project [soundex(utf8_binary#x) AS soundex(utf8_binary)#x, soundex(utf8_lcase#x) AS soundex(utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select soundex(utf8_binary collate utf8_lcase), soundex(utf8_lcase collate utf8_binary) from t5
--- !query analysis
-Project [soundex(collate(utf8_binary#x, utf8_lcase)) AS soundex(collate(utf8_binary, utf8_lcase))#x, soundex(collate(utf8_lcase#x, utf8_binary)) AS soundex(collate(utf8_lcase, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select length(utf8_binary), length(utf8_lcase) from t5
--- !query analysis
-Project [length(utf8_binary#x) AS length(utf8_binary)#x, length(utf8_lcase#x) AS length(utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select length(utf8_binary collate utf8_lcase), length(utf8_lcase collate utf8_binary) from t5
--- !query analysis
-Project [length(collate(utf8_binary#x, utf8_lcase)) AS length(collate(utf8_binary, utf8_lcase))#x, length(collate(utf8_lcase#x, utf8_binary)) AS length(collate(utf8_lcase, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select bit_length(utf8_binary), bit_length(utf8_lcase) from t5
--- !query analysis
-Project [bit_length(utf8_binary#x) AS bit_length(utf8_binary)#x, bit_length(utf8_lcase#x) AS bit_length(utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select bit_length(utf8_binary collate utf8_lcase), bit_length(utf8_lcase collate utf8_binary) from t5
--- !query analysis
-Project [bit_length(collate(utf8_binary#x, utf8_lcase)) AS bit_length(collate(utf8_binary, utf8_lcase))#x, bit_length(collate(utf8_lcase#x, utf8_binary)) AS bit_length(collate(utf8_lcase, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select octet_length(utf8_binary), octet_length(utf8_lcase) from t5
--- !query analysis
-Project [octet_length(utf8_binary#x) AS octet_length(utf8_binary)#x, octet_length(utf8_lcase#x) AS octet_length(utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select octet_length(utf8_binary collate utf8_lcase), octet_length(utf8_lcase collate utf8_binary) from t5
--- !query analysis
-Project [octet_length(collate(utf8_binary#x, utf8_lcase)) AS octet_length(collate(utf8_binary, utf8_lcase))#x, octet_length(collate(utf8_lcase#x, utf8_binary)) AS octet_length(collate(utf8_lcase, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select octet_length(utf8_binary collate utf8_lcase_rtrim), octet_length(utf8_lcase collate utf8_binary_rtrim) from t5
--- !query analysis
-Project [octet_length(collate(utf8_binary#x, utf8_lcase_rtrim)) AS octet_length(collate(utf8_binary, utf8_lcase_rtrim))#x, octet_length(collate(utf8_lcase#x, utf8_binary_rtrim)) AS octet_length(collate(utf8_lcase, utf8_binary_rtrim))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select luhn_check(num) from t9
--- !query analysis
-Project [luhn_check(cast(num#xL as string)) AS luhn_check(num)#x]
-+- SubqueryAlias spark_catalog.default.t9
- +- Relation spark_catalog.default.t9[num#xL] parquet
-
-
--- !query
-select levenshtein(utf8_binary, utf8_lcase) from t5
--- !query analysis
-Project [levenshtein(cast(utf8_binary#x as string collate null), cast(utf8_lcase#x as string collate null), None) AS levenshtein(utf8_binary, utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select levenshtein(s, utf8_binary) from t5
--- !query analysis
-Project [levenshtein(s#x, utf8_binary#x, None) AS levenshtein(s, utf8_binary)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select levenshtein(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
- "sqlState" : "42P21",
- "messageParameters" : {
- "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
- }
-}
-
-
--- !query
-select levenshtein(utf8_binary, utf8_lcase collate utf8_binary) from t5
--- !query analysis
-Project [levenshtein(utf8_binary#x, collate(utf8_lcase#x, utf8_binary), None) AS levenshtein(utf8_binary, collate(utf8_lcase, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select levenshtein(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5
--- !query analysis
-Project [levenshtein(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase), None) AS levenshtein(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select levenshtein(utf8_binary, 'a'), levenshtein(utf8_lcase, 'a') from t5
--- !query analysis
-Project [levenshtein(utf8_binary#x, a, None) AS levenshtein(utf8_binary, a)#x, levenshtein(utf8_lcase#x, a, None) AS levenshtein(utf8_lcase, 'a' collate UTF8_LCASE)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select levenshtein(utf8_binary, 'AaAA' collate utf8_lcase, 3), levenshtein(utf8_lcase, 'AAa' collate utf8_binary, 4) from t5
--- !query analysis
-Project [levenshtein(cast(utf8_binary#x as string collate UTF8_LCASE), collate(AaAA, utf8_lcase), Some(3)) AS levenshtein(utf8_binary, collate(AaAA, utf8_lcase), 3)#x, levenshtein(cast(utf8_lcase#x as string), collate(AAa, utf8_binary), Some(4)) AS levenshtein(utf8_lcase, collate(AAa, utf8_binary), 4)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select is_valid_utf8(utf8_binary), is_valid_utf8(utf8_lcase) from t5
--- !query analysis
-Project [is_valid_utf8(utf8_binary#x) AS is_valid_utf8(utf8_binary)#x, is_valid_utf8(utf8_lcase#x) AS is_valid_utf8(utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select is_valid_utf8(utf8_binary collate utf8_lcase), is_valid_utf8(utf8_lcase collate utf8_binary) from t5
--- !query analysis
-Project [is_valid_utf8(collate(utf8_binary#x, utf8_lcase)) AS is_valid_utf8(collate(utf8_binary, utf8_lcase))#x, is_valid_utf8(collate(utf8_lcase#x, utf8_binary)) AS is_valid_utf8(collate(utf8_lcase, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select is_valid_utf8(utf8_binary collate utf8_lcase_rtrim), is_valid_utf8(utf8_lcase collate utf8_binary_rtrim) from t5
--- !query analysis
-Project [is_valid_utf8(collate(utf8_binary#x, utf8_lcase_rtrim)) AS is_valid_utf8(collate(utf8_binary, utf8_lcase_rtrim))#x, is_valid_utf8(collate(utf8_lcase#x, utf8_binary_rtrim)) AS is_valid_utf8(collate(utf8_lcase, utf8_binary_rtrim))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select make_valid_utf8(utf8_binary), make_valid_utf8(utf8_lcase) from t5
--- !query analysis
-Project [make_valid_utf8(utf8_binary#x) AS make_valid_utf8(utf8_binary)#x, make_valid_utf8(utf8_lcase#x) AS make_valid_utf8(utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select make_valid_utf8(utf8_binary collate utf8_lcase), make_valid_utf8(utf8_lcase collate utf8_binary) from t5
--- !query analysis
-Project [make_valid_utf8(collate(utf8_binary#x, utf8_lcase)) AS make_valid_utf8(collate(utf8_binary, utf8_lcase))#x, make_valid_utf8(collate(utf8_lcase#x, utf8_binary)) AS make_valid_utf8(collate(utf8_lcase, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select make_valid_utf8(utf8_binary collate utf8_lcase_rtrim), make_valid_utf8(utf8_lcase collate utf8_binary_rtrim) from t5
--- !query analysis
-Project [make_valid_utf8(collate(utf8_binary#x, utf8_lcase_rtrim)) AS make_valid_utf8(collate(utf8_binary, utf8_lcase_rtrim))#x, make_valid_utf8(collate(utf8_lcase#x, utf8_binary_rtrim)) AS make_valid_utf8(collate(utf8_lcase, utf8_binary_rtrim))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select validate_utf8(utf8_binary), validate_utf8(utf8_lcase) from t5
--- !query analysis
-Project [validate_utf8(utf8_binary#x) AS validate_utf8(utf8_binary)#x, validate_utf8(utf8_lcase#x) AS validate_utf8(utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select validate_utf8(utf8_binary collate utf8_lcase), validate_utf8(utf8_lcase collate utf8_binary) from t5
--- !query analysis
-Project [validate_utf8(collate(utf8_binary#x, utf8_lcase)) AS validate_utf8(collate(utf8_binary, utf8_lcase))#x, validate_utf8(collate(utf8_lcase#x, utf8_binary)) AS validate_utf8(collate(utf8_lcase, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select validate_utf8(utf8_binary collate utf8_lcase_rtrim), validate_utf8(utf8_lcase collate utf8_binary_rtrim) from t5
--- !query analysis
-Project [validate_utf8(collate(utf8_binary#x, utf8_lcase_rtrim)) AS validate_utf8(collate(utf8_binary, utf8_lcase_rtrim))#x, validate_utf8(collate(utf8_lcase#x, utf8_binary_rtrim)) AS validate_utf8(collate(utf8_lcase, utf8_binary_rtrim))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select try_validate_utf8(utf8_binary), try_validate_utf8(utf8_lcase) from t5
--- !query analysis
-Project [try_validate_utf8(utf8_binary#x) AS try_validate_utf8(utf8_binary)#x, try_validate_utf8(utf8_lcase#x) AS try_validate_utf8(utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select try_validate_utf8(utf8_binary collate utf8_lcase), try_validate_utf8(utf8_lcase collate utf8_binary) from t5
--- !query analysis
-Project [try_validate_utf8(collate(utf8_binary#x, utf8_lcase)) AS try_validate_utf8(collate(utf8_binary, utf8_lcase))#x, try_validate_utf8(collate(utf8_lcase#x, utf8_binary)) AS try_validate_utf8(collate(utf8_lcase, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select try_validate_utf8(utf8_binary collate utf8_lcase_rtrim), try_validate_utf8(utf8_lcase collate utf8_binary_rtrim) from t5
--- !query analysis
-Project [try_validate_utf8(collate(utf8_binary#x, utf8_lcase_rtrim)) AS try_validate_utf8(collate(utf8_binary, utf8_lcase_rtrim))#x, try_validate_utf8(collate(utf8_lcase#x, utf8_binary_rtrim)) AS try_validate_utf8(collate(utf8_lcase, utf8_binary_rtrim))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select substr(utf8_binary, 2, 2), substr(utf8_lcase, 2, 2) from t5
--- !query analysis
-Project [substr(utf8_binary#x, 2, 2) AS substr(utf8_binary, 2, 2)#x, substr(utf8_lcase#x, 2, 2) AS substr(utf8_lcase, 2, 2)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select substr(utf8_binary collate utf8_lcase, 2, 2), substr(utf8_lcase collate utf8_binary, 2, 2) from t5
--- !query analysis
-Project [substr(collate(utf8_binary#x, utf8_lcase), 2, 2) AS substr(collate(utf8_binary, utf8_lcase), 2, 2)#x, substr(collate(utf8_lcase#x, utf8_binary), 2, 2) AS substr(collate(utf8_lcase, utf8_binary), 2, 2)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select right(utf8_binary, 2), right(utf8_lcase, 2) from t5
--- !query analysis
-Project [right(utf8_binary#x, 2) AS right(utf8_binary, 2)#x, right(utf8_lcase#x, 2) AS right(utf8_lcase, 2)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select right(utf8_binary collate utf8_lcase, 2), right(utf8_lcase collate utf8_binary, 2) from t5
--- !query analysis
-Project [right(collate(utf8_binary#x, utf8_lcase), 2) AS right(collate(utf8_binary, utf8_lcase), 2)#x, right(collate(utf8_lcase#x, utf8_binary), 2) AS right(collate(utf8_lcase, utf8_binary), 2)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select left(utf8_binary, '2' collate utf8_lcase), left(utf8_lcase, 2) from t5
--- !query analysis
-Project [left(utf8_binary#x, cast(collate(2, utf8_lcase) as int)) AS left(utf8_binary, collate(2, utf8_lcase))#x, left(utf8_lcase#x, 2) AS left(utf8_lcase, 2)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select left(utf8_binary collate utf8_lcase, 2), left(utf8_lcase collate utf8_binary, 2) from t5
--- !query analysis
-Project [left(collate(utf8_binary#x, utf8_lcase), 2) AS left(collate(utf8_binary, utf8_lcase), 2)#x, left(collate(utf8_lcase#x, utf8_binary), 2) AS left(collate(utf8_lcase, utf8_binary), 2)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select rpad(utf8_binary, 8, utf8_lcase) from t5
--- !query analysis
-Project [rpad(cast(utf8_binary#x as string collate null), 8, cast(utf8_lcase#x as string collate null)) AS rpad(utf8_binary, 8, utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select rpad(s, 8, utf8_binary) from t5
--- !query analysis
-Project [rpad(s#x, 8, utf8_binary#x) AS rpad(s, 8, utf8_binary)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select rpad(utf8_binary collate utf8_binary, 8, s collate utf8_lcase) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
- "sqlState" : "42P21",
- "messageParameters" : {
- "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
- }
-}
-
-
--- !query
-select rpad(utf8_binary, 8, utf8_lcase collate utf8_binary) from t5
--- !query analysis
-Project [rpad(utf8_binary#x, 8, collate(utf8_lcase#x, utf8_binary)) AS rpad(utf8_binary, 8, collate(utf8_lcase, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select rpad(utf8_binary collate utf8_lcase, 8, utf8_lcase collate utf8_lcase) from t5
--- !query analysis
-Project [rpad(collate(utf8_binary#x, utf8_lcase), 8, collate(utf8_lcase#x, utf8_lcase)) AS rpad(collate(utf8_binary, utf8_lcase), 8, collate(utf8_lcase, utf8_lcase))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select lpad(utf8_binary collate utf8_binary_rtrim, 8, utf8_lcase collate utf8_binary_rtrim) from t5
--- !query analysis
-Project [lpad(collate(utf8_binary#x, utf8_binary_rtrim), 8, collate(utf8_lcase#x, utf8_binary_rtrim)) AS lpad(collate(utf8_binary, utf8_binary_rtrim), 8, collate(utf8_lcase, utf8_binary_rtrim))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select rpad(utf8_binary, 8, 'a'), rpad(utf8_lcase, 8, 'a') from t5
--- !query analysis
-Project [rpad(utf8_binary#x, 8, a) AS rpad(utf8_binary, 8, a)#x, rpad(utf8_lcase#x, 8, a) AS rpad(utf8_lcase, 8, 'a' collate UTF8_LCASE)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select rpad(utf8_binary, 8, 'AaAA' collate utf8_lcase), rpad(utf8_lcase, 8, 'AAa' collate utf8_binary) from t5
--- !query analysis
-Project [rpad(cast(utf8_binary#x as string collate UTF8_LCASE), 8, collate(AaAA, utf8_lcase)) AS rpad(utf8_binary, 8, collate(AaAA, utf8_lcase))#x, rpad(cast(utf8_lcase#x as string), 8, collate(AAa, utf8_binary)) AS rpad(utf8_lcase, 8, collate(AAa, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select lpad(utf8_binary, 8, utf8_lcase) from t5
--- !query analysis
-Project [lpad(cast(utf8_binary#x as string collate null), 8, cast(utf8_lcase#x as string collate null)) AS lpad(utf8_binary, 8, utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select lpad(s, 8, utf8_binary) from t5
--- !query analysis
-Project [lpad(s#x, 8, utf8_binary#x) AS lpad(s, 8, utf8_binary)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select lpad(utf8_binary collate utf8_binary, 8, s collate utf8_lcase) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
- "sqlState" : "42P21",
- "messageParameters" : {
- "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
- }
-}
-
-
--- !query
-select lpad(utf8_binary, 8, utf8_lcase collate utf8_binary) from t5
--- !query analysis
-Project [lpad(utf8_binary#x, 8, collate(utf8_lcase#x, utf8_binary)) AS lpad(utf8_binary, 8, collate(utf8_lcase, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select lpad(utf8_binary collate utf8_lcase, 8, utf8_lcase collate utf8_lcase) from t5
--- !query analysis
-Project [lpad(collate(utf8_binary#x, utf8_lcase), 8, collate(utf8_lcase#x, utf8_lcase)) AS lpad(collate(utf8_binary, utf8_lcase), 8, collate(utf8_lcase, utf8_lcase))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select lpad(utf8_binary collate utf8_binary_rtrim, 8, utf8_lcase collate utf8_binary_rtrim) from t5
--- !query analysis
-Project [lpad(collate(utf8_binary#x, utf8_binary_rtrim), 8, collate(utf8_lcase#x, utf8_binary_rtrim)) AS lpad(collate(utf8_binary, utf8_binary_rtrim), 8, collate(utf8_lcase, utf8_binary_rtrim))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select lpad(utf8_binary, 8, 'a'), lpad(utf8_lcase, 8, 'a') from t5
--- !query analysis
-Project [lpad(utf8_binary#x, 8, a) AS lpad(utf8_binary, 8, a)#x, lpad(utf8_lcase#x, 8, a) AS lpad(utf8_lcase, 8, 'a' collate UTF8_LCASE)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select lpad(utf8_binary, 8, 'AaAA' collate utf8_lcase), lpad(utf8_lcase, 8, 'AAa' collate utf8_binary) from t5
--- !query analysis
-Project [lpad(cast(utf8_binary#x as string collate UTF8_LCASE), 8, collate(AaAA, utf8_lcase)) AS lpad(utf8_binary, 8, collate(AaAA, utf8_lcase))#x, lpad(cast(utf8_lcase#x as string), 8, collate(AAa, utf8_binary)) AS lpad(utf8_lcase, 8, collate(AAa, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select locate(utf8_binary, utf8_lcase) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION",
- "sqlState" : "42P22",
- "messageParameters" : {
- "expr" : "\"locate(utf8_binary, utf8_lcase, 1)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 38,
- "fragment" : "locate(utf8_binary, utf8_lcase)"
- } ]
-}
-
-
--- !query
-select locate(s, utf8_binary) from t5
--- !query analysis
-Project [locate(s#x, utf8_binary#x, 1) AS locate(s, utf8_binary, 1)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select locate(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
- "sqlState" : "42P21",
- "messageParameters" : {
- "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
- }
-}
-
-
--- !query
-select locate(utf8_binary, utf8_lcase collate utf8_binary) from t5
--- !query analysis
-Project [locate(utf8_binary#x, collate(utf8_lcase#x, utf8_binary), 1) AS locate(utf8_binary, collate(utf8_lcase, utf8_binary), 1)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select locate(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 3) from t5
--- !query analysis
-Project [locate(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase), 3) AS locate(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase), 3)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select locate(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai, 3) from t5
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"collate(utf8_binary, unicode_ai)\"",
- "inputType" : "\"STRING COLLATE UNICODE_AI\"",
- "paramIndex" : "first",
- "requiredType" : "\"STRING\"",
- "sqlExpr" : "\"locate(collate(utf8_binary, unicode_ai), collate(utf8_lcase, unicode_ai), 3)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 79,
- "fragment" : "locate(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai, 3)"
- } ]
-}
-
-
--- !query
-select locate(utf8_binary, 'a'), locate(utf8_lcase, 'a') from t5
--- !query analysis
-Project [locate(utf8_binary#x, a, 1) AS locate(utf8_binary, a, 1)#x, locate(utf8_lcase#x, a, 1) AS locate(utf8_lcase, 'a' collate UTF8_LCASE, 1)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select locate(utf8_binary, 'AaAA' collate utf8_lcase, 4), locate(utf8_lcase, 'AAa' collate utf8_binary, 4) from t5
--- !query analysis
-Project [locate(cast(utf8_binary#x as string collate UTF8_LCASE), collate(AaAA, utf8_lcase), 4) AS locate(utf8_binary, collate(AaAA, utf8_lcase), 4)#x, locate(cast(utf8_lcase#x as string), collate(AAa, utf8_binary), 4) AS locate(utf8_lcase, collate(AAa, utf8_binary), 4)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select locate(utf8_binary, 'AaAA ' collate utf8_binary_rtrim, 4), locate(utf8_lcase, 'AAa ' collate utf8_binary, 4) from t5
--- !query analysis
-Project [locate(cast(utf8_binary#x as string collate UTF8_BINARY_RTRIM), collate(AaAA , utf8_binary_rtrim), 4) AS locate(utf8_binary, collate(AaAA , utf8_binary_rtrim), 4)#x, locate(cast(utf8_lcase#x as string), collate(AAa , utf8_binary), 4) AS locate(utf8_lcase, collate(AAa , utf8_binary), 4)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select TRIM(utf8_binary, utf8_lcase) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION",
- "sqlState" : "42P22",
- "messageParameters" : {
- "expr" : "\"TRIM(BOTH utf8_binary FROM utf8_lcase)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 36,
- "fragment" : "TRIM(utf8_binary, utf8_lcase)"
- } ]
-}
-
-
--- !query
-select TRIM(s, utf8_binary) from t5
--- !query analysis
-Project [trim(utf8_binary#x, Some(s#x)) AS TRIM(BOTH s FROM utf8_binary)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select TRIM(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
- "sqlState" : "42P21",
- "messageParameters" : {
- "explicitTypes" : "\"STRING COLLATE UTF8_LCASE\", \"STRING\""
- }
-}
-
-
--- !query
-select TRIM(utf8_binary, utf8_lcase collate utf8_binary) from t5
--- !query analysis
-Project [trim(collate(utf8_lcase#x, utf8_binary), Some(utf8_binary#x)) AS TRIM(BOTH utf8_binary FROM collate(utf8_lcase, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select TRIM(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5
--- !query analysis
-Project [trim(collate(utf8_lcase#x, utf8_lcase), Some(collate(utf8_binary#x, utf8_lcase))) AS TRIM(BOTH collate(utf8_binary, utf8_lcase) FROM collate(utf8_lcase, utf8_lcase))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select TRIM(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t5
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"collate(utf8_lcase, unicode_ai)\"",
- "inputType" : "\"STRING COLLATE UNICODE_AI\"",
- "paramIndex" : "first",
- "requiredType" : "\"STRING\"",
- "sqlExpr" : "\"TRIM(BOTH collate(utf8_binary, unicode_ai) FROM collate(utf8_lcase, unicode_ai))\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 74,
- "fragment" : "TRIM(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai)"
- } ]
-}
-
-
--- !query
-select TRIM(utf8_binary collate utf8_binary_rtrim, utf8_lcase collate utf8_binary_rtrim) from t5
--- !query analysis
-Project [trim(collate(utf8_lcase#x, utf8_binary_rtrim), Some(collate(utf8_binary#x, utf8_binary_rtrim))) AS TRIM(BOTH collate(utf8_binary, utf8_binary_rtrim) FROM collate(utf8_lcase, utf8_binary_rtrim))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select TRIM('ABc', utf8_binary), TRIM('ABc', utf8_lcase) from t5
--- !query analysis
-Project [trim(utf8_binary#x, Some(ABc)) AS TRIM(BOTH ABc FROM utf8_binary)#x, trim(utf8_lcase#x, Some(ABc)) AS TRIM(BOTH 'ABc' collate UTF8_LCASE FROM utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select TRIM('ABc' collate utf8_lcase, utf8_binary), TRIM('AAa' collate utf8_binary, utf8_lcase) from t5
--- !query analysis
-Project [trim(cast(utf8_binary#x as string collate UTF8_LCASE), Some(collate(ABc, utf8_lcase))) AS TRIM(BOTH collate(ABc, utf8_lcase) FROM utf8_binary)#x, trim(cast(utf8_lcase#x as string), Some(collate(AAa, utf8_binary))) AS TRIM(BOTH collate(AAa, utf8_binary) FROM utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select BTRIM(utf8_binary, utf8_lcase) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION",
- "sqlState" : "42P22",
- "messageParameters" : {
- "expr" : "\"TRIM(BOTH utf8_lcase FROM utf8_binary)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 37,
- "fragment" : "BTRIM(utf8_binary, utf8_lcase)"
- } ]
-}
-
-
--- !query
-select BTRIM(s, utf8_binary) from t5
--- !query analysis
-Project [btrim(s#x, utf8_binary#x) AS btrim(s, utf8_binary)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select BTRIM(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
- "sqlState" : "42P21",
- "messageParameters" : {
- "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\""
- }
-}
-
-
--- !query
-select BTRIM(utf8_binary, utf8_lcase collate utf8_binary) from t5
--- !query analysis
-Project [btrim(utf8_binary#x, collate(utf8_lcase#x, utf8_binary)) AS btrim(utf8_binary, collate(utf8_lcase, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select BTRIM(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5
--- !query analysis
-Project [btrim(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase)) AS btrim(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select BTRIM(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t5
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"collate(utf8_binary, unicode_ai)\"",
- "inputType" : "\"STRING COLLATE UNICODE_AI\"",
- "paramIndex" : "first",
- "requiredType" : "\"STRING\"",
- "sqlExpr" : "\"TRIM(BOTH collate(utf8_lcase, unicode_ai) FROM collate(utf8_binary, unicode_ai))\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 75,
- "fragment" : "BTRIM(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai)"
- } ]
-}
-
-
--- !query
-select BTRIM(utf8_binary collate utf8_binary_rtrim, utf8_lcase collate utf8_binary_rtrim) from t5
--- !query analysis
-Project [btrim(collate(utf8_binary#x, utf8_binary_rtrim), collate(utf8_lcase#x, utf8_binary_rtrim)) AS btrim(collate(utf8_binary, utf8_binary_rtrim), collate(utf8_lcase, utf8_binary_rtrim))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select BTRIM('ABc', utf8_binary), BTRIM('ABc', utf8_lcase) from t5
--- !query analysis
-Project [btrim(ABc, utf8_binary#x) AS btrim(ABc, utf8_binary)#x, btrim(ABc, utf8_lcase#x) AS btrim(ABc, utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select BTRIM('ABc' collate utf8_lcase, utf8_binary), BTRIM('AAa' collate utf8_binary, utf8_lcase) from t5
--- !query analysis
-Project [btrim(collate(ABc, utf8_lcase), utf8_binary#x) AS btrim(collate(ABc, utf8_lcase), utf8_binary)#x, btrim(collate(AAa, utf8_binary), utf8_lcase#x) AS btrim(collate(AAa, utf8_binary), utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select LTRIM(utf8_binary, utf8_lcase) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION",
- "sqlState" : "42P22",
- "messageParameters" : {
- "expr" : "\"TRIM(LEADING utf8_binary FROM utf8_lcase)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 37,
- "fragment" : "LTRIM(utf8_binary, utf8_lcase)"
- } ]
-}
-
-
--- !query
-select LTRIM(s, utf8_binary) from t5
--- !query analysis
-Project [ltrim(utf8_binary#x, Some(s#x)) AS TRIM(LEADING s FROM utf8_binary)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select LTRIM(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
- "sqlState" : "42P21",
- "messageParameters" : {
- "explicitTypes" : "\"STRING COLLATE UTF8_LCASE\", \"STRING\""
- }
-}
-
-
--- !query
-select LTRIM(utf8_binary, utf8_lcase collate utf8_binary) from t5
--- !query analysis
-Project [ltrim(collate(utf8_lcase#x, utf8_binary), Some(utf8_binary#x)) AS TRIM(LEADING utf8_binary FROM collate(utf8_lcase, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select LTRIM(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5
--- !query analysis
-Project [ltrim(collate(utf8_lcase#x, utf8_lcase), Some(collate(utf8_binary#x, utf8_lcase))) AS TRIM(LEADING collate(utf8_binary, utf8_lcase) FROM collate(utf8_lcase, utf8_lcase))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select LTRIM(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t5
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"collate(utf8_lcase, unicode_ai)\"",
- "inputType" : "\"STRING COLLATE UNICODE_AI\"",
- "paramIndex" : "first",
- "requiredType" : "\"STRING\"",
- "sqlExpr" : "\"TRIM(LEADING collate(utf8_binary, unicode_ai) FROM collate(utf8_lcase, unicode_ai))\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 75,
- "fragment" : "LTRIM(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai)"
- } ]
-}
-
-
--- !query
-select LTRIM(utf8_binary collate utf8_binary_rtrim, utf8_lcase collate utf8_binary_rtrim) from t5
--- !query analysis
-Project [ltrim(collate(utf8_lcase#x, utf8_binary_rtrim), Some(collate(utf8_binary#x, utf8_binary_rtrim))) AS TRIM(LEADING collate(utf8_binary, utf8_binary_rtrim) FROM collate(utf8_lcase, utf8_binary_rtrim))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select LTRIM('ABc', utf8_binary), LTRIM('ABc', utf8_lcase) from t5
--- !query analysis
-Project [ltrim(utf8_binary#x, Some(ABc)) AS TRIM(LEADING ABc FROM utf8_binary)#x, ltrim(utf8_lcase#x, Some(ABc)) AS TRIM(LEADING 'ABc' collate UTF8_LCASE FROM utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select LTRIM('ABc' collate utf8_lcase, utf8_binary), LTRIM('AAa' collate utf8_binary, utf8_lcase) from t5
--- !query analysis
-Project [ltrim(cast(utf8_binary#x as string collate UTF8_LCASE), Some(collate(ABc, utf8_lcase))) AS TRIM(LEADING collate(ABc, utf8_lcase) FROM utf8_binary)#x, ltrim(cast(utf8_lcase#x as string), Some(collate(AAa, utf8_binary))) AS TRIM(LEADING collate(AAa, utf8_binary) FROM utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select RTRIM(utf8_binary, utf8_lcase) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION",
- "sqlState" : "42P22",
- "messageParameters" : {
- "expr" : "\"TRIM(TRAILING utf8_binary FROM utf8_lcase)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 37,
- "fragment" : "RTRIM(utf8_binary, utf8_lcase)"
- } ]
-}
-
-
--- !query
-select RTRIM(s, utf8_binary) from t5
--- !query analysis
-Project [rtrim(utf8_binary#x, Some(s#x)) AS TRIM(TRAILING s FROM utf8_binary)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select RTRIM(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5
--- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "COLLATION_MISMATCH.EXPLICIT",
- "sqlState" : "42P21",
- "messageParameters" : {
- "explicitTypes" : "\"STRING COLLATE UTF8_LCASE\", \"STRING\""
- }
-}
-
-
--- !query
-select RTRIM(utf8_binary, utf8_lcase collate utf8_binary) from t5
--- !query analysis
-Project [rtrim(collate(utf8_lcase#x, utf8_binary), Some(utf8_binary#x)) AS TRIM(TRAILING utf8_binary FROM collate(utf8_lcase, utf8_binary))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select RTRIM(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5
--- !query analysis
-Project [rtrim(collate(utf8_lcase#x, utf8_lcase), Some(collate(utf8_binary#x, utf8_lcase))) AS TRIM(TRAILING collate(utf8_binary, utf8_lcase) FROM collate(utf8_lcase, utf8_lcase))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select RTRIM(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t5
--- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
- "sqlState" : "42K09",
- "messageParameters" : {
- "inputSql" : "\"collate(utf8_lcase, unicode_ai)\"",
- "inputType" : "\"STRING COLLATE UNICODE_AI\"",
- "paramIndex" : "first",
- "requiredType" : "\"STRING\"",
- "sqlExpr" : "\"TRIM(TRAILING collate(utf8_binary, unicode_ai) FROM collate(utf8_lcase, unicode_ai))\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 75,
- "fragment" : "RTRIM(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai)"
- } ]
-}
-
-
--- !query
-select RTRIM(utf8_binary collate utf8_binary_rtrim, utf8_lcase collate utf8_binary_rtrim) from t5
--- !query analysis
-Project [rtrim(collate(utf8_lcase#x, utf8_binary_rtrim), Some(collate(utf8_binary#x, utf8_binary_rtrim))) AS TRIM(TRAILING collate(utf8_binary, utf8_binary_rtrim) FROM collate(utf8_lcase, utf8_binary_rtrim))#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select RTRIM('ABc', utf8_binary), RTRIM('ABc', utf8_lcase) from t5
--- !query analysis
-Project [rtrim(utf8_binary#x, Some(ABc)) AS TRIM(TRAILING ABc FROM utf8_binary)#x, rtrim(utf8_lcase#x, Some(ABc)) AS TRIM(TRAILING 'ABc' collate UTF8_LCASE FROM utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select RTRIM('ABc' collate utf8_lcase, utf8_binary), RTRIM('AAa' collate utf8_binary, utf8_lcase) from t5
--- !query analysis
-Project [rtrim(cast(utf8_binary#x as string collate UTF8_LCASE), Some(collate(ABc, utf8_lcase))) AS TRIM(TRAILING collate(ABc, utf8_lcase) FROM utf8_binary)#x, rtrim(cast(utf8_lcase#x as string), Some(collate(AAa, utf8_binary))) AS TRIM(TRAILING collate(AAa, utf8_binary) FROM utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select concat_ws(' ', utf8_lcase, utf8_lcase) from t5
--- !query analysis
-Project [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
-+- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select `concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)` from (
- select concat_ws(' ', utf8_lcase, utf8_lcase) from t5
-)
--- !query analysis
-Project [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
-+- SubqueryAlias __auto_generated_subquery_name
- +- Project [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
- +- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select * from (
- select concat_ws(' ', utf8_lcase, utf8_lcase) from t5
-)
--- !query analysis
-Project [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
-+- SubqueryAlias __auto_generated_subquery_name
- +- Project [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
- +- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select subq1.* from (
- select concat_ws(' ', utf8_lcase, utf8_lcase) from t5
-) AS subq1
--- !query analysis
-Project [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
-+- SubqueryAlias subq1
- +- Project [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
- +- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-with cte as (
- select concat_ws(' ', utf8_lcase, utf8_lcase) from t5
-)
-select * from cte
--- !query analysis
-WithCTE
-:- CTERelationDef xxxx, false
-: +- SubqueryAlias cte
-: +- Project [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
-: +- SubqueryAlias spark_catalog.default.t5
-: +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-+- Project [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
- +- SubqueryAlias cte
- +- CTERelationRef xxxx, true, [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x], false, false
-
-
--- !query
-select * from values (1) where exists (
- select concat_ws(' ', utf8_lcase, utf8_lcase) from t5
-)
--- !query analysis
-Project [col1#x]
-+- Filter exists#x []
- : +- Project [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
- : +- SubqueryAlias spark_catalog.default.t5
- : +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
- +- LocalRelation [col1#x]
-
-
--- !query
-select (
- select concat_ws(' ', utf8_lcase, utf8_lcase) from t5 limit 1
-)
--- !query analysis
-Project [scalar-subquery#x [] AS scalarsubquery()#x]
-: +- GlobalLimit 1
-: +- LocalLimit 1
-: +- Project [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
-: +- SubqueryAlias spark_catalog.default.t5
-: +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-+- OneRowRelation
-
-
--- !query
-select (
- with cte as (
- select concat_ws(' ', utf8_lcase, utf8_lcase) from t5
- )
- select * from cte limit 1
-)
--- !query analysis
-Project [scalar-subquery#x [] AS scalarsubquery()#x]
-: +- WithCTE
-: :- CTERelationDef xxxx, false
-: : +- SubqueryAlias cte
-: : +- Project [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
-: : +- SubqueryAlias spark_catalog.default.t5
-: : +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-: +- GlobalLimit 1
-: +- LocalLimit 1
-: +- Project [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
-: +- SubqueryAlias cte
-: +- CTERelationRef xxxx, true, [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x], false, false
-+- OneRowRelation
-
-
--- !query
-select * from (
- select concat_ws(' ', utf8_lcase, utf8_lcase) from t5 limit 1
-)
-where (
- `concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)` == 'aaa'
-)
--- !query analysis
-Project [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
-+- Filter (concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x = aaa)
- +- SubqueryAlias __auto_generated_subquery_name
- +- GlobalLimit 1
- +- LocalLimit 1
- +- Project [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
- +- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select lower(`concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)`) from (
- select concat_ws(' ', utf8_lcase, utf8_lcase) from t5
- group by 1
- order by 1
-)
--- !query analysis
-Project [lower(concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x) AS lower(concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase))#x]
-+- SubqueryAlias __auto_generated_subquery_name
- +- Sort [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x ASC NULLS FIRST], true
- +- Aggregate [concat_ws( , utf8_lcase#x, utf8_lcase#x)], [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
- +- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select lower(`concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)`) from (
- select concat_ws(' ', utf8_lcase, utf8_lcase) from t5
- group by 1
- order by max(concat_ws(' ', utf8_lcase, utf8_lcase))
-)
--- !query analysis
-Project [lower(concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x) AS lower(concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase))#x]
-+- SubqueryAlias __auto_generated_subquery_name
- +- Project [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
- +- Sort [max(concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase))#x ASC NULLS FIRST], true
- +- Aggregate [concat_ws( , utf8_lcase#x, utf8_lcase#x)], [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x, max(concat_ws( , utf8_lcase#x, utf8_lcase#x)) AS max(concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase))#x]
- +- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-create temporary view v1 as (
- select concat_ws(' ', utf8_lcase, utf8_lcase) from t5
-)
--- !query analysis
-CreateViewCommand `v1`, (
- select concat_ws(' ', utf8_lcase, utf8_lcase) from t5
-), false, false, LocalTempView, UNSUPPORTED, true
- +- Project [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
- +- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select * from v1
--- !query analysis
-Project [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
-+- SubqueryAlias v1
- +- View (`v1`, [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x])
- +- Project [cast(concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x as string collate UTF8_LCASE) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
- +- Project [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
- +- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-select `concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)` from v1
--- !query analysis
-Project [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
-+- SubqueryAlias v1
- +- View (`v1`, [concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x])
- +- Project [cast(concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x as string collate UTF8_LCASE) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
- +- Project [concat_ws( , utf8_lcase#x, utf8_lcase#x) AS concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)#x]
- +- SubqueryAlias spark_catalog.default.t5
- +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet
-
-
--- !query
-drop view v1
--- !query analysis
-DropTempViewCommand v1
-
-
--- !query
-drop table t5
--- !query analysis
-DropTable false, false
-+- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t5
-
-
--- !query
-drop table t6
--- !query analysis
-DropTable false, false
-+- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t6
-
-
--- !query
-drop table t7
--- !query analysis
-DropTable false, false
-+- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t7
-
-
--- !query
-drop table t8
--- !query analysis
-DropTable false, false
-+- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t8
-
-
--- !query
-drop table t9
--- !query analysis
-DropTable false, false
-+- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t9
-
-
--- !query
-drop table t10
--- !query analysis
-DropTable false, false
-+- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t10
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/columnresolution.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/columnresolution.sql.out
index e0ef6c1248aea..5a2447d440035 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/columnresolution.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/columnresolution.sql.out
@@ -337,7 +337,7 @@ CreateDataSourceTableCommand `spark_catalog`.`mydb1`.`t5`, false
INSERT INTO t5 VALUES(1, (2, 3))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/mydb1.db/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/mydb1.db/t5], Append, `spark_catalog`.`mydb1`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/mydb1.db/t5), [i1, t5]
-+- Project [cast(col1#x as int) AS i1#x, named_struct(i1, cast(col2#x.col1 as int), i2, cast(col2#x.col2 as int)) AS t5#x]
++- Project [col1#x AS i1#x, named_struct(i1, col2#x.col1, i2, col2#x.col2) AS t5#x]
+- LocalRelation [col1#x, col2#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out
index f67d7c6f8142b..23ee616dda8ea 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-recursion.sql.out
@@ -409,20 +409,10 @@ WITH RECURSIVE r(level) AS (
)
SELECT * FROM r
-- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
+org.apache.spark.sql.AnalysisException
{
- "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION",
- "sqlState" : "42703",
- "messageParameters" : {
- "objectName" : "`level`"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 62,
- "stopIndex" : 66,
- "fragment" : "level"
- } ]
+ "errorClass" : "INVALID_RECURSIVE_CTE",
+ "sqlState" : "42836"
}
@@ -471,24 +461,11 @@ WITH RECURSIVE t(col) (
)
SELECT * FROM t LIMIT 5
-- !query analysis
-WithCTE
-:- CTERelationDef xxxx, false
-: +- SubqueryAlias t
-: +- Project [1#x AS col#x]
-: +- UnionLoop xxxx
-: :- Project [1 AS 1#x]
-: : +- OneRowRelation
-: +- Project [scalar-subquery#x [] AS scalarsubquery()#x]
-: : +- Aggregate [max(col#x) AS max(col)#x]
-: : +- SubqueryAlias t
-: : +- Project [1#x AS col#x]
-: : +- UnionLoopRef xxxx, [1#x], false
-: +- OneRowRelation
-+- GlobalLimit 5
- +- LocalLimit 5
- +- Project [col#x]
- +- SubqueryAlias t
- +- CTERelationRef xxxx, true, [col#x], false, false
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE",
+ "sqlState" : "42836"
+}
-- !query
@@ -511,6 +488,35 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException
}
+-- !query
+WITH RECURSIVE t1(n) AS (
+ SELECT 1
+ UNION ALL
+ SELECT (SELECT n+1 FROM t1 WHERE n<5)
+)
+SELECT * FROM t1 LIMIT 5
+-- !query analysis
+WithCTE
+:- CTERelationDef xxxx, false
+: +- SubqueryAlias t1
+: +- Project [1#x AS n#x]
+: +- UnionLoop xxxx
+: :- Project [1 AS 1#x]
+: : +- OneRowRelation
+: +- Project [scalar-subquery#x [] AS scalarsubquery()#x]
+: : +- Project [(n#x + 1) AS (n + 1)#x]
+: : +- Filter (n#x < 5)
+: : +- SubqueryAlias t1
+: : +- Project [1#x AS n#x]
+: : +- UnionLoopRef xxxx, [1#x], false
+: +- OneRowRelation
++- GlobalLimit 5
+ +- LocalLimit 5
+ +- Project [n#x]
+ +- SubqueryAlias t1
+ +- CTERelationRef xxxx, true, [n#x], false, false
+
+
-- !query
WITH RECURSIVE
t1 AS (
@@ -1360,7 +1366,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`tb`, false
INSERT INTO tb VALUES (0), (1)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/tb, false, JSON, [path=file:[not included in comparison]/{warehouse_dir}/tb], Append, `spark_catalog`.`default`.`tb`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/tb), [next]
-+- Project [cast(col1#x as int) AS next#x]
++- Project [col1#x AS next#x]
+- LocalRelation [col1#x]
@@ -1577,28 +1583,11 @@ WITH RECURSIVE t1(n) AS (
SELECT n+1 FROM t2 WHERE n < 5)
SELECT * FROM t1
-- !query analysis
-WithCTE
-:- CTERelationDef xxxx, false
-: +- SubqueryAlias t1
-: +- Project [1#x AS n#x]
-: +- WithCTE
-: :- CTERelationDef xxxx, false
-: : +- SubqueryAlias t2
-: : +- Project [n#x AS n#x]
-: : +- Project [n#x]
-: : +- SubqueryAlias t1
-: : +- Project [1#x AS n#x]
-: : +- UnionLoopRef xxxx, [1#x], false
-: +- UnionLoop xxxx
-: :- Project [1 AS 1#x]
-: : +- OneRowRelation
-: +- Project [(n#x + 1) AS (n + 1)#x]
-: +- Filter (n#x < 5)
-: +- SubqueryAlias t2
-: +- CTERelationRef xxxx, true, [n#x], false, false
-+- Project [n#x]
- +- SubqueryAlias t1
- +- CTERelationRef xxxx, true, [n#x], false, false
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "INVALID_RECURSIVE_CTE",
+ "sqlState" : "42836"
+}
-- !query
@@ -1609,26 +1598,11 @@ WITH RECURSIVE t1 AS (
SELECT n+1 FROM t2 WHERE n < 5)
SELECT * FROM t1
-- !query analysis
-WithCTE
-:- CTERelationDef xxxx, false
-: +- SubqueryAlias t1
-: +- WithCTE
-: :- CTERelationDef xxxx, false
-: : +- SubqueryAlias t2
-: : +- Project [n#x AS n#x]
-: : +- Project [n#x]
-: : +- SubqueryAlias t1
-: : +- UnionLoopRef xxxx, [n#x], false
-: +- UnionLoop xxxx
-: :- Project [1 AS n#x]
-: : +- OneRowRelation
-: +- Project [(n#x + 1) AS (n + 1)#x]
-: +- Filter (n#x < 5)
-: +- SubqueryAlias t2
-: +- CTERelationRef xxxx, true, [n#x], false, false
-+- Project [n#x]
- +- SubqueryAlias t1
- +- CTERelationRef xxxx, true, [n#x], false, false
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "INVALID_RECURSIVE_CTE",
+ "sqlState" : "42836"
+}
-- !query
@@ -1812,3 +1786,348 @@ WITH RECURSIVE randoms(val) AS (
SELECT val FROM randoms LIMIT 5
-- !query analysis
[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+WITH RECURSIVE t1(n, m) AS (
+ SELECT 1, CAST(1 AS BIGINT)
+ UNION ALL
+ SELECT n+1, n+1 FROM t1 WHERE n < 5)
+SELECT * FROM t1
+-- !query analysis
+WithCTE
+:- CTERelationDef xxxx, false
+: +- SubqueryAlias t1
+: +- Project [1#x AS n#x, CAST(1 AS BIGINT)#xL AS m#xL]
+: +- UnionLoop xxxx
+: :- Project [1 AS 1#x, cast(1 as bigint) AS CAST(1 AS BIGINT)#xL]
+: : +- OneRowRelation
+: +- Project [cast((n + 1)#x as int) AS (n + 1)#x, cast((n + 1)#x as bigint) AS (n + 1)#xL]
+: +- Project [(n#x + 1) AS (n + 1)#x, (n#x + 1) AS (n + 1)#x]
+: +- Filter (n#x < 5)
+: +- SubqueryAlias t1
+: +- Project [1#x AS n#x, CAST(1 AS BIGINT)#xL AS m#xL]
+: +- UnionLoopRef xxxx, [1#x, CAST(1 AS BIGINT)#xL], false
++- Project [n#x, m#xL]
+ +- SubqueryAlias t1
+ +- CTERelationRef xxxx, true, [n#x, m#xL], false, false
+
+
+-- !query
+WITH RECURSIVE t1(n, m) AS (
+ SELECT 1, 1
+ UNION ALL
+ SELECT n+1, CAST(n+1 AS BIGINT) FROM t1 WHERE n < 5)
+SELECT * FROM t1
+-- !query analysis
+org.apache.spark.SparkException
+{
+ "errorClass" : "CANNOT_MERGE_INCOMPATIBLE_DATA_TYPE",
+ "sqlState" : "42825",
+ "messageParameters" : {
+ "left" : "\"INT\"",
+ "right" : "\"BIGINT\""
+ }
+}
+
+
+-- !query
+WITH RECURSIVE t1(n) AS (
+ SELECT 1
+ UNION ALL
+ SELECT CASE WHEN n < 5 THEN n + 1 ELSE NULL END FROM t1
+)
+SELECT * FROM t1 LIMIT 25
+-- !query analysis
+WithCTE
+:- CTERelationDef xxxx, false
+: +- SubqueryAlias t1
+: +- Project [1#x AS n#x]
+: +- UnionLoop xxxx
+: :- Project [1 AS 1#x]
+: : +- OneRowRelation
+: +- Project [CASE WHEN (n#x < 5) THEN (n#x + 1) ELSE cast(null as int) END AS CASE WHEN (n < 5) THEN (n + 1) ELSE NULL END#x]
+: +- SubqueryAlias t1
+: +- Project [1#x AS n#x]
+: +- UnionLoopRef xxxx, [1#x], false
++- GlobalLimit 25
+ +- LocalLimit 25
+ +- Project [n#x]
+ +- SubqueryAlias t1
+ +- CTERelationRef xxxx, true, [n#x], false, false
+
+
+-- !query
+WITH RECURSIVE t1(n) AS (
+ SELECT 1
+ UNION ALL
+ SELECT n + 1 FROM t1 WHERE n < 5
+)
+SELECT (SELECT SUM(n) FROM (SELECT * FROM t1)), (SELECT SUM(n) FROM (SELECT * FROM t1 LIMIT 3))
+-- !query analysis
+WithCTE
+:- CTERelationDef xxxx, false
+: +- SubqueryAlias t1
+: +- Project [1#x AS n#x]
+: +- UnionLoop xxxx
+: :- Project [1 AS 1#x]
+: : +- OneRowRelation
+: +- Project [(n#x + 1) AS (n + 1)#x]
+: +- Filter (n#x < 5)
+: +- SubqueryAlias t1
+: +- Project [1#x AS n#x]
+: +- UnionLoopRef xxxx, [1#x], false
++- Project [scalar-subquery#x [] AS scalarsubquery()#xL, scalar-subquery#x [] AS scalarsubquery()#xL]
+ : :- Aggregate [sum(n#x) AS sum(n)#xL]
+ : : +- SubqueryAlias __auto_generated_subquery_name
+ : : +- Project [n#x]
+ : : +- SubqueryAlias t1
+ : : +- CTERelationRef xxxx, true, [n#x], false, false
+ : +- Aggregate [sum(n#x) AS sum(n)#xL]
+ : +- SubqueryAlias __auto_generated_subquery_name
+ : +- GlobalLimit 3
+ : +- LocalLimit 3
+ : +- Project [n#x]
+ : +- SubqueryAlias t1
+ : +- CTERelationRef xxxx, true, [n#x], false, false
+ +- OneRowRelation
+
+
+-- !query
+WITH RECURSIVE t1(n) AS (
+ SELECT 1
+ UNION ALL
+ SELECT n + 1 FROM t1
+)
+SELECT (SELECT SUM(n) FROM (SELECT * FROM t1 LIMIT 5)), (SELECT SUM(n) FROM (SELECT * FROM t1 LIMIT 3))
+-- !query analysis
+WithCTE
+:- CTERelationDef xxxx, false
+: +- SubqueryAlias t1
+: +- Project [1#x AS n#x]
+: +- UnionLoop xxxx
+: :- Project [1 AS 1#x]
+: : +- OneRowRelation
+: +- Project [(n#x + 1) AS (n + 1)#x]
+: +- SubqueryAlias t1
+: +- Project [1#x AS n#x]
+: +- UnionLoopRef xxxx, [1#x], false
++- Project [scalar-subquery#x [] AS scalarsubquery()#xL, scalar-subquery#x [] AS scalarsubquery()#xL]
+ : :- Aggregate [sum(n#x) AS sum(n)#xL]
+ : : +- SubqueryAlias __auto_generated_subquery_name
+ : : +- GlobalLimit 5
+ : : +- LocalLimit 5
+ : : +- Project [n#x]
+ : : +- SubqueryAlias t1
+ : : +- CTERelationRef xxxx, true, [n#x], false, false
+ : +- Aggregate [sum(n#x) AS sum(n)#xL]
+ : +- SubqueryAlias __auto_generated_subquery_name
+ : +- GlobalLimit 3
+ : +- LocalLimit 3
+ : +- Project [n#x]
+ : +- SubqueryAlias t1
+ : +- CTERelationRef xxxx, true, [n#x], false, false
+ +- OneRowRelation
+
+
+-- !query
+WITH RECURSIVE t1(n) AS (
+ SELECT 1
+ UNION ALL
+ SELECT n + 1 FROM t1
+), t2(m) AS (
+ SELECT (SELECT SUM(n) FROM (SELECT n FROM t1 LIMIT 10) AS sums)
+ UNION ALL
+ SELECT m + (SELECT SUM(n) FROM (SELECT n FROM t1 LIMIT 3) AS sums) FROM t2
+)
+SELECT * FROM t2 LIMIT 20
+-- !query analysis
+WithCTE
+:- CTERelationDef xxxx, false
+: +- SubqueryAlias t1
+: +- Project [1#x AS n#x]
+: +- UnionLoop xxxx
+: :- Project [1 AS 1#x]
+: : +- OneRowRelation
+: +- Project [(n#x + 1) AS (n + 1)#x]
+: +- SubqueryAlias t1
+: +- Project [1#x AS n#x]
+: +- UnionLoopRef xxxx, [1#x], false
+:- CTERelationDef xxxx, false
+: +- SubqueryAlias t2
+: +- Project [scalarsubquery()#xL AS m#xL]
+: +- UnionLoop xxxx
+: :- Project [scalar-subquery#x [] AS scalarsubquery()#xL]
+: : : +- Aggregate [sum(n#x) AS sum(n)#xL]
+: : : +- SubqueryAlias sums
+: : : +- GlobalLimit 10
+: : : +- LocalLimit 10
+: : : +- Project [n#x]
+: : : +- SubqueryAlias t1
+: : : +- CTERelationRef xxxx, true, [n#x], false, false
+: : +- OneRowRelation
+: +- Project [(m#xL + scalar-subquery#x []) AS (m + scalarsubquery())#xL]
+: : +- Aggregate [sum(n#x) AS sum(n)#xL]
+: : +- SubqueryAlias sums
+: : +- GlobalLimit 3
+: : +- LocalLimit 3
+: : +- Project [n#x]
+: : +- SubqueryAlias t1
+: : +- CTERelationRef xxxx, true, [n#x], false, false
+: +- SubqueryAlias t2
+: +- Project [scalarsubquery()#xL AS m#xL]
+: +- UnionLoopRef xxxx, [scalarsubquery()#xL], false
++- GlobalLimit 20
+ +- LocalLimit 20
+ +- Project [m#xL]
+ +- SubqueryAlias t2
+ +- CTERelationRef xxxx, true, [m#xL], false, false
+
+
+-- !query
+WITH RECURSIVE t1(n) AS (
+ SELECT 1
+ UNION ALL
+ SELECT n + 1 FROM t1
+)
+ ((SELECT n FROM t1) UNION ALL (SELECT n FROM t1)) LIMIT 20
+-- !query analysis
+WithCTE
+:- CTERelationDef xxxx, false
+: +- SubqueryAlias t1
+: +- Project [1#x AS n#x]
+: +- UnionLoop xxxx
+: :- Project [1 AS 1#x]
+: : +- OneRowRelation
+: +- Project [(n#x + 1) AS (n + 1)#x]
+: +- SubqueryAlias t1
+: +- Project [1#x AS n#x]
+: +- UnionLoopRef xxxx, [1#x], false
++- GlobalLimit 20
+ +- LocalLimit 20
+ +- Union false, false
+ :- Project [n#x]
+ : +- SubqueryAlias t1
+ : +- CTERelationRef xxxx, true, [n#x], false, false
+ +- Project [n#x]
+ +- SubqueryAlias t1
+ +- CTERelationRef xxxx, true, [n#x], false, false
+
+
+-- !query
+WITH RECURSIVE win(id, val) AS (
+ SELECT 1, CAST(10 AS BIGINT)
+ UNION ALL
+ SELECT id + 1, SUM(val) OVER (ORDER BY id ROWS BETWEEN 1 PRECEDING AND CURRENT ROW)
+ FROM win WHERE id < 3
+)
+SELECT * FROM win
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE",
+ "sqlState" : "42836"
+}
+
+
+-- !query
+WITH RECURSIVE t1(n) AS (
+ SELECT 1
+ UNION ALL
+ (SELECT n + 1 FROM t1 WHERE n < 5 ORDER BY n)
+)
+SELECT * FROM t1
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "INVALID_RECURSIVE_REFERENCE.PLACE",
+ "sqlState" : "42836"
+}
+
+
+-- !query
+WITH RECURSIVE t1(n) AS (
+ SELECT 1 FROM t1
+ UNION ALL
+ SELECT n+1 FROM t1 WHERE n < 5)
+SELECT * FROM t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "INVALID_RECURSIVE_CTE",
+ "sqlState" : "42836"
+}
+
+
+-- !query
+WITH RECURSIVE t1 AS (
+ SELECT 1 AS n FROM t1
+ UNION ALL
+ SELECT n+1 FROM t1 WHERE n < 5)
+SELECT * FROM t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "INVALID_RECURSIVE_CTE",
+ "sqlState" : "42836"
+}
+
+
+-- !query
+WITH RECURSIVE t1(n) AS (
+ WITH t2(m) AS (SELECT 1)
+ SELECT 1 FROM t1
+ UNION ALL
+ SELECT n+1 FROM t1 WHERE n < 5)
+SELECT * FROM t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "INVALID_RECURSIVE_CTE",
+ "sqlState" : "42836"
+}
+
+
+-- !query
+WITH RECURSIVE t1 AS (
+ WITH t2(m) AS (SELECT 1)
+ SELECT 1 AS n FROM t1
+ UNION ALL
+ SELECT n+1 FROM t1 WHERE n < 5)
+SELECT * FROM t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "INVALID_RECURSIVE_CTE",
+ "sqlState" : "42836"
+}
+
+
+-- !query
+WITH RECURSIVE t AS (
+ SELECT 1 AS n
+ UNION ALL
+ SELECT n + m
+ FROM (SELECT 2 as m) subq
+ JOIN t ON n = m
+ WHERE n <> m
+)
+SELECT * FROM t
+-- !query analysis
+WithCTE
+:- CTERelationDef xxxx, false
+: +- SubqueryAlias t
+: +- UnionLoop xxxx
+: :- Project [1 AS n#x]
+: : +- OneRowRelation
+: +- Project [(n#x + m#x) AS (n + m)#x]
+: +- Filter NOT (n#x = m#x)
+: +- Join Inner, (n#x = m#x)
+: :- SubqueryAlias subq
+: : +- Project [2 AS m#x]
+: : +- OneRowRelation
+: +- SubqueryAlias t
+: +- UnionLoopRef xxxx, [n#x], false
++- Project [n#x]
+ +- SubqueryAlias t
+ +- CTERelationRef xxxx, true, [n#x], false, false
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-legacy.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-legacy.sql.out
index 3681a5dfd3904..e9d301c649808 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-legacy.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-legacy.sql.out
@@ -1113,6 +1113,235 @@ Project [make_timestamp(1, 1, 1, 1, 1, cast(999.999999 as decimal(16,6)), None,
+- OneRowRelation
+-- !query
+SELECT make_timestamp(make_date(2021, 07, 11))
+-- !query analysis
+Project [make_timestamp(make_date(2021, 7, 11, true), None, None, Some(America/Los_Angeles)) AS make_timestamp(make_date(2021, 7, 11))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT make_timestamp(DATE'1970-01-01')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT make_timestamp(NULL)
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678))
+-- !query analysis
+Project [make_timestamp(make_date(2021, 7, 11, true), Some(make_time(6, 30, cast(45.678 as decimal(16,6)))), None, Some(America/Los_Angeles)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT make_timestamp(NULL, TIME'00:00:00')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT make_timestamp(DATE'1970-01-01', NULL)
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT make_timestamp(NULL, NULL)
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'MIT')
+-- !query analysis
+Project [make_timestamp(make_date(2021, 7, 11, true), Some(make_time(6, 30, cast(45.678 as decimal(16,6)))), Some(MIT), Some(America/Los_Angeles)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), MIT)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'PST')
+-- !query analysis
+Project [make_timestamp(make_date(2021, 7, 11, true), Some(make_time(6, 30, cast(45.678 as decimal(16,6)))), Some(PST), Some(America/Los_Angeles)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), PST)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'UTC')
+-- !query analysis
+Project [make_timestamp(make_date(2021, 7, 11, true), Some(make_time(6, 30, cast(45.678 as decimal(16,6)))), Some(UTC), Some(America/Los_Angeles)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), UTC)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'CET')
+-- !query analysis
+Project [make_timestamp(make_date(2021, 7, 11, true), Some(make_time(6, 30, cast(45.678 as decimal(16,6)))), Some(CET), Some(America/Los_Angeles)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), CET)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'JST')
+-- !query analysis
+Project [make_timestamp(make_date(2021, 7, 11, true), Some(make_time(6, 30, cast(45.678 as decimal(16,6)))), Some(JST), Some(America/Los_Angeles)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), JST)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00', 'UTC')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT make_timestamp(NULL, TIME'00:00:00', 'UTC')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT make_timestamp(DATE'1970-01-01', NULL, 'UTC')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00', NULL)
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT make_timestamp(NULL, NULL, 'UTC')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT make_timestamp(NULL, TIME'00:00:00', NULL)
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT make_timestamp(DATE'1970-01-01', NULL, NULL)
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT make_timestamp(NULL, NULL, NULL)
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT make_timestamp(timestamp '2018-11-17 13:33:33', TIME'0:0:0', 'CET')
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"TIMESTAMP '2018-11-17 13:33:33'\"",
+ "inputType" : "\"TIMESTAMP\"",
+ "paramIndex" : "first",
+ "requiredType" : "\"DATE\"",
+ "sqlExpr" : "\"make_timestamp(TIMESTAMP '2018-11-17 13:33:33', TIME '00:00:00', CET)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 74,
+ "fragment" : "make_timestamp(timestamp '2018-11-17 13:33:33', TIME'0:0:0', 'CET')"
+ } ]
+}
+
+
+-- !query
+SELECT make_timestamp(timestamp_ntz '2018-11-17 13:33:33', TIME'0:0:0', 'CET')
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"TIMESTAMP_NTZ '2018-11-17 13:33:33'\"",
+ "inputType" : "\"TIMESTAMP_NTZ\"",
+ "paramIndex" : "first",
+ "requiredType" : "\"DATE\"",
+ "sqlExpr" : "\"make_timestamp(TIMESTAMP_NTZ '2018-11-17 13:33:33', TIME '00:00:00', CET)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 78,
+ "fragment" : "make_timestamp(timestamp_ntz '2018-11-17 13:33:33', TIME'0:0:0', 'CET')"
+ } ]
+}
+
+
+-- !query
+SELECT make_timestamp(DATE'1970-01-01', timestamp '2018-11-17 13:33:33', 'CET')
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"TIMESTAMP '2018-11-17 13:33:33'\"",
+ "inputType" : "\"TIMESTAMP\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"TIME\"",
+ "sqlExpr" : "\"make_timestamp(DATE '1970-01-01', TIMESTAMP '2018-11-17 13:33:33', CET)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 79,
+ "fragment" : "make_timestamp(DATE'1970-01-01', timestamp '2018-11-17 13:33:33', 'CET')"
+ } ]
+}
+
+
+-- !query
+SELECT make_timestamp(DATE'1970-01-01', timestamp_ntz '2018-11-17 13:33:33', 'CET')
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"TIMESTAMP_NTZ '2018-11-17 13:33:33'\"",
+ "inputType" : "\"TIMESTAMP_NTZ\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"TIME\"",
+ "sqlExpr" : "\"make_timestamp(DATE '1970-01-01', TIMESTAMP_NTZ '2018-11-17 13:33:33', CET)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 83,
+ "fragment" : "make_timestamp(DATE'1970-01-01', timestamp_ntz '2018-11-17 13:33:33', 'CET')"
+ } ]
+}
+
+
-- !query
select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null)
-- !query analysis
@@ -1595,45 +1824,13 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException
-- !query
select timestamp'2011-11-11 11:11:11' + null
-- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
- "sqlState" : "42K09",
- "messageParameters" : {
- "left" : "\"TIMESTAMP\"",
- "right" : "\"VOID\"",
- "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + NULL)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 44,
- "fragment" : "timestamp'2011-11-11 11:11:11' + null"
- } ]
-}
+[Analyzer test output redacted due to nondeterminism]
-- !query
select null + timestamp'2011-11-11 11:11:11'
-- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
- "sqlState" : "42K09",
- "messageParameters" : {
- "left" : "\"VOID\"",
- "right" : "\"TIMESTAMP\"",
- "sqlExpr" : "\"(NULL + TIMESTAMP '2011-11-11 11:11:11')\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 44,
- "fragment" : "null + timestamp'2011-11-11 11:11:11'"
- } ]
-}
+[Analyzer test output redacted due to nondeterminism]
-- !query
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/decimalArithmeticOperations.sql.out
index d75f4d41bd425..5c136f6fe1779 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/decimalArithmeticOperations.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/decimalArithmeticOperations.sql.out
@@ -51,7 +51,7 @@ insert into decimals_test values(1, 100.0, 999.0), (2, 12345.123, 12345.123),
(3, 0.1234567891011, 1234.1), (4, 123456789123456789.0, 1.123456789123456789)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/decimals_test, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/decimals_test], Append, `spark_catalog`.`default`.`decimals_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/decimals_test), [id, a, b]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,18)) AS a#x, cast(col3#x as decimal(38,18)) AS b#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,18)) AS a#x, cast(col3#x as decimal(38,18)) AS b#x]
+- LocalRelation [col1#x, col2#x, col3#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/describe-part-after-analyze.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/describe-part-after-analyze.sql.out
index 8bb042e99795f..5da91fd74014f 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/describe-part-after-analyze.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/describe-part-after-analyze.sql.out
@@ -12,7 +12,7 @@ VALUES ('k1', 100), ('k2', 200), ('k3', 300)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t, [ds=2017-08-01, hr=10], false, [ds#x, hr#x], Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t], Append, `spark_catalog`.`default`.`t`, org.apache.spark.sql.execution.datasources.CatalogFileIndex(file:[not included in comparison]/{warehouse_dir}/t), [key, value, ds, hr]
+- Project [key#x, value#x, cast(2017-08-01 as string) AS ds#x, cast(10 as int) AS hr#x]
- +- Project [cast(col1#x as string) AS key#x, cast(col2#x as string) AS value#x]
+ +- Project [col1#x AS key#x, cast(col2#x as string) AS value#x]
+- LocalRelation [col1#x, col2#x]
@@ -22,7 +22,7 @@ VALUES ('k1', 101), ('k2', 201), ('k3', 301), ('k4', 401)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t, [ds=2017-08-01, hr=11], false, [ds#x, hr#x], Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t], Append, `spark_catalog`.`default`.`t`, org.apache.spark.sql.execution.datasources.CatalogFileIndex(file:[not included in comparison]/{warehouse_dir}/t), [key, value, ds, hr]
+- Project [key#x, value#x, cast(2017-08-01 as string) AS ds#x, cast(11 as int) AS hr#x]
- +- Project [cast(col1#x as string) AS key#x, cast(col2#x as string) AS value#x]
+ +- Project [col1#x AS key#x, cast(col2#x as string) AS value#x]
+- LocalRelation [col1#x, col2#x]
@@ -32,7 +32,7 @@ VALUES ('k1', 102), ('k2', 202)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t, [ds=2017-09-01, hr=5], false, [ds#x, hr#x], Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t], Append, `spark_catalog`.`default`.`t`, org.apache.spark.sql.execution.datasources.CatalogFileIndex(file:[not included in comparison]/{warehouse_dir}/t), [key, value, ds, hr]
+- Project [key#x, value#x, cast(2017-09-01 as string) AS ds#x, cast(5 as int) AS hr#x]
- +- Project [cast(col1#x as string) AS key#x, cast(col2#x as string) AS value#x]
+ +- Project [col1#x AS key#x, cast(col2#x as string) AS value#x]
+- LocalRelation [col1#x, col2#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out
index 2f7237663b646..e49673d33943b 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out
@@ -213,6 +213,41 @@ org.apache.spark.sql.catalyst.parser.ParseException
}
+-- !query
+DROP SCHEMA IF EXISTS test_schema
+-- !query analysis
+DropNamespace true, false
++- ResolvedNamespace V2SessionCatalog(spark_catalog), [test_schema]
+
+
+-- !query
+CREATE SCHEMA test_schema DEFAULT COLLATION UNICODE
+-- !query analysis
+CreateNamespace false, [collation=UNICODE]
++- ResolvedNamespace V2SessionCatalog(spark_catalog), [test_schema]
+
+
+-- !query
+DESCRIBE SCHEMA EXTENDED test_schema
+-- !query analysis
+DescribeNamespace true, [info_name#x, info_value#x]
++- ResolvedNamespace V2SessionCatalog(spark_catalog), [test_schema]
+
+
+-- !query
+ALTER SCHEMA test_schema DEFAULT COLLATION UTF8_LCASE
+-- !query analysis
+SetNamespaceCollationCommand UTF8_LCASE
++- ResolvedNamespace V2SessionCatalog(spark_catalog), [test_schema]
+
+
+-- !query
+DESCRIBE SCHEMA EXTENDED test_schema
+-- !query analysis
+DescribeNamespace true, [info_name#x, info_value#x]
++- ResolvedNamespace V2SessionCatalog(spark_catalog), [test_schema]
+
+
-- !query
DESC temp_v
-- !query analysis
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers-enabled.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers-enabled.sql.out
index cb42a54b85d57..51569e6c965c8 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers-enabled.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers-enabled.sql.out
@@ -154,6 +154,26 @@ org.apache.spark.sql.AnalysisException
}
+-- !query
+select 1 from "not_""exists"
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`not_\"exists`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 28,
+ "fragment" : "\"not_\"\"exists\""
+ } ]
+}
+
+
-- !query
SELECT 1 FROM `hello`
-- !query analysis
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers.sql.out
index a02bf525f947d..5c8b549a50658 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers.sql.out
@@ -111,6 +111,20 @@ org.apache.spark.sql.catalyst.parser.ParseException
}
+-- !query
+select 1 from "not_""exists"
+-- !query analysis
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"not_\"\"exists\"'",
+ "hint" : ""
+ }
+}
+
+
-- !query
SELECT 1 FROM `hello`
-- !query analysis
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out
index 78bf1ccb1678c..d575cac56d28b 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out
@@ -204,7 +204,7 @@ Project [id#x, name#x, data#x]
EXECUTE IMMEDIATE 'INSERT INTO x VALUES(?)' USING 1
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/x, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/x], Append, `spark_catalog`.`default`.`x`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/x), [id]
-+- Project [cast(col1#x as int) AS id#x]
++- Project [col1#x AS id#x]
+- LocalRelation [col1#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/group-analytics.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/group-analytics.sql.out
index fff84a1a253b4..fc0f6fef8c16c 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/group-analytics.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/group-analytics.sql.out
@@ -383,7 +383,7 @@ HAVING GROUPING(year) = 1 AND GROUPING_ID(course, year) > 0 ORDER BY course, yea
Sort [course#x ASC NULLS FIRST, year#x ASC NULLS FIRST], true
+- Project [course#x, year#x]
+- Filter ((cast(cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) as int) = 1) AND (spark_grouping_id#xL > cast(0 as bigint)))
- +- Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, spark_grouping_id#xL, spark_grouping_id#xL]
+ +- Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, spark_grouping_id#xL]
+- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL]
+- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#x]
+- SubqueryAlias coursesales
@@ -434,34 +434,32 @@ Project [course#x, year#x]
SELECT course, year, GROUPING(course), GROUPING(year) FROM courseSales GROUP BY CUBE(course, year)
ORDER BY GROUPING(course), GROUPING(year), course, year
-- !query analysis
-Project [course#x, year#x, grouping(course)#x, grouping(year)#x]
-+- Sort [cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) ASC NULLS FIRST, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) ASC NULLS FIRST, course#x ASC NULLS FIRST, year#x ASC NULLS FIRST], true
- +- Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(course)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(year)#x, spark_grouping_id#xL, spark_grouping_id#xL]
- +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL]
- +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#x]
- +- SubqueryAlias coursesales
- +- View (`courseSales`, [course#x, year#x, earnings#x])
- +- Project [cast(course#x as string) AS course#x, cast(year#x as int) AS year#x, cast(earnings#x as int) AS earnings#x]
- +- Project [course#x, year#x, earnings#x]
- +- SubqueryAlias courseSales
- +- LocalRelation [course#x, year#x, earnings#x]
+Sort [grouping(course)#x ASC NULLS FIRST, grouping(year)#x ASC NULLS FIRST, course#x ASC NULLS FIRST, year#x ASC NULLS FIRST], true
++- Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(course)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(year)#x]
+ +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL]
+ +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#x]
+ +- SubqueryAlias coursesales
+ +- View (`courseSales`, [course#x, year#x, earnings#x])
+ +- Project [cast(course#x as string) AS course#x, cast(year#x as int) AS year#x, cast(earnings#x as int) AS earnings#x]
+ +- Project [course#x, year#x, earnings#x]
+ +- SubqueryAlias courseSales
+ +- LocalRelation [course#x, year#x, earnings#x]
-- !query
SELECT course, year, GROUPING_ID(course, year) FROM courseSales GROUP BY CUBE(course, year)
ORDER BY GROUPING(course), GROUPING(year), course, year
-- !query analysis
-Project [course#x, year#x, grouping_id(course, year)#xL]
-+- Sort [cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) ASC NULLS FIRST, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) ASC NULLS FIRST, course#x ASC NULLS FIRST, year#x ASC NULLS FIRST], true
- +- Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, spark_grouping_id#xL AS grouping_id(course, year)#xL, spark_grouping_id#xL, spark_grouping_id#xL]
- +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL]
- +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#x]
- +- SubqueryAlias coursesales
- +- View (`courseSales`, [course#x, year#x, earnings#x])
- +- Project [cast(course#x as string) AS course#x, cast(year#x as int) AS year#x, cast(earnings#x as int) AS earnings#x]
- +- Project [course#x, year#x, earnings#x]
- +- SubqueryAlias courseSales
- +- LocalRelation [course#x, year#x, earnings#x]
+Sort [cast((shiftright(grouping_id(course, year)#xL, 1) & 1) as tinyint) ASC NULLS FIRST, cast((shiftright(grouping_id(course, year)#xL, 0) & 1) as tinyint) ASC NULLS FIRST, course#x ASC NULLS FIRST, year#x ASC NULLS FIRST], true
++- Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, spark_grouping_id#xL AS grouping_id(course, year)#xL]
+ +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL]
+ +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#x]
+ +- SubqueryAlias coursesales
+ +- View (`courseSales`, [course#x, year#x, earnings#x])
+ +- Project [cast(course#x as string) AS course#x, cast(year#x as int) AS year#x, cast(earnings#x as int) AS earnings#x]
+ +- Project [course#x, year#x, earnings#x]
+ +- SubqueryAlias courseSales
+ +- LocalRelation [course#x, year#x, earnings#x]
-- !query
@@ -630,7 +628,7 @@ Aggregate [a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x], [a#x, b#x, coun
SELECT a, b, count(1) FROM testData GROUP BY a, CUBE(a, b), ROLLUP(a, b), GROUPING SETS((a, b), (a), ())
-- !query analysis
Aggregate [a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x], [a#x, b#x, count(1) AS count(1)#xL]
-+- Expand [[a#x, b#x, a#x, b#x, 0, 0], [a#x, b#x, a#x, b#x, 0, 1], [a#x, b#x, a#x, b#x, 0, 2], [a#x, b#x, a#x, b#x, 0, 3], [a#x, b#x, a#x, b#x, 0, 4], [a#x, b#x, a#x, b#x, 0, 5], [a#x, b#x, a#x, b#x, 0, 6], [a#x, b#x, a#x, b#x, 0, 7], [a#x, b#x, a#x, b#x, 0, 8], [a#x, b#x, a#x, b#x, 0, 9], [a#x, b#x, a#x, b#x, 0, 10], [a#x, b#x, a#x, b#x, 0, 11], [a#x, b#x, a#x, b#x, 0, 12], [a#x, b#x, a#x, null, 1, 13], [a#x, b#x, a#x, null, 1, 14], [a#x, b#x, a#x, b#x, 0, 15], [a#x, b#x, a#x, null, 1, 16], [a#x, b#x, a#x, null, 1, 17], [a#x, b#x, a#x, b#x, 0, 18], [a#x, b#x, a#x, b#x, 0, 19], [a#x, b#x, a#x, b#x, 0, 20], [a#x, b#x, a#x, b#x, 0, 21], [a#x, b#x, a#x, b#x, 0, 22], [a#x, b#x, a#x, b#x, 0, 23], [a#x, b#x, a#x, b#x, 0, 24], ... 11 more fields], [a#x, b#x, a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x]
++- Expand [[a#x, b#x, a#x, b#x, 0, 0], [a#x, b#x, a#x, b#x, 0, 1], [a#x, b#x, a#x, b#x, 0, 2], [a#x, b#x, a#x, b#x, 0, 3], [a#x, b#x, a#x, b#x, 0, 4], [a#x, b#x, a#x, b#x, 0, 5], [a#x, b#x, a#x, b#x, 0, 6], [a#x, b#x, a#x, b#x, 0, 7], [a#x, b#x, a#x, b#x, 0, 8], [a#x, b#x, a#x, b#x, 0, 9], [a#x, b#x, a#x, b#x, 0, 10], [a#x, b#x, a#x, b#x, 0, 11], [a#x, b#x, a#x, b#x, 0, 12], [a#x, b#x, a#x, null, 1, 13], [a#x, b#x, a#x, null, 1, 14], [a#x, b#x, a#x, b#x, 0, 15], [a#x, b#x, a#x, null, 1, 16], [a#x, b#x, a#x, null, 1, 17], [a#x, b#x, a#x, b#x, 0, 18], [a#x, b#x, a#x, b#x, 0, 19], [a#x, b#x, a#x, b#x, 0, 20], [a#x, b#x, a#x, b#x, 0, 21], [a#x, b#x, a#x, b#x, 0, 22], [a#x, b#x, a#x, b#x, 0, 23], [a#x, b#x, a#x, b#x, 0, 24], [a#x, b#x, a#x, b#x, 0, 25], [a#x, b#x, a#x, b#x, 0, 26], [a#x, b#x, a#x, b#x, 0, 27], [a#x, b#x, a#x, b#x, 0, 28], [a#x, b#x, a#x, b#x, 0, 29], [a#x, b#x, a#x, b#x, 0, 30], [a#x, b#x, a#x, null, 1, 31], [a#x, b#x, a#x, null, 1, 32], [a#x, b#x, a#x, b#x, 0, 33], [a#x, b#x, a#x, null, 1, 34], [a#x, b#x, a#x, null, 1, 35]], [a#x, b#x, a#x, b#x, spark_grouping_id#xL, _gen_grouping_pos#x]
+- Project [a#x, b#x, a#x AS a#x, b#x AS b#x]
+- SubqueryAlias testdata
+- View (`testData`, [a#x, b#x])
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/grouping_set.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/grouping_set.sql.out
index 254f9d0785408..2c63fb1525a46 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/grouping_set.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/grouping_set.sql.out
@@ -116,7 +116,7 @@ FROM (VALUES ('x', 'a', 10), ('y', 'b', 20) ) AS t (c1, c2, c3)
GROUP BY GROUPING SETS ( ( c1 ), ( c2 ) )
HAVING GROUPING__ID > 1
-- !query analysis
-Filter (grouping__id#xL > cast(1 as bigint))
+Filter (GROUPING__ID#xL > cast(1 as bigint))
+- Aggregate [c1#x, c2#x, spark_grouping_id#xL], [c1#x, c2#x, sum(c3#x) AS sum(c3)#xL, spark_grouping_id#xL AS grouping__id#xL]
+- Expand [[c1#x, c2#x, c3#x, c1#x, null, 1], [c1#x, c2#x, c3#x, null, c2#x, 2]], [c1#x, c2#x, c3#x, c1#x, c2#x, spark_grouping_id#xL]
+- Project [c1#x, c2#x, c3#x, c1#x AS c1#x, c2#x AS c2#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/having-and-order-by-recursive-type-name-resolution.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/having-and-order-by-recursive-type-name-resolution.sql.out
new file mode 100644
index 0000000000000..7c843b6ef4757
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/having-and-order-by-recursive-type-name-resolution.sql.out
@@ -0,0 +1,313 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+SELECT 'a' AS col1 FROM VALUES (NAMED_STRUCT('a', 1)) t (col1) GROUP BY col1 ORDER BY col1.a
+-- !query analysis
+Project [col1#x]
++- Sort [col1#x.a ASC NULLS FIRST], true
+ +- Aggregate [col1#x], [a AS col1#x, col1#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT 'a' AS col1 FROM VALUES (NAMED_STRUCT('a', 1)) t (col1) GROUP BY col1 HAVING col1.a > 0
+-- !query analysis
+Project [col1#x]
++- Filter (col1#x.a > 0)
+ +- Aggregate [col1#x], [a AS col1#x, col1#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT 'a' AS col1 FROM VALUES (NAMED_STRUCT('a', 1)) t (col1) GROUP BY col1 HAVING col1.a > 0 ORDER BY col1.a
+-- !query analysis
+Project [col1#x]
++- Sort [col1#x.a ASC NULLS FIRST], true
+ +- Project [col1#x, col1#x]
+ +- Filter (col1#x.a > 0)
+ +- Aggregate [col1#x], [a AS col1#x, col1#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT NAMED_STRUCT('a', 1) AS col1 FROM VALUES ('a') t (col1) GROUP BY col1 ORDER BY col1.a
+-- !query analysis
+Sort [col1#x.a ASC NULLS FIRST], true
++- Aggregate [col1#x], [named_struct(a, 1) AS col1#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT NAMED_STRUCT('a', 1) AS col1 FROM VALUES ('a') t (col1) GROUP BY col1 HAVING col1.a > 0
+-- !query analysis
+Filter (col1#x.a > 0)
++- Aggregate [col1#x], [named_struct(a, 1) AS col1#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT NAMED_STRUCT('a', 1) AS col1 FROM VALUES ('a') t (col1) GROUP BY col1 HAVING col1.a > 0 ORDER BY col1.a
+-- !query analysis
+Sort [col1#x.a ASC NULLS FIRST], true
++- Filter (col1#x.a > 0)
+ +- Aggregate [col1#x], [named_struct(a, 1) AS col1#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT NAMED_STRUCT('a', 1) AS col1 FROM VALUES (NAMED_STRUCT('a', 0)) t (col1) GROUP BY col1 ORDER BY col1.a
+-- !query analysis
+Sort [col1#x.a ASC NULLS FIRST], true
++- Aggregate [col1#x], [named_struct(a, 1) AS col1#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT NAMED_STRUCT('a', 1) AS col1 FROM VALUES (NAMED_STRUCT('a', 0)) t (col1) GROUP BY col1 HAVING col1.a > 0
+-- !query analysis
+Project [col1#x]
++- Filter (col1#x.a > 0)
+ +- Aggregate [col1#x], [named_struct(a, 1) AS col1#x, col1#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT NAMED_STRUCT('a', 1) AS col1 FROM VALUES (NAMED_STRUCT('a', 0)) t (col1) GROUP BY col1 HAVING col1.a > 0 ORDER BY col1.a
+-- !query analysis
+Sort [col1#x.a ASC NULLS FIRST], true
++- Project [col1#x]
+ +- Filter (col1#x.a > 0)
+ +- Aggregate [col1#x], [named_struct(a, 1) AS col1#x, col1#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT 'a' AS col1 FROM VALUES (ARRAY(1)) t (col1) GROUP BY col1 ORDER BY col1[0]
+-- !query analysis
+Project [col1#x]
++- Sort [col1#x[0] ASC NULLS FIRST], true
+ +- Aggregate [col1#x], [a AS col1#x, col1#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT 'a' AS col1 FROM VALUES (ARRAY(1)) t (col1) GROUP BY col1 HAVING col1[0] > 0
+-- !query analysis
+Project [col1#x]
++- Filter (col1#x[0] > 0)
+ +- Aggregate [col1#x], [a AS col1#x, col1#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT 'a' AS col1 FROM VALUES (ARRAY(1)) t (col1) GROUP BY col1 HAVING col1[0] > 0 ORDER BY col1[0]
+-- !query analysis
+Project [col1#x]
++- Sort [col1#x[0] ASC NULLS FIRST], true
+ +- Project [col1#x, col1#x]
+ +- Filter (col1#x[0] > 0)
+ +- Aggregate [col1#x], [a AS col1#x, col1#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT ARRAY(1) AS col1 FROM VALUES ('a') t (col1) GROUP BY col1 ORDER BY col1[0]
+-- !query analysis
+Sort [col1#x[0] ASC NULLS FIRST], true
++- Aggregate [col1#x], [array(1) AS col1#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT ARRAY(1) AS col1 FROM VALUES ('a') t (col1) GROUP BY col1 HAVING col1[0] > 0
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "INVALID_EXTRACT_BASE_FIELD_TYPE",
+ "sqlState" : "42000",
+ "messageParameters" : {
+ "base" : "\"tempresolvedcolumn(col1)\"",
+ "other" : "\"STRING\""
+ }
+}
+
+
+-- !query
+SELECT NAMED_STRUCT('a', NAMED_STRUCT('b', 1)) AS col1 FROM VALUES ('a') t (col1) GROUP BY col1 ORDER BY col1.a.b
+-- !query analysis
+Sort [col1#x.a.b ASC NULLS FIRST], true
++- Aggregate [col1#x], [named_struct(a, named_struct(b, 1)) AS col1#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT NAMED_STRUCT('a', NAMED_STRUCT('b', 1)) AS col1 FROM VALUES ('a') t (col1) GROUP BY col1 HAVING col1.a.b > 0
+-- !query analysis
+Filter (col1#x.a.b > 0)
++- Aggregate [col1#x], [named_struct(a, named_struct(b, 1)) AS col1#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT NAMED_STRUCT('a', NAMED_STRUCT('b', 1)) AS col1 FROM VALUES ('a') t (col1) GROUP BY col1 HAVING col1.a.b > 0 ORDER BY col1.a.b
+-- !query analysis
+Sort [col1#x.a.b ASC NULLS FIRST], true
++- Filter (col1#x.a.b > 0)
+ +- Aggregate [col1#x], [named_struct(a, named_struct(b, 1)) AS col1#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT ARRAY(NAMED_STRUCT('a', 1)) AS col1 FROM VALUES ('a') t (col1) GROUP BY col1 ORDER BY col1[0].a
+-- !query analysis
+Sort [col1#x[0].a ASC NULLS FIRST], true
++- Aggregate [col1#x], [array(named_struct(a, 1)) AS col1#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT ARRAY(NAMED_STRUCT('a', 1)) AS col1 FROM VALUES ('a') t (col1) GROUP BY col1 HAVING col1[0].a > 0
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "INVALID_EXTRACT_BASE_FIELD_TYPE",
+ "sqlState" : "42000",
+ "messageParameters" : {
+ "base" : "\"tempresolvedcolumn(col1)\"",
+ "other" : "\"STRING\""
+ }
+}
+
+
+-- !query
+SELECT 'a' AS col1 FROM VALUES (MAP('key', 1)) t (col1) GROUP BY col1 ORDER BY col1['key']
+-- !query analysis
+Project [col1#x]
++- Sort [col1#x[key] ASC NULLS FIRST], true
+ +- Aggregate [col1#x], [a AS col1#x, col1#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT 'a' AS col1 FROM VALUES (MAP('key', 1)) t (col1) GROUP BY col1 HAVING col1['key'] > 0
+-- !query analysis
+Project [col1#x]
++- Filter (col1#x[key] > 0)
+ +- Aggregate [col1#x], [a AS col1#x, col1#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT 'a' AS col1 FROM VALUES (MAP('key', 1)) t (col1) GROUP BY col1 HAVING col1['key'] > 0 ORDER BY col1['key']
+-- !query analysis
+Project [col1#x]
++- Sort [col1#x[key] ASC NULLS FIRST], true
+ +- Project [col1#x, col1#x]
+ +- Filter (col1#x[key] > 0)
+ +- Aggregate [col1#x], [a AS col1#x, col1#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT MAP('key', 1) AS col1 FROM VALUES ('a') t (col1) GROUP BY col1 ORDER BY col1['key']
+-- !query analysis
+Sort [col1#x[key] ASC NULLS FIRST], true
++- Aggregate [col1#x], [map(key, 1) AS col1#x]
+ +- SubqueryAlias t
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT MAP('key', 1) AS col1 FROM VALUES ('a') t (col1) GROUP BY col1 HAVING col1['key'] > 0
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "INVALID_EXTRACT_BASE_FIELD_TYPE",
+ "sqlState" : "42000",
+ "messageParameters" : {
+ "base" : "\"tempresolvedcolumn(col1)\"",
+ "other" : "\"STRING\""
+ }
+}
+
+
+-- !query
+SELECT named_struct('a',1) as col, col1
+FROM values(named_struct('a',1))
+ORDER BY col1.a
+-- !query analysis
+Sort [col1#x.a ASC NULLS FIRST], true
++- Project [named_struct(a, 1) AS col#x, col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT named_struct('a', 1) AS col1, col1
+FROM values(named_struct('a', 1))
+GROUP BY col1
+HAVING col1.a > 0
+-- !query analysis
+Filter (col1#x.a > 0)
++- Aggregate [col1#x], [named_struct(a, 1) AS col1#x, col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT array(1, 2, 3) AS col1, col1
+FROM values(array(1, 2, 3))
+ORDER BY col1[0]
+-- !query analysis
+Sort [col1#x[0] ASC NULLS FIRST], true
++- Project [array(1, 2, 3) AS col1#x, col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT array(1, 2, 3) AS col1, col1
+FROM values(array(1, 2, 3))
+GROUP BY col1
+HAVING col1[1] > 1
+-- !query analysis
+Filter (col1#x[1] > 1)
++- Aggregate [col1#x], [array(1, 2, 3) AS col1#x, col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT map('a', 1, 'b', 2) AS col1, col1
+FROM values(map('a', 1, 'b', 2))
+ORDER BY col1['a']
+-- !query analysis
+Sort [col1#x[a] ASC NULLS FIRST], true
++- Project [map(a, 1, b, 2) AS col1#x, col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT map('a', 1, 'b', 2) AS col1, col1
+FROM values(map('a', 1, 'b', 2))
+GROUP BY col1
+HAVING col1['b'] > 1
+-- !query analysis
+Filter (col1#x[b] > 1)
++- Aggregate [col1#x], [map(a, 1, b, 2) AS col1#x, col1#x]
+ +- LocalRelation [col1#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/having.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/having.sql.out
index 95329ac46390f..dab6b87e80737 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/having.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/having.sql.out
@@ -426,3 +426,83 @@ Project [((sum(v) + 1) + min(v))#xL]
+- Project [k#x, v#x]
+- SubqueryAlias hav
+- LocalRelation [k#x, v#x]
+
+
+-- !query
+SELECT col1 AS alias
+FROM values(1)
+GROUP BY col1
+HAVING (
+ SELECT col1 = 1
+)
+-- !query analysis
+Filter cast(scalar-subquery#x [alias#x] as boolean)
+: +- Project [(outer(alias#x) = 1) AS (outer(col1) = 1)#x]
+: +- OneRowRelation
++- Aggregate [col1#x], [col1#x AS alias#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT col1 AS alias
+FROM values(named_struct('a', 1))
+GROUP BY col1
+HAVING (
+ SELECT col1.a = 1
+)
+-- !query analysis
+Filter cast(scalar-subquery#x [alias#x] as boolean)
+: +- Project [(outer(alias#x).a = 1) AS (outer(col1).a = 1)#x]
+: +- OneRowRelation
++- Aggregate [col1#x], [col1#x AS alias#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT col1 AS alias
+FROM values(array(1))
+GROUP BY col1
+HAVING (
+ SELECT col1[0] = 1
+)
+-- !query analysis
+Filter cast(scalar-subquery#x [alias#x] as boolean)
+: +- Project [(outer(alias#x)[0] = 1) AS (outer(col1)[0] = 1)#x]
+: +- OneRowRelation
++- Aggregate [col1#x], [col1#x AS alias#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT col1 AS alias
+FROM values(map('a', 1))
+GROUP BY col1
+HAVING (
+ SELECT col1[0] = 1
+)
+-- !query analysis
+Filter cast(scalar-subquery#x [alias#x] as boolean)
+: +- Project [(outer(alias#x)[cast(0 as string)] = 1) AS (outer(col1)[0] = 1)#x]
+: +- OneRowRelation
++- Aggregate [col1#x], [col1#x AS alias#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT col1 FROM VALUES(1,2) GROUP BY col1, col2 HAVING col2 = col2
+-- !query analysis
+Project [col1#x]
++- Filter (col2#x = col2#x)
+ +- Aggregate [col1#x, col2#x], [col1#x, col2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT col1 AS a, a AS b FROM VALUES(1,2) GROUP BY col1, col2 HAVING col2 = col2
+-- !query analysis
+Project [a#x, b#x]
++- Filter (col2#x = col2#x)
+ +- Project [a#x, a#x AS b#x, col2#x]
+ +- Project [col1#x, col2#x, col1#x AS a#x]
+ +- Aggregate [col1#x, col2#x], [col1#x, col2#x]
+ +- LocalRelation [col1#x, col2#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/higher-order-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/higher-order-functions.sql.out
index c06d1e5534aed..ab09f9ee47856 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/higher-order-functions.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/higher-order-functions.sql.out
@@ -370,3 +370,741 @@ select aggregate(split('abcdefgh',''), array(array('')), (acc, x) -> array(array
-- !query analysis
Project [aggregate(split(abcdefgh, , -1), array(array()), lambdafunction(array(array(lambda x#x)), lambda acc#x, lambda x#x, false), lambdafunction(lambda id#x, lambda id#x, false)) AS aggregate(split(abcdefgh, , -1), array(array()), lambdafunction(array(array(namedlambdavariable())), namedlambdavariable(), namedlambdavariable()), lambdafunction(namedlambdavariable(), namedlambdavariable()))#x]
+- OneRowRelation
+
+
+-- !query
+select aggregate(array(1, 2, 3), 0, 100) as aggregate_int_literal
+-- !query analysis
+Project [aggregate(array(1, 2, 3), 0, lambdafunction(100, lambda col0#x, lambda col1#x, true), lambdafunction(lambda id#x, lambda id#x, false)) AS aggregate_int_literal#x]
++- OneRowRelation
+
+
+-- !query
+select aggregate(array(1, 2, 3), map(), map('result', 999)) as aggregate_map_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(map(result, 999), namedlambdavariable(), namedlambdavariable())\"",
+ "inputType" : "\"MAP\"",
+ "paramIndex" : "third",
+ "requiredType" : "\"MAP\"",
+ "sqlExpr" : "\"aggregate(array(1, 2, 3), map(), lambdafunction(map(result, 999), namedlambdavariable(), namedlambdavariable()), lambdafunction(namedlambdavariable(), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 59,
+ "fragment" : "aggregate(array(1, 2, 3), map(), map('result', 999))"
+ } ]
+}
+
+
+-- !query
+select aggregate(array(1, 2, 3), struct('init', 0), struct('final', 999)) as aggregate_struct_literal
+-- !query analysis
+Project [aggregate(array(1, 2, 3), struct(col1, init, col2, 0), lambdafunction(struct(col1, final, col2, 999), lambda col0#x, lambda col1#x, true), lambdafunction(lambda id#x, lambda id#x, false)) AS aggregate_struct_literal#x]
++- OneRowRelation
+
+
+-- !query
+select aggregate(array(1, 2, 3), array(), array('result')) as aggregate_array_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(array(result), namedlambdavariable(), namedlambdavariable())\"",
+ "inputType" : "\"ARRAY\"",
+ "paramIndex" : "third",
+ "requiredType" : "\"ARRAY\"",
+ "sqlExpr" : "\"aggregate(array(1, 2, 3), array(), lambdafunction(array(result), namedlambdavariable(), namedlambdavariable()), lambdafunction(namedlambdavariable(), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 58,
+ "fragment" : "aggregate(array(1, 2, 3), array(), array('result'))"
+ } ]
+}
+
+
+-- !query
+select array_sort(array(3, 1, 2), 1) as array_sort_int_literal
+-- !query analysis
+Project [array_sort(array(3, 1, 2), lambdafunction(1, lambda col0#x, lambda col1#x, true), false) AS array_sort_int_literal#x]
++- OneRowRelation
+
+
+-- !query
+select array_sort(array(3, 1, 2), map('compare', 0)) as array_sort_map_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_RETURN_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "actualType" : "\"MAP\"",
+ "expectedType" : "\"INT\"",
+ "functionName" : "`lambdafunction`",
+ "sqlExpr" : "\"array_sort(array(3, 1, 2), lambdafunction(map(compare, 0), namedlambdavariable(), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 52,
+ "fragment" : "array_sort(array(3, 1, 2), map('compare', 0))"
+ } ]
+}
+
+
+-- !query
+select array_sort(array(3, 1, 2), struct('result', 0)) as array_sort_struct_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_RETURN_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "actualType" : "\"STRUCT\"",
+ "expectedType" : "\"INT\"",
+ "functionName" : "`lambdafunction`",
+ "sqlExpr" : "\"array_sort(array(3, 1, 2), lambdafunction(struct(result, 0), namedlambdavariable(), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 54,
+ "fragment" : "array_sort(array(3, 1, 2), struct('result', 0))"
+ } ]
+}
+
+
+-- !query
+select array_sort(array(3, 1, 2), array(0)) as array_sort_array_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_RETURN_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "actualType" : "\"ARRAY\"",
+ "expectedType" : "\"INT\"",
+ "functionName" : "`lambdafunction`",
+ "sqlExpr" : "\"array_sort(array(3, 1, 2), lambdafunction(array(0), namedlambdavariable(), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 43,
+ "fragment" : "array_sort(array(3, 1, 2), array(0))"
+ } ]
+}
+
+
+-- !query
+select exists(array(1, 2, 3), 1) as exists_int_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(1, namedlambdavariable())\"",
+ "inputType" : "\"INT\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"exists(array(1, 2, 3), lambdafunction(1, namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 32,
+ "fragment" : "exists(array(1, 2, 3), 1)"
+ } ]
+}
+
+
+-- !query
+select exists(array(1, 2, 3), map('found', true)) as exists_map_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(map(found, true), namedlambdavariable())\"",
+ "inputType" : "\"MAP\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"exists(array(1, 2, 3), lambdafunction(map(found, true), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 49,
+ "fragment" : "exists(array(1, 2, 3), map('found', true))"
+ } ]
+}
+
+
+-- !query
+select exists(array(1, 2, 3), struct('exists', true)) as exists_struct_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(struct(exists, true), namedlambdavariable())\"",
+ "inputType" : "\"STRUCT\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"exists(array(1, 2, 3), lambdafunction(struct(exists, true), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 53,
+ "fragment" : "exists(array(1, 2, 3), struct('exists', true))"
+ } ]
+}
+
+
+-- !query
+select exists(array(1, 2, 3), array(true)) as exists_array_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(array(true), namedlambdavariable())\"",
+ "inputType" : "\"ARRAY\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"exists(array(1, 2, 3), lambdafunction(array(true), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 42,
+ "fragment" : "exists(array(1, 2, 3), array(true))"
+ } ]
+}
+
+
+-- !query
+select filter(array(1, 2, 3), 1) as filter_int_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(1, namedlambdavariable())\"",
+ "inputType" : "\"INT\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"filter(array(1, 2, 3), lambdafunction(1, namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 32,
+ "fragment" : "filter(array(1, 2, 3), 1)"
+ } ]
+}
+
+
+-- !query
+select filter(array(1, 2, 3), map('key', 'value')) as filter_map_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(map(key, value), namedlambdavariable())\"",
+ "inputType" : "\"MAP\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"filter(array(1, 2, 3), lambdafunction(map(key, value), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 50,
+ "fragment" : "filter(array(1, 2, 3), map('key', 'value'))"
+ } ]
+}
+
+
+-- !query
+select filter(array(1, 2, 3), struct('valid', true)) as filter_struct_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(struct(valid, true), namedlambdavariable())\"",
+ "inputType" : "\"STRUCT\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"filter(array(1, 2, 3), lambdafunction(struct(valid, true), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 52,
+ "fragment" : "filter(array(1, 2, 3), struct('valid', true))"
+ } ]
+}
+
+
+-- !query
+select filter(array(1, 2, 3), array(true, false)) as filter_array_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(array(true, false), namedlambdavariable())\"",
+ "inputType" : "\"ARRAY\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"filter(array(1, 2, 3), lambdafunction(array(true, false), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 49,
+ "fragment" : "filter(array(1, 2, 3), array(true, false))"
+ } ]
+}
+
+
+-- !query
+select forall(array(1, 2, 3), 1) as forall_int_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(1, namedlambdavariable())\"",
+ "inputType" : "\"INT\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"forall(array(1, 2, 3), lambdafunction(1, namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 32,
+ "fragment" : "forall(array(1, 2, 3), 1)"
+ } ]
+}
+
+
+-- !query
+select forall(array(1, 2, 3), map('all', true)) as forall_map_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(map(all, true), namedlambdavariable())\"",
+ "inputType" : "\"MAP\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"forall(array(1, 2, 3), lambdafunction(map(all, true), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 47,
+ "fragment" : "forall(array(1, 2, 3), map('all', true))"
+ } ]
+}
+
+
+-- !query
+select forall(array(1, 2, 3), struct('all', true)) as forall_struct_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(struct(all, true), namedlambdavariable())\"",
+ "inputType" : "\"STRUCT\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"forall(array(1, 2, 3), lambdafunction(struct(all, true), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 50,
+ "fragment" : "forall(array(1, 2, 3), struct('all', true))"
+ } ]
+}
+
+
+-- !query
+select forall(array(1, 2, 3), array(true, true)) as forall_array_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(array(true, true), namedlambdavariable())\"",
+ "inputType" : "\"ARRAY\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"forall(array(1, 2, 3), lambdafunction(array(true, true), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 48,
+ "fragment" : "forall(array(1, 2, 3), array(true, true))"
+ } ]
+}
+
+
+-- !query
+select map_filter(map('a', 1, 'b', 2), 1) as map_filter_int_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(1, namedlambdavariable(), namedlambdavariable())\"",
+ "inputType" : "\"INT\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"map_filter(map(a, 1, b, 2), lambdafunction(1, namedlambdavariable(), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 41,
+ "fragment" : "map_filter(map('a', 1, 'b', 2), 1)"
+ } ]
+}
+
+
+-- !query
+select map_filter(map('a', 1, 'b', 2), map('keep', true)) as map_filter_map_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(map(keep, true), namedlambdavariable(), namedlambdavariable())\"",
+ "inputType" : "\"MAP\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"map_filter(map(a, 1, b, 2), lambdafunction(map(keep, true), namedlambdavariable(), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 57,
+ "fragment" : "map_filter(map('a', 1, 'b', 2), map('keep', true))"
+ } ]
+}
+
+
+-- !query
+select map_filter(map('a', 1, 'b', 2), struct('filter', true)) as map_filter_struct_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(struct(filter, true), namedlambdavariable(), namedlambdavariable())\"",
+ "inputType" : "\"STRUCT\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"map_filter(map(a, 1, b, 2), lambdafunction(struct(filter, true), namedlambdavariable(), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 62,
+ "fragment" : "map_filter(map('a', 1, 'b', 2), struct('filter', true))"
+ } ]
+}
+
+
+-- !query
+select map_filter(map('a', 1, 'b', 2), array(true)) as map_filter_array_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(array(true), namedlambdavariable(), namedlambdavariable())\"",
+ "inputType" : "\"ARRAY\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"map_filter(map(a, 1, b, 2), lambdafunction(array(true), namedlambdavariable(), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 51,
+ "fragment" : "map_filter(map('a', 1, 'b', 2), array(true))"
+ } ]
+}
+
+
+-- !query
+select map_zip_with(map('a', 1), map('a', 10), 100) as map_zipwith_int_literal
+-- !query analysis
+Project [map_zip_with(map(a, 1), map(a, 10), lambdafunction(100, lambda col0#x, lambda col1#x, lambda col2#x, true)) AS map_zipwith_int_literal#x]
++- OneRowRelation
+
+
+-- !query
+select map_zip_with(map('a', 1), map('a', 10), map('merged', true)) as map_zipwith_map_literal
+-- !query analysis
+Project [map_zip_with(map(a, 1), map(a, 10), lambdafunction(map(merged, true), lambda col0#x, lambda col1#x, lambda col2#x, true)) AS map_zipwith_map_literal#x]
++- OneRowRelation
+
+
+-- !query
+select map_zip_with(map('a', 1), map('a', 10), struct('left', 1, 'right', 10)) as map_zipwith_struct_literal
+-- !query analysis
+Project [map_zip_with(map(a, 1), map(a, 10), lambdafunction(struct(col1, left, col2, 1, col3, right, col4, 10), lambda col0#x, lambda col1#x, lambda col2#x, true)) AS map_zipwith_struct_literal#x]
++- OneRowRelation
+
+
+-- !query
+select map_zip_with(map('a', 1), map('a', 10), array('combined')) as map_zipwith_array_literal
+-- !query analysis
+Project [map_zip_with(map(a, 1), map(a, 10), lambdafunction(array(combined), lambda col0#x, lambda col1#x, lambda col2#x, true)) AS map_zipwith_array_literal#x]
++- OneRowRelation
+
+
+-- !query
+select reduce(array(1, 2, 3), 0, 100) as reduce_int_literal
+-- !query analysis
+Project [reduce(array(1, 2, 3), 0, lambdafunction(100, lambda col0#x, lambda col1#x, true), lambdafunction(lambda id#x, lambda id#x, false)) AS reduce_int_literal#x]
++- OneRowRelation
+
+
+-- !query
+select reduce(array(1, 2, 3), map(), map('result', 999)) as reduce_map_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(map(result, 999), namedlambdavariable(), namedlambdavariable())\"",
+ "inputType" : "\"MAP\"",
+ "paramIndex" : "third",
+ "requiredType" : "\"MAP\"",
+ "sqlExpr" : "\"reduce(array(1, 2, 3), map(), lambdafunction(map(result, 999), namedlambdavariable(), namedlambdavariable()), lambdafunction(namedlambdavariable(), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 56,
+ "fragment" : "reduce(array(1, 2, 3), map(), map('result', 999))"
+ } ]
+}
+
+
+-- !query
+select reduce(array(1, 2, 3), struct('init', 0), struct('final', 999)) as reduce_struct_literal
+-- !query analysis
+Project [reduce(array(1, 2, 3), struct(col1, init, col2, 0), lambdafunction(struct(col1, final, col2, 999), lambda col0#x, lambda col1#x, true), lambdafunction(lambda id#x, lambda id#x, false)) AS reduce_struct_literal#x]
++- OneRowRelation
+
+
+-- !query
+select reduce(array(1, 2, 3), array(), array('result')) as reduce_array_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(array(result), namedlambdavariable(), namedlambdavariable())\"",
+ "inputType" : "\"ARRAY\"",
+ "paramIndex" : "third",
+ "requiredType" : "\"ARRAY\"",
+ "sqlExpr" : "\"reduce(array(1, 2, 3), array(), lambdafunction(array(result), namedlambdavariable(), namedlambdavariable()), lambdafunction(namedlambdavariable(), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 55,
+ "fragment" : "reduce(array(1, 2, 3), array(), array('result'))"
+ } ]
+}
+
+
+-- !query
+select transform(array(1, 2, 3), 42) as transform_int_literal
+-- !query analysis
+Project [transform(array(1, 2, 3), lambdafunction(42, lambda col0#x, true)) AS transform_int_literal#x]
++- OneRowRelation
+
+
+-- !query
+select transform(array(1, 2, 3), map('key', 'value')) as transform_map_literal
+-- !query analysis
+Project [transform(array(1, 2, 3), lambdafunction(map(key, value), lambda col0#x, true)) AS transform_map_literal#x]
++- OneRowRelation
+
+
+-- !query
+select transform(array(1, 2, 3), struct('id', 99, 'name', 'test')) as transform_struct_literal
+-- !query analysis
+Project [transform(array(1, 2, 3), lambdafunction(struct(col1, id, col2, 99, col3, name, col4, test), lambda col0#x, true)) AS transform_struct_literal#x]
++- OneRowRelation
+
+
+-- !query
+select transform(array(1, 2, 3), array('a', 'b')) as transform_array_literal
+-- !query analysis
+Project [transform(array(1, 2, 3), lambdafunction(array(a, b), lambda col0#x, true)) AS transform_array_literal#x]
++- OneRowRelation
+
+
+-- !query
+select transform_keys(map('a', 1, 'b', 2), 42) as transform_keys_int_literal
+-- !query analysis
+Project [transform_keys(map(a, 1, b, 2), lambdafunction(42, lambda col0#x, lambda col1#x, true)) AS transform_keys_int_literal#x]
++- OneRowRelation
+
+
+-- !query
+select transform_keys(map('a', 1, 'b', 2), map('new', 'key')) as transform_keys_map_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.INVALID_MAP_KEY_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "keyType" : "\"MAP\"",
+ "sqlExpr" : "\"transform_keys(map(a, 1, b, 2), lambdafunction(map(new, key), namedlambdavariable(), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 61,
+ "fragment" : "transform_keys(map('a', 1, 'b', 2), map('new', 'key'))"
+ } ]
+}
+
+
+-- !query
+select transform_keys(map('a', 1, 'b', 2), struct('key', 'value')) as transform_keys_struct_literal
+-- !query analysis
+Project [transform_keys(map(a, 1, b, 2), lambdafunction(struct(col1, key, col2, value), lambda col0#x, lambda col1#x, true)) AS transform_keys_struct_literal#x]
++- OneRowRelation
+
+
+-- !query
+select transform_keys(map('a', 1, 'b', 2), array('new_key')) as transform_keys_array_literal
+-- !query analysis
+Project [transform_keys(map(a, 1, b, 2), lambdafunction(array(new_key), lambda col0#x, lambda col1#x, true)) AS transform_keys_array_literal#x]
++- OneRowRelation
+
+
+-- !query
+select transform_values(map('a', 1, 'b', 2), 999) as transform_values_int_literal
+-- !query analysis
+Project [transform_values(map(a, 1, b, 2), lambdafunction(999, lambda col0#x, lambda col1#x, true)) AS transform_values_int_literal#x]
++- OneRowRelation
+
+
+-- !query
+select transform_values(map('a', 1, 'b', 2), map('new', 'value')) as transform_values_map_literal
+-- !query analysis
+Project [transform_values(map(a, 1, b, 2), lambdafunction(map(new, value), lambda col0#x, lambda col1#x, true)) AS transform_values_map_literal#x]
++- OneRowRelation
+
+
+-- !query
+select transform_values(map('a', 1, 'b', 2), struct('val', 999)) as transform_values_struct_literal
+-- !query analysis
+Project [transform_values(map(a, 1, b, 2), lambdafunction(struct(col1, val, col2, 999), lambda col0#x, lambda col1#x, true)) AS transform_values_struct_literal#x]
++- OneRowRelation
+
+
+-- !query
+select transform_values(map('a', 1, 'b', 2), array('new_value')) as transform_values_array_literal
+-- !query analysis
+Project [transform_values(map(a, 1, b, 2), lambdafunction(array(new_value), lambda col0#x, lambda col1#x, true)) AS transform_values_array_literal#x]
++- OneRowRelation
+
+
+-- !query
+select zip_with(array(1, 2, 3), array(4, 5, 6), 100) as zipwith_int_literal
+-- !query analysis
+Project [zip_with(array(1, 2, 3), array(4, 5, 6), lambdafunction(100, lambda col0#x, lambda col1#x, true)) AS zipwith_int_literal#x]
++- OneRowRelation
+
+
+-- !query
+select zip_with(array(1, 2, 3), array(4, 5, 6), map('merged', true)) as zipwith_map_literal
+-- !query analysis
+Project [zip_with(array(1, 2, 3), array(4, 5, 6), lambdafunction(map(merged, true), lambda col0#x, lambda col1#x, true)) AS zipwith_map_literal#x]
++- OneRowRelation
+
+
+-- !query
+select zip_with(array(1, 2, 3), array(4, 5, 6), struct('left', 1, 'right', 2)) as zipwith_struct_literal
+-- !query analysis
+Project [zip_with(array(1, 2, 3), array(4, 5, 6), lambdafunction(struct(col1, left, col2, 1, col3, right, col4, 2), lambda col0#x, lambda col1#x, true)) AS zipwith_struct_literal#x]
++- OneRowRelation
+
+
+-- !query
+select zip_with(array(1, 2, 3), array(4, 5, 6), array('combined')) as zipwith_array_literal
+-- !query analysis
+Project [zip_with(array(1, 2, 3), array(4, 5, 6), lambdafunction(array(combined), lambda col0#x, lambda col1#x, true)) AS zipwith_array_literal#x]
++- OneRowRelation
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out
index deb2526e346b6..38b28f2d3b232 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out
@@ -84,7 +84,7 @@ SetNamespaceCommand [s]
INSERT INTO IDENTIFIER('ta' || 'b') VALUES(1)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/s.db/tab, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/s.db/tab], Append, `spark_catalog`.`s`.`tab`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/s.db/tab), [c1]
-+- Project [cast(col1#x as int) AS c1#x]
++- Project [col1#x AS c1#x]
+- LocalRelation [col1#x]
@@ -308,7 +308,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`tab`, false
INSERT INTO tab VALUES (1)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/tab, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/tab], Append, `spark_catalog`.`default`.`tab`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/tab), [c1]
-+- Project [cast(col1#x as int) AS c1#x]
++- Project [col1#x AS c1#x]
+- LocalRelation [col1#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/literals.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/literals.sql.out
index 9724fb01a1a81..39951f9ef3adc 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/literals.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/literals.sql.out
@@ -706,3 +706,51 @@ select -0, -0.0
-- !query analysis
Project [0 AS 0#x, 0.0 AS 0.0#x]
+- OneRowRelation
+
+
+-- !query
+SELECT "S""par""k" AS c1, "S\"par\"k" AS c2, 'S""par""k' AS c3
+-- !query analysis
+Project [S"par"k AS c1#x, S"par"k AS c2#x, S""par""k AS c3#x]
++- OneRowRelation
+
+
+-- !query
+SELECT 'S''par''k' AS c1, 'S\'par\'k' AS c2, "S''par''k" AS c3
+-- !query analysis
+Project [S'par'k AS c1#x, S'par'k AS c2#x, S''par''k AS c3#x]
++- OneRowRelation
+
+
+-- !query
+SELECT "S" "par" "k" AS c1, 'S' 'par' 'k' AS c2, "S" 'par' "k" AS c3, 'S' "par" 'k' AS c4, "S"'par'"k" AS c5, 'S'"par"'k' AS c6
+-- !query analysis
+Project [Spark AS c1#x, Spark AS c2#x, Spark AS c3#x, Spark AS c4#x, Spark AS c5#x, Spark AS c6#x]
++- OneRowRelation
+
+
+-- !query
+SET spark.sql.legacy.consecutiveStringLiterals.enabled=true
+-- !query analysis
+SetCommand (spark.sql.legacy.consecutiveStringLiterals.enabled,Some(true))
+
+
+-- !query
+SELECT "S""par""k" AS c1, "S\"par\"k" AS c2, 'S""par""k' AS c3
+-- !query analysis
+Project [Spark AS c1#x, S"par"k AS c2#x, S""par""k AS c3#x]
++- OneRowRelation
+
+
+-- !query
+SELECT 'S''par''k' AS c1, 'S\'par\'k' AS c2, "S''par''k" AS c3
+-- !query analysis
+Project [Spark AS c1#x, S'par'k AS c2#x, S''par''k AS c3#x]
++- OneRowRelation
+
+
+-- !query
+SELECT "S" "par" "k" AS c1, 'S' 'par' 'k' AS c2, "S" 'par' "k" AS c3, 'S' "par" 'k' AS c4, "S"'par'"k" AS c5, 'S'"par"'k' AS c6
+-- !query analysis
+Project [Spark AS c1#x, Spark AS c2#x, Spark AS c3#x, Spark AS c4#x, Spark AS c5#x, Spark AS c6#x]
++- OneRowRelation
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/name-precedence-in-order-by-and-having-with-conflicting-attributes.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/name-precedence-in-order-by-and-having-with-conflicting-attributes.sql.out
new file mode 100644
index 0000000000000..6a0f6809dedc9
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/name-precedence-in-order-by-and-having-with-conflicting-attributes.sql.out
@@ -0,0 +1,675 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+CREATE OR REPLACE TEMPORARY VIEW v1 AS SELECT col1 FROM VALUES
+ (42), (17), (99), (5), (42),
+ (23), (8), (17), (76), (33),
+ (99), (55), (3), (42), (8)
+-- !query analysis
+CreateViewCommand `v1`, SELECT col1 FROM VALUES
+ (42), (17), (99), (5), (42),
+ (23), (8), (17), (76), (33),
+ (99), (55), (3), (42), (8), false, true, LocalTempView, UNSUPPORTED, true
+ +- Project [col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+CREATE OR REPLACE TEMPORARY VIEW v2 AS SELECT col1 FROM VALUES
+ ('apple'), ('banana'), ('cherry'), ('date'), ('apple'),
+ ('fig'), ('grape'), ('banana'), ('kiwi'), ('lemon'),
+ ('cherry'), ('mango'), ('orange'), ('apple'), ('grape')
+-- !query analysis
+CreateViewCommand `v2`, SELECT col1 FROM VALUES
+ ('apple'), ('banana'), ('cherry'), ('date'), ('apple'),
+ ('fig'), ('grape'), ('banana'), ('kiwi'), ('lemon'),
+ ('cherry'), ('mango'), ('orange'), ('apple'), ('grape'), false, true, LocalTempView, UNSUPPORTED, true
+ +- Project [col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT 2 AS col1 FROM v1 ORDER BY col1
+-- !query analysis
+Sort [col1#x ASC NULLS FIRST], true
++- Project [2 AS col1#x]
+ +- SubqueryAlias v1
+ +- View (`v1`, [col1#x])
+ +- Project [cast(col1#x as int) AS col1#x]
+ +- Project [col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT 2 AS col1 FROM v1 GROUP BY ALL ORDER BY col1
+-- !query analysis
+Sort [col1#x ASC NULLS FIRST], true
++- Aggregate [2], [2 AS col1#x]
+ +- SubqueryAlias v1
+ +- View (`v1`, [col1#x])
+ +- Project [cast(col1#x as int) AS col1#x]
+ +- Project [col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT 2 AS col1 FROM v1 GROUP BY ALL HAVING col1 > 50
+-- !query analysis
+Filter (col1#x > 50)
++- Aggregate [2], [2 AS col1#x]
+ +- SubqueryAlias v1
+ +- View (`v1`, [col1#x])
+ +- Project [cast(col1#x as int) AS col1#x]
+ +- Project [col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT * FROM (
+ SELECT col1 AS c, 2 AS col1 FROM v1 ORDER BY col1
+) ORDER BY 1
+-- !query analysis
+Sort [c#x ASC NULLS FIRST], true
++- Project [c#x, col1#x]
+ +- SubqueryAlias __auto_generated_subquery_name
+ +- Sort [col1#x ASC NULLS FIRST], true
+ +- Project [col1#x AS c#x, 2 AS col1#x]
+ +- SubqueryAlias v1
+ +- View (`v1`, [col1#x])
+ +- Project [cast(col1#x as int) AS col1#x]
+ +- Project [col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT * FROM (
+ SELECT col1 AS c, 2 AS col1 FROM v1 GROUP BY ALL ORDER BY col1
+) ORDER BY 1
+-- !query analysis
+Sort [c#x ASC NULLS FIRST], true
++- Project [c#x, col1#x]
+ +- SubqueryAlias __auto_generated_subquery_name
+ +- Sort [col1#x ASC NULLS FIRST], true
+ +- Aggregate [col1#x, 2], [col1#x AS c#x, 2 AS col1#x]
+ +- SubqueryAlias v1
+ +- View (`v1`, [col1#x])
+ +- Project [cast(col1#x as int) AS col1#x]
+ +- Project [col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT * FROM (
+ SELECT col1 AS c, 2 AS col1 FROM v1 GROUP BY ALL HAVING col1 > 50
+) ORDER BY 1
+-- !query analysis
+Sort [c#x ASC NULLS FIRST], true
++- Project [c#x, col1#x]
+ +- SubqueryAlias __auto_generated_subquery_name
+ +- Filter (c#x > 50)
+ +- Aggregate [col1#x, 2], [col1#x AS c#x, 2 AS col1#x]
+ +- SubqueryAlias v1
+ +- View (`v1`, [col1#x])
+ +- Project [cast(col1#x as int) AS col1#x]
+ +- Project [col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT 2 AS col1, 3 AS col1 FROM v1 ORDER BY col1
+-- !query analysis
+Project [col1#x, col1#x]
++- Sort [col1#x ASC NULLS FIRST], true
+ +- Project [2 AS col1#x, 3 AS col1#x, col1#x]
+ +- SubqueryAlias v1
+ +- View (`v1`, [col1#x])
+ +- Project [cast(col1#x as int) AS col1#x]
+ +- Project [col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT 2 AS col1, 3 AS col1 FROM v1 GROUP BY ALL ORDER BY col1
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION",
+ "sqlState" : "42703",
+ "messageParameters" : {
+ "objectName" : "`col1`",
+ "proposal" : "`col1`, `col1`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 59,
+ "stopIndex" : 62,
+ "fragment" : "col1"
+ } ]
+}
+
+
+-- !query
+SELECT 2 AS col1, 3 AS col1 FROM v1 GROUP BY ALL HAVING col1 > 50
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "AMBIGUOUS_REFERENCE",
+ "sqlState" : "42704",
+ "messageParameters" : {
+ "name" : "`col1`",
+ "referenceNames" : "[`col1`, `col1`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 57,
+ "stopIndex" : 60,
+ "fragment" : "col1"
+ } ]
+}
+
+
+-- !query
+SELECT col1 AS c, 2 AS col1, 3 AS col1 FROM v1 ORDER BY col1
+-- !query analysis
+Project [c#x, col1#x, col1#x]
++- Sort [col1#x ASC NULLS FIRST], true
+ +- Project [col1#x AS c#x, 2 AS col1#x, 3 AS col1#x, col1#x]
+ +- SubqueryAlias v1
+ +- View (`v1`, [col1#x])
+ +- Project [cast(col1#x as int) AS col1#x]
+ +- Project [col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT col1 AS c, 2 AS col1, 3 AS col1 FROM v1 GROUP BY ALL ORDER BY col1
+-- !query analysis
+Sort [c#x ASC NULLS FIRST], true
++- Aggregate [col1#x, 2, 3], [col1#x AS c#x, 2 AS col1#x, 3 AS col1#x]
+ +- SubqueryAlias v1
+ +- View (`v1`, [col1#x])
+ +- Project [cast(col1#x as int) AS col1#x]
+ +- Project [col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT * FROM (
+ SELECT col1 AS c, 2 AS col1, 3 AS col1 FROM v1 GROUP BY ALL HAVING col1 > 50
+) ORDER BY 1
+-- !query analysis
+Sort [c#x ASC NULLS FIRST], true
++- Project [c#x, col1#x, col1#x]
+ +- SubqueryAlias __auto_generated_subquery_name
+ +- Filter (c#x > 50)
+ +- Aggregate [col1#x, 2, 3], [col1#x AS c#x, 2 AS col1#x, 3 AS col1#x]
+ +- SubqueryAlias v1
+ +- View (`v1`, [col1#x])
+ +- Project [cast(col1#x as int) AS col1#x]
+ +- Project [col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT col1, 2 AS col1 FROM v1 ORDER BY col1
+-- !query analysis
+Sort [col1#x ASC NULLS FIRST], true
++- Project [col1#x, 2 AS col1#x]
+ +- SubqueryAlias v1
+ +- View (`v1`, [col1#x])
+ +- Project [cast(col1#x as int) AS col1#x]
+ +- Project [col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT col1, 2 AS col1 FROM v1 GROUP BY ALL ORDER BY col1
+-- !query analysis
+Sort [col1#x ASC NULLS FIRST], true
++- Aggregate [col1#x, 2], [col1#x, 2 AS col1#x]
+ +- SubqueryAlias v1
+ +- View (`v1`, [col1#x])
+ +- Project [cast(col1#x as int) AS col1#x]
+ +- Project [col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT * FROM (
+ SELECT col1, 2 AS col1 FROM v1 GROUP BY ALL HAVING col1 > 50
+) ORDER BY 1
+-- !query analysis
+Sort [col1#x ASC NULLS FIRST], true
++- Project [col1#x, col1#x]
+ +- SubqueryAlias __auto_generated_subquery_name
+ +- Filter (col1#x > 50)
+ +- Aggregate [col1#x, 2], [col1#x, 2 AS col1#x]
+ +- SubqueryAlias v1
+ +- View (`v1`, [col1#x])
+ +- Project [cast(col1#x as int) AS col1#x]
+ +- Project [col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT 3 AS col1 FROM (SELECT 1 AS col1, 2 AS col1) ORDER BY col1
+-- !query analysis
+Sort [col1#x ASC NULLS FIRST], true
++- Project [3 AS col1#x]
+ +- SubqueryAlias __auto_generated_subquery_name
+ +- Project [1 AS col1#x, 2 AS col1#x]
+ +- OneRowRelation
+
+
+-- !query
+SELECT 3 AS col1 FROM (SELECT 1 AS col1, 2 AS col1) GROUP BY ALL ORDER BY col1
+-- !query analysis
+Sort [col1#x ASC NULLS FIRST], true
++- Aggregate [3], [3 AS col1#x]
+ +- SubqueryAlias __auto_generated_subquery_name
+ +- Project [1 AS col1#x, 2 AS col1#x]
+ +- OneRowRelation
+
+
+-- !query
+SELECT 3 AS col1 FROM (SELECT 1 AS col1, 2 AS col1) GROUP BY ALL HAVING col1 > 50
+-- !query analysis
+Filter (col1#x > 50)
++- Aggregate [3], [3 AS col1#x]
+ +- SubqueryAlias __auto_generated_subquery_name
+ +- Project [1 AS col1#x, 2 AS col1#x]
+ +- OneRowRelation
+
+
+-- !query
+SELECT col1 AS c, 3 AS col1 FROM (SELECT 1 AS col1, 2 AS col1) ORDER BY col1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "AMBIGUOUS_REFERENCE",
+ "sqlState" : "42704",
+ "messageParameters" : {
+ "name" : "`col1`",
+ "referenceNames" : "[`col1`, `col1`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 11,
+ "fragment" : "col1"
+ } ]
+}
+
+
+-- !query
+SELECT col1 AS c, 3 AS col1 FROM (SELECT 1 AS col1, 2 AS col1) GROUP BY ALL ORDER BY col1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "AMBIGUOUS_REFERENCE",
+ "sqlState" : "42704",
+ "messageParameters" : {
+ "name" : "`col1`",
+ "referenceNames" : "[`col1`, `col1`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 11,
+ "fragment" : "col1"
+ } ]
+}
+
+
+-- !query
+SELECT col1 AS c, 3 AS col1 FROM (SELECT 1 AS col1, 2 AS col1) GROUP BY ALL HAVING col1 > 50
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "AMBIGUOUS_REFERENCE",
+ "sqlState" : "42704",
+ "messageParameters" : {
+ "name" : "`col1`",
+ "referenceNames" : "[`col1`, `col1`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 11,
+ "fragment" : "col1"
+ } ]
+}
+
+
+-- !query
+SELECT 'col1' FROM v2 ORDER BY col1
+-- !query analysis
+Sort [col1#x ASC NULLS FIRST], true
++- Project [col1 AS col1#x]
+ +- SubqueryAlias v2
+ +- View (`v2`, [col1#x])
+ +- Project [cast(col1#x as string) AS col1#x]
+ +- Project [col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT 'col1' FROM v2 GROUP BY ALL ORDER BY col1
+-- !query analysis
+Sort [col1#x ASC NULLS FIRST], true
++- Aggregate [col1], [col1 AS col1#x]
+ +- SubqueryAlias v2
+ +- View (`v2`, [col1#x])
+ +- Project [cast(col1#x as string) AS col1#x]
+ +- Project [col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT 'col1' FROM v2 GROUP BY ALL HAVING col1 > 'banana'
+-- !query analysis
+Filter (col1#x > banana)
++- Aggregate [col1], [col1 AS col1#x]
+ +- SubqueryAlias v2
+ +- View (`v2`, [col1#x])
+ +- Project [cast(col1#x as string) AS col1#x]
+ +- Project [col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT * FROM (
+ SELECT col1 AS c, 'col1' FROM v2 ORDER BY col1
+) ORDER BY 1
+-- !query analysis
+Sort [c#x ASC NULLS FIRST], true
++- Project [c#x, col1#x]
+ +- SubqueryAlias __auto_generated_subquery_name
+ +- Sort [col1#x ASC NULLS FIRST], true
+ +- Project [col1#x AS c#x, col1 AS col1#x]
+ +- SubqueryAlias v2
+ +- View (`v2`, [col1#x])
+ +- Project [cast(col1#x as string) AS col1#x]
+ +- Project [col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT * FROM (
+ SELECT col1 AS c, 'col1' FROM v2 GROUP BY ALL ORDER BY col1
+) ORDER BY 1
+-- !query analysis
+Sort [c#x ASC NULLS FIRST], true
++- Project [c#x, col1#x]
+ +- SubqueryAlias __auto_generated_subquery_name
+ +- Sort [col1#x ASC NULLS FIRST], true
+ +- Aggregate [col1#x, col1], [col1#x AS c#x, col1 AS col1#x]
+ +- SubqueryAlias v2
+ +- View (`v2`, [col1#x])
+ +- Project [cast(col1#x as string) AS col1#x]
+ +- Project [col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT * FROM (
+ SELECT col1 AS c, 'col1' FROM v2 GROUP BY ALL HAVING col1 > 'banana'
+) ORDER BY 1
+-- !query analysis
+Sort [c#x ASC NULLS FIRST], true
++- Project [c#x, col1#x]
+ +- SubqueryAlias __auto_generated_subquery_name
+ +- Filter (c#x > banana)
+ +- Aggregate [col1#x, col1], [col1#x AS c#x, col1 AS col1#x]
+ +- SubqueryAlias v2
+ +- View (`v2`, [col1#x])
+ +- Project [cast(col1#x as string) AS col1#x]
+ +- Project [col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT 'col1', 'col1' FROM v2 ORDER BY col1
+-- !query analysis
+Project [col1#x, col1#x]
++- Sort [col1#x ASC NULLS FIRST], true
+ +- Project [col1 AS col1#x, col1 AS col1#x, col1#x]
+ +- SubqueryAlias v2
+ +- View (`v2`, [col1#x])
+ +- Project [cast(col1#x as string) AS col1#x]
+ +- Project [col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT 'col1', 'col1' FROM v2 GROUP BY ALL ORDER BY col1
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION",
+ "sqlState" : "42703",
+ "messageParameters" : {
+ "objectName" : "`col1`",
+ "proposal" : "`col1`, `col1`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 53,
+ "stopIndex" : 56,
+ "fragment" : "col1"
+ } ]
+}
+
+
+-- !query
+SELECT 'col1', 'col1' FROM v2 GROUP BY ALL HAVING col1 > 'banana'
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "AMBIGUOUS_REFERENCE",
+ "sqlState" : "42704",
+ "messageParameters" : {
+ "name" : "`col1`",
+ "referenceNames" : "[`col1`, `col1`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 51,
+ "stopIndex" : 54,
+ "fragment" : "col1"
+ } ]
+}
+
+
+-- !query
+SELECT col1 AS c, 'col1', 'col1' FROM v2 ORDER BY col1
+-- !query analysis
+Project [c#x, col1#x, col1#x]
++- Sort [col1#x ASC NULLS FIRST], true
+ +- Project [col1#x AS c#x, col1 AS col1#x, col1 AS col1#x, col1#x]
+ +- SubqueryAlias v2
+ +- View (`v2`, [col1#x])
+ +- Project [cast(col1#x as string) AS col1#x]
+ +- Project [col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT col1 AS c, 'col1', 'col1' FROM v2 GROUP BY ALL ORDER BY col1
+-- !query analysis
+Sort [c#x ASC NULLS FIRST], true
++- Aggregate [col1#x, col1, col1], [col1#x AS c#x, col1 AS col1#x, col1 AS col1#x]
+ +- SubqueryAlias v2
+ +- View (`v2`, [col1#x])
+ +- Project [cast(col1#x as string) AS col1#x]
+ +- Project [col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT * FROM (
+ SELECT col1 AS c, 'col1', 'col1' FROM v2 GROUP BY ALL HAVING col1 > 'banana'
+) ORDER BY 1
+-- !query analysis
+Sort [c#x ASC NULLS FIRST], true
++- Project [c#x, col1#x, col1#x]
+ +- SubqueryAlias __auto_generated_subquery_name
+ +- Filter (c#x > banana)
+ +- Aggregate [col1#x, col1, col1], [col1#x AS c#x, col1 AS col1#x, col1 AS col1#x]
+ +- SubqueryAlias v2
+ +- View (`v2`, [col1#x])
+ +- Project [cast(col1#x as string) AS col1#x]
+ +- Project [col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT col1, 'col1' FROM v2 ORDER BY col1
+-- !query analysis
+Sort [col1#x ASC NULLS FIRST], true
++- Project [col1#x, col1 AS col1#x]
+ +- SubqueryAlias v2
+ +- View (`v2`, [col1#x])
+ +- Project [cast(col1#x as string) AS col1#x]
+ +- Project [col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT col1, 'col1' FROM v2 GROUP BY ALL ORDER BY col1
+-- !query analysis
+Sort [col1#x ASC NULLS FIRST], true
++- Aggregate [col1#x, col1], [col1#x, col1 AS col1#x]
+ +- SubqueryAlias v2
+ +- View (`v2`, [col1#x])
+ +- Project [cast(col1#x as string) AS col1#x]
+ +- Project [col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT * FROM (
+ SELECT col1, 'col1' FROM v2 GROUP BY ALL HAVING col1 > 'banana'
+) ORDER BY 1
+-- !query analysis
+Sort [col1#x ASC NULLS FIRST], true
++- Project [col1#x, col1#x]
+ +- SubqueryAlias __auto_generated_subquery_name
+ +- Filter (col1#x > banana)
+ +- Aggregate [col1#x, col1], [col1#x, col1 AS col1#x]
+ +- SubqueryAlias v2
+ +- View (`v2`, [col1#x])
+ +- Project [cast(col1#x as string) AS col1#x]
+ +- Project [col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT 'col1' FROM (SELECT 'a' AS col1, 'b' AS col1) ORDER BY col1
+-- !query analysis
+Sort [col1#x ASC NULLS FIRST], true
++- Project [col1 AS col1#x]
+ +- SubqueryAlias __auto_generated_subquery_name
+ +- Project [a AS col1#x, b AS col1#x]
+ +- OneRowRelation
+
+
+-- !query
+SELECT 'col1' FROM (SELECT 'a' AS col1, 'b' AS col1) GROUP BY ALL ORDER BY col1
+-- !query analysis
+Sort [col1#x ASC NULLS FIRST], true
++- Aggregate [col1], [col1 AS col1#x]
+ +- SubqueryAlias __auto_generated_subquery_name
+ +- Project [a AS col1#x, b AS col1#x]
+ +- OneRowRelation
+
+
+-- !query
+SELECT 'col1' FROM (SELECT 'a' AS col1, 'b' AS col1) GROUP BY ALL HAVING col1 > 'banana'
+-- !query analysis
+Filter (col1#x > banana)
++- Aggregate [col1], [col1 AS col1#x]
+ +- SubqueryAlias __auto_generated_subquery_name
+ +- Project [a AS col1#x, b AS col1#x]
+ +- OneRowRelation
+
+
+-- !query
+SELECT col1 AS c, 'col1' FROM (SELECT 'a' AS col1, 'b' AS col1) ORDER BY col1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "AMBIGUOUS_REFERENCE",
+ "sqlState" : "42704",
+ "messageParameters" : {
+ "name" : "`col1`",
+ "referenceNames" : "[`col1`, `col1`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 11,
+ "fragment" : "col1"
+ } ]
+}
+
+
+-- !query
+SELECT col1 AS c, 'col1' FROM (SELECT 'a' AS col1, 'b' AS col1) GROUP BY ALL ORDER BY col1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "AMBIGUOUS_REFERENCE",
+ "sqlState" : "42704",
+ "messageParameters" : {
+ "name" : "`col1`",
+ "referenceNames" : "[`col1`, `col1`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 11,
+ "fragment" : "col1"
+ } ]
+}
+
+
+-- !query
+SELECT col1 AS c, 'col1' FROM (SELECT 'a' AS col1, 'b' AS col1) GROUP BY ALL HAVING col1 > 'banana'
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "AMBIGUOUS_REFERENCE",
+ "sqlState" : "42704",
+ "messageParameters" : {
+ "name" : "`col1`",
+ "referenceNames" : "[`col1`, `col1`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 11,
+ "fragment" : "col1"
+ } ]
+}
+
+
+-- !query
+DROP VIEW v2
+-- !query analysis
+DropTempViewCommand v2
+
+
+-- !query
+DROP VIEW v1
+-- !query analysis
+DropTempViewCommand v1
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/cast.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/cast.sql.out
index e0687b564d3d1..0113716bdf712 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/cast.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/cast.sql.out
@@ -745,6 +745,97 @@ Project [cast(10.654321 as interval month) AS CAST(10.654321 AS INTERVAL MONTH)#
+- OneRowRelation
+-- !query
+SELECT CAST(TIME '00:01:52' AS tinyint)
+-- !query analysis
+Project [cast(00:01:52 as tinyint) AS CAST(TIME '00:01:52' AS TINYINT)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(TIME '00:01:52' AS smallint)
+-- !query analysis
+Project [cast(00:01:52 as smallint) AS CAST(TIME '00:01:52' AS SMALLINT)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(TIME '00:01:52' AS int)
+-- !query analysis
+Project [cast(00:01:52 as int) AS CAST(TIME '00:01:52' AS INT)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(TIME '00:01:52' AS bigint)
+-- !query analysis
+Project [cast(00:01:52 as bigint) AS CAST(TIME '00:01:52' AS BIGINT)#xL]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(TIME '23:59:59' AS tinyint)
+-- !query analysis
+Project [cast(23:59:59 as tinyint) AS CAST(TIME '23:59:59' AS TINYINT)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(TIME '23:59:59' AS smallint)
+-- !query analysis
+Project [cast(23:59:59 as smallint) AS CAST(TIME '23:59:59' AS SMALLINT)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(TIME '23:59:59' AS int)
+-- !query analysis
+Project [cast(23:59:59 as int) AS CAST(TIME '23:59:59' AS INT)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(TIME '23:59:59' AS bigint)
+-- !query analysis
+Project [cast(23:59:59 as bigint) AS CAST(TIME '23:59:59' AS BIGINT)#xL]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(TIME '00:00:17.5' AS tinyint)
+-- !query analysis
+Project [cast(00:00:17.5 as tinyint) AS CAST(TIME '00:00:17.5' AS TINYINT)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(TIME '00:00:17.5' AS int)
+-- !query analysis
+Project [cast(00:00:17.5 as int) AS CAST(TIME '00:00:17.5' AS INT)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(TIME '00:00:17.9' AS int)
+-- !query analysis
+Project [cast(00:00:17.9 as int) AS CAST(TIME '00:00:17.9' AS INT)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(TIME '00:00:00' AS tinyint)
+-- !query analysis
+Project [cast(00:00:00 as tinyint) AS CAST(TIME '00:00:00' AS TINYINT)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(TIME '00:00:00' AS int)
+-- !query analysis
+Project [cast(00:00:00 as int) AS CAST(TIME '00:00:00' AS INT)#x]
++- OneRowRelation
+
+
-- !query
SELECT '1.23' :: int
-- !query analysis
@@ -880,6 +971,34 @@ Project [cast(10.123456 as interval day to second) AS CAST(10.123456 AS INTERVAL
+- OneRowRelation
+-- !query
+SELECT TIME '00:01:52' :: tinyint
+-- !query analysis
+Project [cast(00:01:52 as tinyint) AS CAST(TIME '00:01:52' AS TINYINT)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT TIME '00:01:52' :: int
+-- !query analysis
+Project [cast(00:01:52 as int) AS CAST(TIME '00:01:52' AS INT)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT TIME '23:59:59' :: tinyint
+-- !query analysis
+Project [cast(23:59:59 as tinyint) AS CAST(TIME '23:59:59' AS TINYINT)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT TIME '23:59:59' :: int
+-- !query analysis
+Project [cast(23:59:59 as int) AS CAST(TIME '23:59:59' AS INT)#x]
++- OneRowRelation
+
+
-- !query
SELECT '1.23' :: int :: long
-- !query analysis
@@ -960,3 +1079,122 @@ org.apache.spark.sql.catalyst.parser.ParseException
"hint" : ""
}
}
+
+
+-- !query
+SELECT CAST(time '00:00:00' AS decimal(1, 0))
+-- !query analysis
+Project [cast(00:00:00 as decimal(1,0)) AS CAST(TIME '00:00:00' AS DECIMAL(1,0))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '12:00:00' AS decimal(7, 2))
+-- !query analysis
+Project [cast(12:00:00 as decimal(7,2)) AS CAST(TIME '12:00:00' AS DECIMAL(7,2))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '01:30:45' AS decimal(8, 3))
+-- !query analysis
+Project [cast(01:30:45 as decimal(8,3)) AS CAST(TIME '01:30:45' AS DECIMAL(8,3))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '23:59:59' AS decimal(9, 4))
+-- !query analysis
+Project [cast(23:59:59 as decimal(9,4)) AS CAST(TIME '23:59:59' AS DECIMAL(9,4))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '01:02:03' AS decimal(15, 9))
+-- !query analysis
+Project [cast(01:02:03 as decimal(15,9)) AS CAST(TIME '01:02:03' AS DECIMAL(15,9))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '10:20:30' AS decimal(20, 10))
+-- !query analysis
+Project [cast(10:20:30 as decimal(20,10)) AS CAST(TIME '10:20:30' AS DECIMAL(20,10))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '23:59:59.001' AS decimal(8, 3))
+-- !query analysis
+Project [cast(23:59:59.001 as decimal(8,3)) AS CAST(TIME '23:59:59.001' AS DECIMAL(8,3))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '23:59:59.999999' AS decimal(11, 6))
+-- !query analysis
+Project [cast(23:59:59.999999 as decimal(11,6)) AS CAST(TIME '23:59:59.999999' AS DECIMAL(11,6))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '23:59:59.999999999' AS decimal(14, 9))
+-- !query analysis
+Project [cast(23:59:59.999999 as decimal(14,9)) AS CAST(TIME '23:59:59.999999' AS DECIMAL(14,9))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '23:59:59.999999999' AS decimal(20, 10))
+-- !query analysis
+Project [cast(23:59:59.999999 as decimal(20,10)) AS CAST(TIME '23:59:59.999999' AS DECIMAL(20,10))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '00:01:00' AS decimal(1, 0))
+-- !query analysis
+Project [cast(00:01:00 as decimal(1,0)) AS CAST(TIME '00:01:00' AS DECIMAL(1,0))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '01:00:00' AS decimal(3, 0))
+-- !query analysis
+Project [cast(01:00:00 as decimal(3,0)) AS CAST(TIME '01:00:00' AS DECIMAL(3,0))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '10:00:00' AS decimal(5, 2))
+-- !query analysis
+Project [cast(10:00:00 as decimal(5,2)) AS CAST(TIME '10:00:00' AS DECIMAL(5,2))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '23:59:59.9' AS decimal(6, 0))
+-- !query analysis
+Project [cast(23:59:59.9 as decimal(6,0)) AS CAST(TIME '23:59:59.9' AS DECIMAL(6,0))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '23:59:59.999' AS decimal(8, 2))
+-- !query analysis
+Project [cast(23:59:59.999 as decimal(8,2)) AS CAST(TIME '23:59:59.999' AS DECIMAL(8,2))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '23:59:59.999999' AS decimal(11, 5))
+-- !query analysis
+Project [cast(23:59:59.999999 as decimal(11,5)) AS CAST(TIME '23:59:59.999999' AS DECIMAL(11,5))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT CAST(time '23:59:59.999999999' AS decimal(14, 8))
+-- !query analysis
+Project [cast(23:59:59.999999 as decimal(14,8)) AS CAST(TIME '23:59:59.999999' AS DECIMAL(14,8))#x]
++- OneRowRelation
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/decimalArithmeticOperations.sql.out
index d75f4d41bd425..5c136f6fe1779 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/decimalArithmeticOperations.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/decimalArithmeticOperations.sql.out
@@ -51,7 +51,7 @@ insert into decimals_test values(1, 100.0, 999.0), (2, 12345.123, 12345.123),
(3, 0.1234567891011, 1234.1), (4, 123456789123456789.0, 1.123456789123456789)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/decimals_test, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/decimals_test], Append, `spark_catalog`.`default`.`decimals_test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/decimals_test), [id, a, b]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,18)) AS a#x, cast(col3#x as decimal(38,18)) AS b#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,18)) AS a#x, cast(col3#x as decimal(38,18)) AS b#x]
+- LocalRelation [col1#x, col2#x, col3#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/double-quoted-identifiers.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/double-quoted-identifiers.sql.out
index a02bf525f947d..5c8b549a50658 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/double-quoted-identifiers.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/double-quoted-identifiers.sql.out
@@ -111,6 +111,20 @@ org.apache.spark.sql.catalyst.parser.ParseException
}
+-- !query
+select 1 from "not_""exists"
+-- !query analysis
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"not_\"\"exists\"'",
+ "hint" : ""
+ }
+}
+
+
-- !query
SELECT 1 FROM `hello`
-- !query analysis
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/higher-order-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/higher-order-functions.sql.out
index 1281b19eb2f86..d4717b18bac5b 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/higher-order-functions.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/higher-order-functions.sql.out
@@ -370,3 +370,741 @@ select aggregate(split('abcdefgh',''), array(array('')), (acc, x) -> array(array
-- !query analysis
Project [aggregate(split(abcdefgh, , -1), array(array()), lambdafunction(array(array(lambda x#x)), lambda acc#x, lambda x#x, false), lambdafunction(lambda id#x, lambda id#x, false)) AS aggregate(split(abcdefgh, , -1), array(array()), lambdafunction(array(array(namedlambdavariable())), namedlambdavariable(), namedlambdavariable()), lambdafunction(namedlambdavariable(), namedlambdavariable()))#x]
+- OneRowRelation
+
+
+-- !query
+select aggregate(array(1, 2, 3), 0, 100) as aggregate_int_literal
+-- !query analysis
+Project [aggregate(array(1, 2, 3), 0, lambdafunction(100, lambda col0#x, lambda col1#x, true), lambdafunction(lambda id#x, lambda id#x, false)) AS aggregate_int_literal#x]
++- OneRowRelation
+
+
+-- !query
+select aggregate(array(1, 2, 3), map(), map('result', 999)) as aggregate_map_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(map(result, 999), namedlambdavariable(), namedlambdavariable())\"",
+ "inputType" : "\"MAP\"",
+ "paramIndex" : "third",
+ "requiredType" : "\"MAP\"",
+ "sqlExpr" : "\"aggregate(array(1, 2, 3), map(), lambdafunction(map(result, 999), namedlambdavariable(), namedlambdavariable()), lambdafunction(namedlambdavariable(), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 59,
+ "fragment" : "aggregate(array(1, 2, 3), map(), map('result', 999))"
+ } ]
+}
+
+
+-- !query
+select aggregate(array(1, 2, 3), struct('init', 0), struct('final', 999)) as aggregate_struct_literal
+-- !query analysis
+Project [aggregate(array(1, 2, 3), struct(col1, init, col2, 0), lambdafunction(struct(col1, final, col2, 999), lambda col0#x, lambda col1#x, true), lambdafunction(lambda id#x, lambda id#x, false)) AS aggregate_struct_literal#x]
++- OneRowRelation
+
+
+-- !query
+select aggregate(array(1, 2, 3), array(), array('result')) as aggregate_array_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(array(result), namedlambdavariable(), namedlambdavariable())\"",
+ "inputType" : "\"ARRAY\"",
+ "paramIndex" : "third",
+ "requiredType" : "\"ARRAY\"",
+ "sqlExpr" : "\"aggregate(array(1, 2, 3), array(), lambdafunction(array(result), namedlambdavariable(), namedlambdavariable()), lambdafunction(namedlambdavariable(), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 58,
+ "fragment" : "aggregate(array(1, 2, 3), array(), array('result'))"
+ } ]
+}
+
+
+-- !query
+select array_sort(array(3, 1, 2), 1) as array_sort_int_literal
+-- !query analysis
+Project [array_sort(array(3, 1, 2), lambdafunction(1, lambda col0#x, lambda col1#x, true), false) AS array_sort_int_literal#x]
++- OneRowRelation
+
+
+-- !query
+select array_sort(array(3, 1, 2), map('compare', 0)) as array_sort_map_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_RETURN_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "actualType" : "\"MAP\"",
+ "expectedType" : "\"INT\"",
+ "functionName" : "`lambdafunction`",
+ "sqlExpr" : "\"array_sort(array(3, 1, 2), lambdafunction(map(compare, 0), namedlambdavariable(), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 52,
+ "fragment" : "array_sort(array(3, 1, 2), map('compare', 0))"
+ } ]
+}
+
+
+-- !query
+select array_sort(array(3, 1, 2), struct('result', 0)) as array_sort_struct_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_RETURN_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "actualType" : "\"STRUCT\"",
+ "expectedType" : "\"INT\"",
+ "functionName" : "`lambdafunction`",
+ "sqlExpr" : "\"array_sort(array(3, 1, 2), lambdafunction(struct(result, 0), namedlambdavariable(), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 54,
+ "fragment" : "array_sort(array(3, 1, 2), struct('result', 0))"
+ } ]
+}
+
+
+-- !query
+select array_sort(array(3, 1, 2), array(0)) as array_sort_array_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_RETURN_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "actualType" : "\"ARRAY\"",
+ "expectedType" : "\"INT\"",
+ "functionName" : "`lambdafunction`",
+ "sqlExpr" : "\"array_sort(array(3, 1, 2), lambdafunction(array(0), namedlambdavariable(), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 43,
+ "fragment" : "array_sort(array(3, 1, 2), array(0))"
+ } ]
+}
+
+
+-- !query
+select exists(array(1, 2, 3), 1) as exists_int_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(1, namedlambdavariable())\"",
+ "inputType" : "\"INT\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"exists(array(1, 2, 3), lambdafunction(1, namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 32,
+ "fragment" : "exists(array(1, 2, 3), 1)"
+ } ]
+}
+
+
+-- !query
+select exists(array(1, 2, 3), map('found', true)) as exists_map_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(map(found, true), namedlambdavariable())\"",
+ "inputType" : "\"MAP\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"exists(array(1, 2, 3), lambdafunction(map(found, true), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 49,
+ "fragment" : "exists(array(1, 2, 3), map('found', true))"
+ } ]
+}
+
+
+-- !query
+select exists(array(1, 2, 3), struct('exists', true)) as exists_struct_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(struct(exists, true), namedlambdavariable())\"",
+ "inputType" : "\"STRUCT\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"exists(array(1, 2, 3), lambdafunction(struct(exists, true), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 53,
+ "fragment" : "exists(array(1, 2, 3), struct('exists', true))"
+ } ]
+}
+
+
+-- !query
+select exists(array(1, 2, 3), array(true)) as exists_array_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(array(true), namedlambdavariable())\"",
+ "inputType" : "\"ARRAY\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"exists(array(1, 2, 3), lambdafunction(array(true), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 42,
+ "fragment" : "exists(array(1, 2, 3), array(true))"
+ } ]
+}
+
+
+-- !query
+select filter(array(1, 2, 3), 1) as filter_int_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(1, namedlambdavariable())\"",
+ "inputType" : "\"INT\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"filter(array(1, 2, 3), lambdafunction(1, namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 32,
+ "fragment" : "filter(array(1, 2, 3), 1)"
+ } ]
+}
+
+
+-- !query
+select filter(array(1, 2, 3), map('key', 'value')) as filter_map_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(map(key, value), namedlambdavariable())\"",
+ "inputType" : "\"MAP\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"filter(array(1, 2, 3), lambdafunction(map(key, value), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 50,
+ "fragment" : "filter(array(1, 2, 3), map('key', 'value'))"
+ } ]
+}
+
+
+-- !query
+select filter(array(1, 2, 3), struct('valid', true)) as filter_struct_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(struct(valid, true), namedlambdavariable())\"",
+ "inputType" : "\"STRUCT\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"filter(array(1, 2, 3), lambdafunction(struct(valid, true), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 52,
+ "fragment" : "filter(array(1, 2, 3), struct('valid', true))"
+ } ]
+}
+
+
+-- !query
+select filter(array(1, 2, 3), array(true, false)) as filter_array_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(array(true, false), namedlambdavariable())\"",
+ "inputType" : "\"ARRAY\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"filter(array(1, 2, 3), lambdafunction(array(true, false), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 49,
+ "fragment" : "filter(array(1, 2, 3), array(true, false))"
+ } ]
+}
+
+
+-- !query
+select forall(array(1, 2, 3), 1) as forall_int_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(1, namedlambdavariable())\"",
+ "inputType" : "\"INT\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"forall(array(1, 2, 3), lambdafunction(1, namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 32,
+ "fragment" : "forall(array(1, 2, 3), 1)"
+ } ]
+}
+
+
+-- !query
+select forall(array(1, 2, 3), map('all', true)) as forall_map_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(map(all, true), namedlambdavariable())\"",
+ "inputType" : "\"MAP\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"forall(array(1, 2, 3), lambdafunction(map(all, true), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 47,
+ "fragment" : "forall(array(1, 2, 3), map('all', true))"
+ } ]
+}
+
+
+-- !query
+select forall(array(1, 2, 3), struct('all', true)) as forall_struct_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(struct(all, true), namedlambdavariable())\"",
+ "inputType" : "\"STRUCT\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"forall(array(1, 2, 3), lambdafunction(struct(all, true), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 50,
+ "fragment" : "forall(array(1, 2, 3), struct('all', true))"
+ } ]
+}
+
+
+-- !query
+select forall(array(1, 2, 3), array(true, true)) as forall_array_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(array(true, true), namedlambdavariable())\"",
+ "inputType" : "\"ARRAY\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"forall(array(1, 2, 3), lambdafunction(array(true, true), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 48,
+ "fragment" : "forall(array(1, 2, 3), array(true, true))"
+ } ]
+}
+
+
+-- !query
+select map_filter(map('a', 1, 'b', 2), 1) as map_filter_int_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(1, namedlambdavariable(), namedlambdavariable())\"",
+ "inputType" : "\"INT\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"map_filter(map(a, 1, b, 2), lambdafunction(1, namedlambdavariable(), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 41,
+ "fragment" : "map_filter(map('a', 1, 'b', 2), 1)"
+ } ]
+}
+
+
+-- !query
+select map_filter(map('a', 1, 'b', 2), map('keep', true)) as map_filter_map_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(map(keep, true), namedlambdavariable(), namedlambdavariable())\"",
+ "inputType" : "\"MAP\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"map_filter(map(a, 1, b, 2), lambdafunction(map(keep, true), namedlambdavariable(), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 57,
+ "fragment" : "map_filter(map('a', 1, 'b', 2), map('keep', true))"
+ } ]
+}
+
+
+-- !query
+select map_filter(map('a', 1, 'b', 2), struct('filter', true)) as map_filter_struct_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(struct(filter, true), namedlambdavariable(), namedlambdavariable())\"",
+ "inputType" : "\"STRUCT\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"map_filter(map(a, 1, b, 2), lambdafunction(struct(filter, true), namedlambdavariable(), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 62,
+ "fragment" : "map_filter(map('a', 1, 'b', 2), struct('filter', true))"
+ } ]
+}
+
+
+-- !query
+select map_filter(map('a', 1, 'b', 2), array(true)) as map_filter_array_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(array(true), namedlambdavariable(), namedlambdavariable())\"",
+ "inputType" : "\"ARRAY\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"map_filter(map(a, 1, b, 2), lambdafunction(array(true), namedlambdavariable(), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 51,
+ "fragment" : "map_filter(map('a', 1, 'b', 2), array(true))"
+ } ]
+}
+
+
+-- !query
+select map_zip_with(map('a', 1), map('a', 10), 100) as map_zipwith_int_literal
+-- !query analysis
+Project [map_zip_with(map(a, 1), map(a, 10), lambdafunction(100, lambda col0#x, lambda col1#x, lambda col2#x, true)) AS map_zipwith_int_literal#x]
++- OneRowRelation
+
+
+-- !query
+select map_zip_with(map('a', 1), map('a', 10), map('merged', true)) as map_zipwith_map_literal
+-- !query analysis
+Project [map_zip_with(map(a, 1), map(a, 10), lambdafunction(map(merged, true), lambda col0#x, lambda col1#x, lambda col2#x, true)) AS map_zipwith_map_literal#x]
++- OneRowRelation
+
+
+-- !query
+select map_zip_with(map('a', 1), map('a', 10), struct('left', 1, 'right', 10)) as map_zipwith_struct_literal
+-- !query analysis
+Project [map_zip_with(map(a, 1), map(a, 10), lambdafunction(struct(col1, left, col2, 1, col3, right, col4, 10), lambda col0#x, lambda col1#x, lambda col2#x, true)) AS map_zipwith_struct_literal#x]
++- OneRowRelation
+
+
+-- !query
+select map_zip_with(map('a', 1), map('a', 10), array('combined')) as map_zipwith_array_literal
+-- !query analysis
+Project [map_zip_with(map(a, 1), map(a, 10), lambdafunction(array(combined), lambda col0#x, lambda col1#x, lambda col2#x, true)) AS map_zipwith_array_literal#x]
++- OneRowRelation
+
+
+-- !query
+select reduce(array(1, 2, 3), 0, 100) as reduce_int_literal
+-- !query analysis
+Project [reduce(array(1, 2, 3), 0, lambdafunction(100, lambda col0#x, lambda col1#x, true), lambdafunction(lambda id#x, lambda id#x, false)) AS reduce_int_literal#x]
++- OneRowRelation
+
+
+-- !query
+select reduce(array(1, 2, 3), map(), map('result', 999)) as reduce_map_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(map(result, 999), namedlambdavariable(), namedlambdavariable())\"",
+ "inputType" : "\"MAP\"",
+ "paramIndex" : "third",
+ "requiredType" : "\"MAP\"",
+ "sqlExpr" : "\"reduce(array(1, 2, 3), map(), lambdafunction(map(result, 999), namedlambdavariable(), namedlambdavariable()), lambdafunction(namedlambdavariable(), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 56,
+ "fragment" : "reduce(array(1, 2, 3), map(), map('result', 999))"
+ } ]
+}
+
+
+-- !query
+select reduce(array(1, 2, 3), struct('init', 0), struct('final', 999)) as reduce_struct_literal
+-- !query analysis
+Project [reduce(array(1, 2, 3), struct(col1, init, col2, 0), lambdafunction(struct(col1, final, col2, 999), lambda col0#x, lambda col1#x, true), lambdafunction(lambda id#x, lambda id#x, false)) AS reduce_struct_literal#x]
++- OneRowRelation
+
+
+-- !query
+select reduce(array(1, 2, 3), array(), array('result')) as reduce_array_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"lambdafunction(array(result), namedlambdavariable(), namedlambdavariable())\"",
+ "inputType" : "\"ARRAY\"",
+ "paramIndex" : "third",
+ "requiredType" : "\"ARRAY\"",
+ "sqlExpr" : "\"reduce(array(1, 2, 3), array(), lambdafunction(array(result), namedlambdavariable(), namedlambdavariable()), lambdafunction(namedlambdavariable(), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 55,
+ "fragment" : "reduce(array(1, 2, 3), array(), array('result'))"
+ } ]
+}
+
+
+-- !query
+select transform(array(1, 2, 3), 42) as transform_int_literal
+-- !query analysis
+Project [transform(array(1, 2, 3), lambdafunction(42, lambda col0#x, true)) AS transform_int_literal#x]
++- OneRowRelation
+
+
+-- !query
+select transform(array(1, 2, 3), map('key', 'value')) as transform_map_literal
+-- !query analysis
+Project [transform(array(1, 2, 3), lambdafunction(map(key, value), lambda col0#x, true)) AS transform_map_literal#x]
++- OneRowRelation
+
+
+-- !query
+select transform(array(1, 2, 3), struct('id', 99, 'name', 'test')) as transform_struct_literal
+-- !query analysis
+Project [transform(array(1, 2, 3), lambdafunction(struct(col1, id, col2, 99, col3, name, col4, test), lambda col0#x, true)) AS transform_struct_literal#x]
++- OneRowRelation
+
+
+-- !query
+select transform(array(1, 2, 3), array('a', 'b')) as transform_array_literal
+-- !query analysis
+Project [transform(array(1, 2, 3), lambdafunction(array(a, b), lambda col0#x, true)) AS transform_array_literal#x]
++- OneRowRelation
+
+
+-- !query
+select transform_keys(map('a', 1, 'b', 2), 42) as transform_keys_int_literal
+-- !query analysis
+Project [transform_keys(map(a, 1, b, 2), lambdafunction(42, lambda col0#x, lambda col1#x, true)) AS transform_keys_int_literal#x]
++- OneRowRelation
+
+
+-- !query
+select transform_keys(map('a', 1, 'b', 2), map('new', 'key')) as transform_keys_map_literal
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.INVALID_MAP_KEY_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "keyType" : "\"MAP\"",
+ "sqlExpr" : "\"transform_keys(map(a, 1, b, 2), lambdafunction(map(new, key), namedlambdavariable(), namedlambdavariable()))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 61,
+ "fragment" : "transform_keys(map('a', 1, 'b', 2), map('new', 'key'))"
+ } ]
+}
+
+
+-- !query
+select transform_keys(map('a', 1, 'b', 2), struct('key', 'value')) as transform_keys_struct_literal
+-- !query analysis
+Project [transform_keys(map(a, 1, b, 2), lambdafunction(struct(col1, key, col2, value), lambda col0#x, lambda col1#x, true)) AS transform_keys_struct_literal#x]
++- OneRowRelation
+
+
+-- !query
+select transform_keys(map('a', 1, 'b', 2), array('new_key')) as transform_keys_array_literal
+-- !query analysis
+Project [transform_keys(map(a, 1, b, 2), lambdafunction(array(new_key), lambda col0#x, lambda col1#x, true)) AS transform_keys_array_literal#x]
++- OneRowRelation
+
+
+-- !query
+select transform_values(map('a', 1, 'b', 2), 999) as transform_values_int_literal
+-- !query analysis
+Project [transform_values(map(a, 1, b, 2), lambdafunction(999, lambda col0#x, lambda col1#x, true)) AS transform_values_int_literal#x]
++- OneRowRelation
+
+
+-- !query
+select transform_values(map('a', 1, 'b', 2), map('new', 'value')) as transform_values_map_literal
+-- !query analysis
+Project [transform_values(map(a, 1, b, 2), lambdafunction(map(new, value), lambda col0#x, lambda col1#x, true)) AS transform_values_map_literal#x]
++- OneRowRelation
+
+
+-- !query
+select transform_values(map('a', 1, 'b', 2), struct('val', 999)) as transform_values_struct_literal
+-- !query analysis
+Project [transform_values(map(a, 1, b, 2), lambdafunction(struct(col1, val, col2, 999), lambda col0#x, lambda col1#x, true)) AS transform_values_struct_literal#x]
++- OneRowRelation
+
+
+-- !query
+select transform_values(map('a', 1, 'b', 2), array('new_value')) as transform_values_array_literal
+-- !query analysis
+Project [transform_values(map(a, 1, b, 2), lambdafunction(array(new_value), lambda col0#x, lambda col1#x, true)) AS transform_values_array_literal#x]
++- OneRowRelation
+
+
+-- !query
+select zip_with(array(1, 2, 3), array(4, 5, 6), 100) as zipwith_int_literal
+-- !query analysis
+Project [zip_with(array(1, 2, 3), array(4, 5, 6), lambdafunction(100, lambda col0#x, lambda col1#x, true)) AS zipwith_int_literal#x]
++- OneRowRelation
+
+
+-- !query
+select zip_with(array(1, 2, 3), array(4, 5, 6), map('merged', true)) as zipwith_map_literal
+-- !query analysis
+Project [zip_with(array(1, 2, 3), array(4, 5, 6), lambdafunction(map(merged, true), lambda col0#x, lambda col1#x, true)) AS zipwith_map_literal#x]
++- OneRowRelation
+
+
+-- !query
+select zip_with(array(1, 2, 3), array(4, 5, 6), struct('left', 1, 'right', 2)) as zipwith_struct_literal
+-- !query analysis
+Project [zip_with(array(1, 2, 3), array(4, 5, 6), lambdafunction(struct(col1, left, col2, 1, col3, right, col4, 2), lambda col0#x, lambda col1#x, true)) AS zipwith_struct_literal#x]
++- OneRowRelation
+
+
+-- !query
+select zip_with(array(1, 2, 3), array(4, 5, 6), array('combined')) as zipwith_array_literal
+-- !query analysis
+Project [zip_with(array(1, 2, 3), array(4, 5, 6), lambdafunction(array(combined), lambda col0#x, lambda col1#x, true)) AS zipwith_array_literal#x]
++- OneRowRelation
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/literals.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/literals.sql.out
index 9724fb01a1a81..39951f9ef3adc 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/literals.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/literals.sql.out
@@ -706,3 +706,51 @@ select -0, -0.0
-- !query analysis
Project [0 AS 0#x, 0.0 AS 0.0#x]
+- OneRowRelation
+
+
+-- !query
+SELECT "S""par""k" AS c1, "S\"par\"k" AS c2, 'S""par""k' AS c3
+-- !query analysis
+Project [S"par"k AS c1#x, S"par"k AS c2#x, S""par""k AS c3#x]
++- OneRowRelation
+
+
+-- !query
+SELECT 'S''par''k' AS c1, 'S\'par\'k' AS c2, "S''par''k" AS c3
+-- !query analysis
+Project [S'par'k AS c1#x, S'par'k AS c2#x, S''par''k AS c3#x]
++- OneRowRelation
+
+
+-- !query
+SELECT "S" "par" "k" AS c1, 'S' 'par' 'k' AS c2, "S" 'par' "k" AS c3, 'S' "par" 'k' AS c4, "S"'par'"k" AS c5, 'S'"par"'k' AS c6
+-- !query analysis
+Project [Spark AS c1#x, Spark AS c2#x, Spark AS c3#x, Spark AS c4#x, Spark AS c5#x, Spark AS c6#x]
++- OneRowRelation
+
+
+-- !query
+SET spark.sql.legacy.consecutiveStringLiterals.enabled=true
+-- !query analysis
+SetCommand (spark.sql.legacy.consecutiveStringLiterals.enabled,Some(true))
+
+
+-- !query
+SELECT "S""par""k" AS c1, "S\"par\"k" AS c2, 'S""par""k' AS c3
+-- !query analysis
+Project [Spark AS c1#x, S"par"k AS c2#x, S""par""k AS c3#x]
++- OneRowRelation
+
+
+-- !query
+SELECT 'S''par''k' AS c1, 'S\'par\'k' AS c2, "S''par''k" AS c3
+-- !query analysis
+Project [Spark AS c1#x, S'par'k AS c2#x, S''par''k AS c3#x]
++- OneRowRelation
+
+
+-- !query
+SELECT "S" "par" "k" AS c1, 'S' 'par' 'k' AS c2, "S" 'par' "k" AS c3, 'S' "par" 'k' AS c4, "S"'par'"k" AS c5, 'S'"par"'k' AS c6
+-- !query analysis
+Project [Spark AS c1#x, Spark AS c2#x, Spark AS c3#x, Spark AS c4#x, Spark AS c5#x, Spark AS c6#x]
++- OneRowRelation
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/string-functions.sql.out
index ee4ad922fa8a5..b352d0edf896b 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/string-functions.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/string-functions.sql.out
@@ -128,6 +128,41 @@ Project [split(hello, , -1) AS split(hello, , -1)#x]
+- OneRowRelation
+-- !query
+SELECT split('hello', '', 0)
+-- !query analysis
+Project [split(hello, , 0) AS split(hello, , 0)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT split('hello', '', 1)
+-- !query analysis
+Project [split(hello, , 1) AS split(hello, , 1)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT split('hello', '', 3)
+-- !query analysis
+Project [split(hello, , 3) AS split(hello, , 3)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT split('hello', '', 5)
+-- !query analysis
+Project [split(hello, , 5) AS split(hello, , 5)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT split('hello', '', 100)
+-- !query analysis
+Project [split(hello, , 100) AS split(hello, , 100)#x]
++- OneRowRelation
+
+
-- !query
SELECT split('', '')
-- !query analysis
@@ -135,6 +170,27 @@ Project [split(, , -1) AS split(, , -1)#x]
+- OneRowRelation
+-- !query
+SELECT split('', '', -1)
+-- !query analysis
+Project [split(, , -1) AS split(, , -1)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT split('', '', 0)
+-- !query analysis
+Project [split(, , 0) AS split(, , 0)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT split('', '', 1)
+-- !query analysis
+Project [split(, , 1) AS split(, , 1)#x]
++- OneRowRelation
+
+
-- !query
SELECT split('abc', null)
-- !query analysis
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/timestamp.sql.out
index dcfd783b648f8..963424bdaa09e 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/timestamp.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/timestamp.sql.out
@@ -157,6 +157,235 @@ Project [make_timestamp(1, 1, 1, 1, 1, cast(999.999999 as decimal(16,6)), None,
+- OneRowRelation
+-- !query
+SELECT make_timestamp(make_date(2021, 07, 11))
+-- !query analysis
+Project [make_timestamp(make_date(2021, 7, 11, false), None, None, Some(America/Los_Angeles)) AS make_timestamp(make_date(2021, 7, 11))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT make_timestamp(DATE'1970-01-01')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT make_timestamp(NULL)
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678))
+-- !query analysis
+Project [make_timestamp(make_date(2021, 7, 11, false), Some(make_time(6, 30, cast(45.678 as decimal(16,6)))), None, Some(America/Los_Angeles)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678))#x]
++- OneRowRelation
+
+
+-- !query
+SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT make_timestamp(NULL, TIME'00:00:00')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT make_timestamp(DATE'1970-01-01', NULL)
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT make_timestamp(NULL, NULL)
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'MIT')
+-- !query analysis
+Project [make_timestamp(make_date(2021, 7, 11, false), Some(make_time(6, 30, cast(45.678 as decimal(16,6)))), Some(MIT), Some(America/Los_Angeles)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), MIT)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'PST')
+-- !query analysis
+Project [make_timestamp(make_date(2021, 7, 11, false), Some(make_time(6, 30, cast(45.678 as decimal(16,6)))), Some(PST), Some(America/Los_Angeles)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), PST)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'UTC')
+-- !query analysis
+Project [make_timestamp(make_date(2021, 7, 11, false), Some(make_time(6, 30, cast(45.678 as decimal(16,6)))), Some(UTC), Some(America/Los_Angeles)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), UTC)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'CET')
+-- !query analysis
+Project [make_timestamp(make_date(2021, 7, 11, false), Some(make_time(6, 30, cast(45.678 as decimal(16,6)))), Some(CET), Some(America/Los_Angeles)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), CET)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT make_timestamp(make_date(2021, 07, 11), make_time(6, 30, 45.678), 'JST')
+-- !query analysis
+Project [make_timestamp(make_date(2021, 7, 11, false), Some(make_time(6, 30, cast(45.678 as decimal(16,6)))), Some(JST), Some(America/Los_Angeles)) AS make_timestamp(make_date(2021, 7, 11), make_time(6, 30, 45.678), JST)#x]
++- OneRowRelation
+
+
+-- !query
+SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00', 'UTC')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT make_timestamp(NULL, TIME'00:00:00', 'UTC')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT make_timestamp(DATE'1970-01-01', NULL, 'UTC')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT make_timestamp(DATE'1970-01-01', TIME'00:00:00', NULL)
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT make_timestamp(NULL, NULL, 'UTC')
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT make_timestamp(NULL, TIME'00:00:00', NULL)
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT make_timestamp(DATE'1970-01-01', NULL, NULL)
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT make_timestamp(NULL, NULL, NULL)
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT make_timestamp(timestamp '2018-11-17 13:33:33', TIME'0:0:0', 'CET')
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"TIMESTAMP '2018-11-17 13:33:33'\"",
+ "inputType" : "\"TIMESTAMP\"",
+ "paramIndex" : "first",
+ "requiredType" : "\"DATE\"",
+ "sqlExpr" : "\"make_timestamp(TIMESTAMP '2018-11-17 13:33:33', TIME '00:00:00', CET)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 74,
+ "fragment" : "make_timestamp(timestamp '2018-11-17 13:33:33', TIME'0:0:0', 'CET')"
+ } ]
+}
+
+
+-- !query
+SELECT make_timestamp(timestamp_ntz '2018-11-17 13:33:33', TIME'0:0:0', 'CET')
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"TIMESTAMP_NTZ '2018-11-17 13:33:33'\"",
+ "inputType" : "\"TIMESTAMP_NTZ\"",
+ "paramIndex" : "first",
+ "requiredType" : "\"DATE\"",
+ "sqlExpr" : "\"make_timestamp(TIMESTAMP_NTZ '2018-11-17 13:33:33', TIME '00:00:00', CET)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 78,
+ "fragment" : "make_timestamp(timestamp_ntz '2018-11-17 13:33:33', TIME'0:0:0', 'CET')"
+ } ]
+}
+
+
+-- !query
+SELECT make_timestamp(DATE'1970-01-01', timestamp '2018-11-17 13:33:33', 'CET')
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"TIMESTAMP '2018-11-17 13:33:33'\"",
+ "inputType" : "\"TIMESTAMP\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"TIME\"",
+ "sqlExpr" : "\"make_timestamp(DATE '1970-01-01', TIMESTAMP '2018-11-17 13:33:33', CET)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 79,
+ "fragment" : "make_timestamp(DATE'1970-01-01', timestamp '2018-11-17 13:33:33', 'CET')"
+ } ]
+}
+
+
+-- !query
+SELECT make_timestamp(DATE'1970-01-01', timestamp_ntz '2018-11-17 13:33:33', 'CET')
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"TIMESTAMP_NTZ '2018-11-17 13:33:33'\"",
+ "inputType" : "\"TIMESTAMP_NTZ\"",
+ "paramIndex" : "second",
+ "requiredType" : "\"TIME\"",
+ "sqlExpr" : "\"make_timestamp(DATE '1970-01-01', TIMESTAMP_NTZ '2018-11-17 13:33:33', CET)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 83,
+ "fragment" : "make_timestamp(DATE'1970-01-01', timestamp_ntz '2018-11-17 13:33:33', 'CET')"
+ } ]
+}
+
+
-- !query
select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null)
-- !query analysis
@@ -711,45 +940,13 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException
-- !query
select timestamp'2011-11-11 11:11:11' + null
-- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
- "sqlState" : "42K09",
- "messageParameters" : {
- "left" : "\"TIMESTAMP\"",
- "right" : "\"VOID\"",
- "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + NULL)\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 44,
- "fragment" : "timestamp'2011-11-11 11:11:11' + null"
- } ]
-}
+[Analyzer test output redacted due to nondeterminism]
-- !query
select null + timestamp'2011-11-11 11:11:11'
-- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
- "sqlState" : "42K09",
- "messageParameters" : {
- "left" : "\"VOID\"",
- "right" : "\"TIMESTAMP\"",
- "sqlExpr" : "\"(NULL + TIMESTAMP '2011-11-11 11:11:11')\""
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 8,
- "stopIndex" : 44,
- "fragment" : "null + timestamp'2011-11-11 11:11:11'"
- } ]
-}
+[Analyzer test output redacted due to nondeterminism]
-- !query
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/null-handling.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/null-handling.sql.out
index 37d84f6c5fc00..86ee26bd1cf62 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/null-handling.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/null-handling.sql.out
@@ -9,7 +9,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false
insert into t1 values(1,0,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [a, b, c]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x]
++- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -17,7 +17,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
insert into t1 values(2,0,1)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [a, b, c]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x]
++- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -25,7 +25,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
insert into t1 values(3,1,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [a, b, c]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x]
++- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -33,7 +33,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
insert into t1 values(4,1,1)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [a, b, c]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x]
++- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -41,7 +41,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
insert into t1 values(5,null,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [a, b, c]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x]
++- Project [col1#x AS a#x, cast(col2#x as int) AS b#x, col3#x AS c#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -49,7 +49,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
insert into t1 values(6,null,1)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [a, b, c]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x]
++- Project [col1#x AS a#x, cast(col2#x as int) AS b#x, col3#x AS c#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -57,7 +57,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
insert into t1 values(7,null,null)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [a, b, c]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x]
++- Project [col1#x AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x]
+- LocalRelation [col1#x, col2#x, col3#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/order-by-ordinal.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/order-by-ordinal.sql.out
index f1f189517dea7..842d099db396c 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/order-by-ordinal.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/order-by-ordinal.sql.out
@@ -150,6 +150,284 @@ Sort [a#x DESC NULLS LAST], false
+- LocalRelation [a#x, b#x]
+-- !query
+set spark.sql.prioritizeOrdinalResolutionInSort.enabled=true
+-- !query analysis
+SetCommand (spark.sql.prioritizeOrdinalResolutionInSort.enabled,Some(true))
+
+
+-- !query
+SELECT a FROM data ORDER BY 2, b
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "ORDER_BY_POS_OUT_OF_RANGE",
+ "sqlState" : "42805",
+ "messageParameters" : {
+ "index" : "2",
+ "size" : "1"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 29,
+ "stopIndex" : 29,
+ "fragment" : "2"
+ } ]
+}
+
+
+-- !query
+SELECT a FROM data ORDER BY b, 2
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "ORDER_BY_POS_OUT_OF_RANGE",
+ "sqlState" : "42805",
+ "messageParameters" : {
+ "index" : "2",
+ "size" : "1"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 32,
+ "stopIndex" : 32,
+ "fragment" : "2"
+ } ]
+}
+
+
+-- !query
+SELECT a FROM data ORDER BY 'b', 2
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "ORDER_BY_POS_OUT_OF_RANGE",
+ "sqlState" : "42805",
+ "messageParameters" : {
+ "index" : "2",
+ "size" : "1"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 34,
+ "stopIndex" : 34,
+ "fragment" : "2"
+ } ]
+}
+
+
+-- !query
+SELECT a FROM data ORDER BY `b`, 2
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "ORDER_BY_POS_OUT_OF_RANGE",
+ "sqlState" : "42805",
+ "messageParameters" : {
+ "index" : "2",
+ "size" : "1"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 34,
+ "stopIndex" : 34,
+ "fragment" : "2"
+ } ]
+}
+
+
+-- !query
+SELECT a FROM data ORDER BY a, 2
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "ORDER_BY_POS_OUT_OF_RANGE",
+ "sqlState" : "42805",
+ "messageParameters" : {
+ "index" : "2",
+ "size" : "1"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 32,
+ "stopIndex" : 32,
+ "fragment" : "2"
+ } ]
+}
+
+
+-- !query
+SELECT a FROM data ORDER BY b, 3
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "ORDER_BY_POS_OUT_OF_RANGE",
+ "sqlState" : "42805",
+ "messageParameters" : {
+ "index" : "3",
+ "size" : "1"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 32,
+ "stopIndex" : 32,
+ "fragment" : "3"
+ } ]
+}
+
+
+-- !query
+SELECT a, a + 1 FROM data ORDER BY b, 3
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "ORDER_BY_POS_OUT_OF_RANGE",
+ "sqlState" : "42805",
+ "messageParameters" : {
+ "index" : "3",
+ "size" : "2"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 39,
+ "stopIndex" : 39,
+ "fragment" : "3"
+ } ]
+}
+
+
+-- !query
+set spark.sql.prioritizeOrdinalResolutionInSort.enabled=false
+-- !query analysis
+SetCommand (spark.sql.prioritizeOrdinalResolutionInSort.enabled,Some(false))
+
+
+-- !query
+SELECT a FROM data ORDER BY 2, b
+-- !query analysis
+Project [a#x]
++- Sort [b#x ASC NULLS FIRST, b#x ASC NULLS FIRST], true
+ +- Project [a#x, b#x]
+ +- SubqueryAlias data
+ +- View (`data`, [a#x, b#x])
+ +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x]
+ +- Project [a#x, b#x]
+ +- SubqueryAlias data
+ +- LocalRelation [a#x, b#x]
+
+
+-- !query
+SELECT a FROM data ORDER BY b, 2
+-- !query analysis
+Project [a#x]
++- Sort [b#x ASC NULLS FIRST, b#x ASC NULLS FIRST], true
+ +- Project [a#x, b#x]
+ +- SubqueryAlias data
+ +- View (`data`, [a#x, b#x])
+ +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x]
+ +- Project [a#x, b#x]
+ +- SubqueryAlias data
+ +- LocalRelation [a#x, b#x]
+
+
+-- !query
+SELECT a FROM data ORDER BY 'b', 2
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "ORDER_BY_POS_OUT_OF_RANGE",
+ "sqlState" : "42805",
+ "messageParameters" : {
+ "index" : "2",
+ "size" : "1"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 34,
+ "stopIndex" : 34,
+ "fragment" : "2"
+ } ]
+}
+
+
+-- !query
+SELECT a FROM data ORDER BY `b`, 2
+-- !query analysis
+Project [a#x]
++- Sort [b#x ASC NULLS FIRST, b#x ASC NULLS FIRST], true
+ +- Project [a#x, b#x]
+ +- SubqueryAlias data
+ +- View (`data`, [a#x, b#x])
+ +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x]
+ +- Project [a#x, b#x]
+ +- SubqueryAlias data
+ +- LocalRelation [a#x, b#x]
+
+
+-- !query
+SELECT a FROM data ORDER BY a, 2
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "ORDER_BY_POS_OUT_OF_RANGE",
+ "sqlState" : "42805",
+ "messageParameters" : {
+ "index" : "2",
+ "size" : "1"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 32,
+ "stopIndex" : 32,
+ "fragment" : "2"
+ } ]
+}
+
+
+-- !query
+SELECT a FROM data ORDER BY b, 3
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "ORDER_BY_POS_OUT_OF_RANGE",
+ "sqlState" : "42805",
+ "messageParameters" : {
+ "index" : "3",
+ "size" : "2"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 32,
+ "stopIndex" : 32,
+ "fragment" : "3"
+ } ]
+}
+
+
+-- !query
+SELECT a, a + 1 FROM data ORDER BY b, 3
+-- !query analysis
+Project [a#x, (a + 1)#x]
++- Sort [b#x ASC NULLS FIRST, b#x ASC NULLS FIRST], true
+ +- Project [a#x, (a#x + 1) AS (a + 1)#x, b#x]
+ +- SubqueryAlias data
+ +- View (`data`, [a#x, b#x])
+ +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x]
+ +- Project [a#x, b#x]
+ +- SubqueryAlias data
+ +- LocalRelation [a#x, b#x]
+
+
-- !query
set spark.sql.orderByOrdinal=false
-- !query analysis
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/order-by.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/order-by.sql.out
index 9c9305ac632db..920e16ea8687a 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/order-by.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/order-by.sql.out
@@ -454,6 +454,26 @@ Sort [(sum(b) + 1)#xL ASC NULLS FIRST], true
+- LocalRelation [a#x, b#x]
+-- !query
+SELECT col1 FROM VALUES(1,2) GROUP BY col1, col2 ORDER BY col2, col2
+-- !query analysis
+Project [col1#x]
++- Sort [col2#x ASC NULLS FIRST, col2#x ASC NULLS FIRST], true
+ +- Aggregate [col1#x, col2#x], [col1#x, col2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT col1 AS a, a AS b FROM VALUES(1,2) GROUP BY col1, col2 ORDER BY col2, col2
+-- !query analysis
+Project [a#x, b#x]
++- Sort [col2#x ASC NULLS FIRST, col2#x ASC NULLS FIRST], true
+ +- Project [a#x, a#x AS b#x, col2#x]
+ +- Project [col1#x, col2#x, col1#x AS a#x]
+ +- Aggregate [col1#x, col2#x], [col1#x, col2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
-- !query
DROP VIEW IF EXISTS testData
-- !query analysis
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out
index 3de5ec38a2bad..dda0722e21d75 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out
@@ -16,7 +16,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`t`, false
insert into t values (0, 'abc'), (1, 'def')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/t], Append, `spark_catalog`.`default`.`t`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t), [x, y]
-+- Project [cast(col1#x as int) AS x#x, cast(col2#x as string) AS y#x]
++- Project [col1#x AS x#x, col2#x AS y#x]
+- LocalRelation [col1#x, col2#x]
@@ -37,7 +37,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`other`, false
insert into other values (1, 1), (1, 2), (2, 4)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/other, false, JSON, [path=file:[not included in comparison]/{warehouse_dir}/other], Append, `spark_catalog`.`default`.`other`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/other), [a, b]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x]
++- Project [col1#x AS a#x, col2#x AS b#x]
+- LocalRelation [col1#x, col2#x]
@@ -58,7 +58,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`st`, false
insert into st values (1, (2, 3))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/st, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/st], Append, `spark_catalog`.`default`.`st`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/st), [x, col]
-+- Project [cast(col1#x as int) AS x#x, named_struct(i1, cast(col2#x.col1 as int), i2, cast(col2#x.col2 as int)) AS col#x]
++- Project [col1#x AS x#x, named_struct(i1, col2#x.col1, i2, col2#x.col2) AS col#x]
+- LocalRelation [col1#x, col2#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/boolean.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/boolean.sql.out
index 277ab866f97f7..9e5733212c24e 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/boolean.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/boolean.sql.out
@@ -290,7 +290,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`BOOLTBL1`, false
INSERT INTO BOOLTBL1 VALUES (cast('t' as boolean))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/booltbl1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/booltbl1], Append, `spark_catalog`.`default`.`booltbl1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/booltbl1), [f1]
-+- Project [cast(col1#x as boolean) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -298,7 +298,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO BOOLTBL1 VALUES (cast('True' as boolean))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/booltbl1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/booltbl1], Append, `spark_catalog`.`default`.`booltbl1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/booltbl1), [f1]
-+- Project [cast(col1#x as boolean) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -306,7 +306,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO BOOLTBL1 VALUES (cast('true' as boolean))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/booltbl1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/booltbl1], Append, `spark_catalog`.`default`.`booltbl1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/booltbl1), [f1]
-+- Project [cast(col1#x as boolean) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -355,7 +355,7 @@ Project [ AS zero#x, f1#x]
INSERT INTO BOOLTBL1 VALUES (boolean('f'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/booltbl1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/booltbl1], Append, `spark_catalog`.`default`.`booltbl1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/booltbl1), [f1]
-+- Project [cast(col1#x as boolean) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -380,7 +380,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`BOOLTBL2`, false
INSERT INTO BOOLTBL2 VALUES (boolean('f'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/booltbl2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/booltbl2], Append, `spark_catalog`.`default`.`booltbl2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/booltbl2), [f1]
-+- Project [cast(col1#x as boolean) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -388,7 +388,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO BOOLTBL2 VALUES (boolean('false'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/booltbl2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/booltbl2], Append, `spark_catalog`.`default`.`booltbl2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/booltbl2), [f1]
-+- Project [cast(col1#x as boolean) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -396,7 +396,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO BOOLTBL2 VALUES (boolean('False'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/booltbl2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/booltbl2], Append, `spark_catalog`.`default`.`booltbl2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/booltbl2), [f1]
-+- Project [cast(col1#x as boolean) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -404,7 +404,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO BOOLTBL2 VALUES (boolean('FALSE'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/booltbl2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/booltbl2], Append, `spark_catalog`.`default`.`booltbl2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/booltbl2), [f1]
-+- Project [cast(col1#x as boolean) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -593,7 +593,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`BOOLTBL3`, false
INSERT INTO BOOLTBL3 VALUES ('true', true, 1)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/booltbl3, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/booltbl3], Append, `spark_catalog`.`default`.`booltbl3`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/booltbl3), [d, b, o]
-+- Project [cast(col1#x as string) AS d#x, cast(col2#x as boolean) AS b#x, cast(col3#x as int) AS o#x]
++- Project [col1#x AS d#x, col2#x AS b#x, col3#x AS o#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -601,7 +601,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO BOOLTBL3 VALUES ('false', false, 2)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/booltbl3, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/booltbl3], Append, `spark_catalog`.`default`.`booltbl3`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/booltbl3), [d, b, o]
-+- Project [cast(col1#x as string) AS d#x, cast(col2#x as boolean) AS b#x, cast(col3#x as int) AS o#x]
++- Project [col1#x AS d#x, col2#x AS b#x, col3#x AS o#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -609,7 +609,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO BOOLTBL3 VALUES ('null', null, 3)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/booltbl3, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/booltbl3], Append, `spark_catalog`.`default`.`booltbl3`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/booltbl3), [d, b, o]
-+- Project [cast(col1#x as string) AS d#x, cast(col2#x as boolean) AS b#x, cast(col3#x as int) AS o#x]
++- Project [col1#x AS d#x, cast(col2#x as boolean) AS b#x, col3#x AS o#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -641,7 +641,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`booltbl4`, false
INSERT INTO booltbl4 VALUES (false, true, null)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/booltbl4, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/booltbl4], Append, `spark_catalog`.`default`.`booltbl4`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/booltbl4), [isfalse, istrue, isnul]
-+- Project [cast(col1#x as boolean) AS isfalse#x, cast(col2#x as boolean) AS istrue#x, cast(col3#x as boolean) AS isnul#x]
++- Project [col1#x AS isfalse#x, col2#x AS istrue#x, cast(col3#x as boolean) AS isnul#x]
+- LocalRelation [col1#x, col2#x, col3#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/case.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/case.sql.out
index 31d199e4754e0..465da4d2e737a 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/case.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/case.sql.out
@@ -21,7 +21,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`CASE2_TBL`, false
INSERT INTO CASE_TBL VALUES (1, 10.1)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case_tbl], Append, `spark_catalog`.`default`.`case_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case_tbl), [i, f]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as double) AS f#x]
++- Project [col1#x AS i#x, cast(col2#x as double) AS f#x]
+- LocalRelation [col1#x, col2#x]
@@ -29,7 +29,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO CASE_TBL VALUES (2, 20.2)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case_tbl], Append, `spark_catalog`.`default`.`case_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case_tbl), [i, f]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as double) AS f#x]
++- Project [col1#x AS i#x, cast(col2#x as double) AS f#x]
+- LocalRelation [col1#x, col2#x]
@@ -37,7 +37,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO CASE_TBL VALUES (3, -30.3)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case_tbl], Append, `spark_catalog`.`default`.`case_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case_tbl), [i, f]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as double) AS f#x]
++- Project [col1#x AS i#x, cast(col2#x as double) AS f#x]
+- LocalRelation [col1#x, col2#x]
@@ -45,7 +45,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO CASE_TBL VALUES (4, NULL)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case_tbl], Append, `spark_catalog`.`default`.`case_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case_tbl), [i, f]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as double) AS f#x]
++- Project [col1#x AS i#x, cast(col2#x as double) AS f#x]
+- LocalRelation [col1#x, col2#x]
@@ -53,7 +53,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO CASE2_TBL VALUES (1, -1)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case2_tbl], Append, `spark_catalog`.`default`.`case2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case2_tbl), [i, j]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x]
++- Project [col1#x AS i#x, col2#x AS j#x]
+- LocalRelation [col1#x, col2#x]
@@ -61,7 +61,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO CASE2_TBL VALUES (2, -2)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case2_tbl], Append, `spark_catalog`.`default`.`case2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case2_tbl), [i, j]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x]
++- Project [col1#x AS i#x, col2#x AS j#x]
+- LocalRelation [col1#x, col2#x]
@@ -69,7 +69,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO CASE2_TBL VALUES (3, -3)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case2_tbl], Append, `spark_catalog`.`default`.`case2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case2_tbl), [i, j]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x]
++- Project [col1#x AS i#x, col2#x AS j#x]
+- LocalRelation [col1#x, col2#x]
@@ -77,7 +77,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO CASE2_TBL VALUES (2, -4)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case2_tbl], Append, `spark_catalog`.`default`.`case2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case2_tbl), [i, j]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x]
++- Project [col1#x AS i#x, col2#x AS j#x]
+- LocalRelation [col1#x, col2#x]
@@ -85,7 +85,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO CASE2_TBL VALUES (1, NULL)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case2_tbl], Append, `spark_catalog`.`default`.`case2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case2_tbl), [i, j]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x]
++- Project [col1#x AS i#x, cast(col2#x as int) AS j#x]
+- LocalRelation [col1#x, col2#x]
@@ -93,7 +93,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO CASE2_TBL VALUES (NULL, -6)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/case2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/case2_tbl], Append, `spark_catalog`.`default`.`case2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/case2_tbl), [i, j]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x]
++- Project [cast(col1#x as int) AS i#x, col2#x AS j#x]
+- LocalRelation [col1#x, col2#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/create_view.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/create_view.sql.out
index e4e4be8fee7d3..e39f479f6ea65 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/create_view.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/create_view.sql.out
@@ -94,7 +94,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`viewtest_tbl`, false
INSERT INTO viewtest_tbl VALUES (5, 10), (10, 15), (15, 20), (20, 25)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/viewtest_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/viewtest_tbl], Append, `spark_catalog`.`default`.`viewtest_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/viewtest_tbl), [a, b]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x]
++- Project [col1#x AS a#x, col2#x AS b#x]
+- LocalRelation [col1#x, col2#x]
@@ -833,7 +833,7 @@ CreateDataSourceTableCommand `spark_catalog`.`testviewschm2`.`tmptbl`, false
INSERT INTO tmptbl VALUES (1, 1)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/testviewschm2.db/tmptbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/testviewschm2.db/tmptbl], Append, `spark_catalog`.`testviewschm2`.`tmptbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/testviewschm2.db/tmptbl), [i, j]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x]
++- Project [col1#x AS i#x, col2#x AS j#x]
+- LocalRelation [col1#x, col2#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/date.sql.out
index d3ac6a3eb2b5c..16d1ffffd7d95 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/date.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/date.sql.out
@@ -9,7 +9,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`DATE_TBL`, false
INSERT INTO DATE_TBL VALUES (date('1957-04-09'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1]
-+- Project [cast(col1#x as date) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -17,7 +17,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO DATE_TBL VALUES (date('1957-06-13'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1]
-+- Project [cast(col1#x as date) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -25,7 +25,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO DATE_TBL VALUES (date('1996-02-28'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1]
-+- Project [cast(col1#x as date) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -33,7 +33,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO DATE_TBL VALUES (date('1996-02-29'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1]
-+- Project [cast(col1#x as date) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -41,7 +41,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO DATE_TBL VALUES (date('1996-03-01'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1]
-+- Project [cast(col1#x as date) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -49,7 +49,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO DATE_TBL VALUES (date('1996-03-02'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1]
-+- Project [cast(col1#x as date) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -57,7 +57,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO DATE_TBL VALUES (date('1997-02-28'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1]
-+- Project [cast(col1#x as date) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -65,7 +65,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO DATE_TBL VALUES (date('1997-03-01'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1]
-+- Project [cast(col1#x as date) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -73,7 +73,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO DATE_TBL VALUES (date('1997-03-02'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1]
-+- Project [cast(col1#x as date) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -81,7 +81,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO DATE_TBL VALUES (date('2000-04-01'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1]
-+- Project [cast(col1#x as date) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -89,7 +89,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO DATE_TBL VALUES (date('2000-04-02'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1]
-+- Project [cast(col1#x as date) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -97,7 +97,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO DATE_TBL VALUES (date('2000-04-03'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1]
-+- Project [cast(col1#x as date) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -105,7 +105,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO DATE_TBL VALUES (date('2038-04-08'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1]
-+- Project [cast(col1#x as date) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -113,7 +113,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO DATE_TBL VALUES (date('2039-04-09'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1]
-+- Project [cast(col1#x as date) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -121,7 +121,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO DATE_TBL VALUES (date('2040-04-10'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/date_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/date_tbl], Append, `spark_catalog`.`default`.`date_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/date_tbl), [f1]
-+- Project [cast(col1#x as date) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/float4.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/float4.sql.out
index 0618a1d8432ed..445edd15f473f 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/float4.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/float4.sql.out
@@ -9,7 +9,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`FLOAT4_TBL`, false
INSERT INTO FLOAT4_TBL VALUES (float(' 0.0'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float4_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float4_tbl], Append, `spark_catalog`.`default`.`float4_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float4_tbl), [f1]
-+- Project [cast(col1#x as float) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -17,7 +17,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO FLOAT4_TBL VALUES (float('1004.30 '))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float4_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float4_tbl], Append, `spark_catalog`.`default`.`float4_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float4_tbl), [f1]
-+- Project [cast(col1#x as float) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -25,7 +25,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO FLOAT4_TBL VALUES (float(' -34.84 '))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float4_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float4_tbl], Append, `spark_catalog`.`default`.`float4_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float4_tbl), [f1]
-+- Project [cast(col1#x as float) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -33,7 +33,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO FLOAT4_TBL VALUES (float('1.2345678901234e+20'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float4_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float4_tbl], Append, `spark_catalog`.`default`.`float4_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float4_tbl), [f1]
-+- Project [cast(col1#x as float) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -41,7 +41,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO FLOAT4_TBL VALUES (float('1.2345678901234e-20'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float4_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float4_tbl], Append, `spark_catalog`.`default`.`float4_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float4_tbl), [f1]
-+- Project [cast(col1#x as float) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/float8.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/float8.sql.out
index 2f2beda4f1cec..31f315d61bc9d 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/float8.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/float8.sql.out
@@ -9,7 +9,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`FLOAT8_TBL`, false
INSERT INTO FLOAT8_TBL VALUES (double(' 0.0 '))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float8_tbl], Append, `spark_catalog`.`default`.`float8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float8_tbl), [f1]
-+- Project [cast(col1#x as double) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -17,7 +17,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO FLOAT8_TBL VALUES (double('1004.30 '))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float8_tbl], Append, `spark_catalog`.`default`.`float8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float8_tbl), [f1]
-+- Project [cast(col1#x as double) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -25,7 +25,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO FLOAT8_TBL VALUES (double(' -34.84'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float8_tbl], Append, `spark_catalog`.`default`.`float8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float8_tbl), [f1]
-+- Project [cast(col1#x as double) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -33,7 +33,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO FLOAT8_TBL VALUES (double('1.2345678901234e+200'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float8_tbl], Append, `spark_catalog`.`default`.`float8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float8_tbl), [f1]
-+- Project [cast(col1#x as double) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -41,7 +41,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO FLOAT8_TBL VALUES (double('1.2345678901234e-200'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float8_tbl], Append, `spark_catalog`.`default`.`float8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float8_tbl), [f1]
-+- Project [cast(col1#x as double) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -612,7 +612,7 @@ TruncateTableCommand `spark_catalog`.`default`.`float8_tbl`
INSERT INTO FLOAT8_TBL VALUES (double('0.0'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float8_tbl], Append, `spark_catalog`.`default`.`float8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float8_tbl), [f1]
-+- Project [cast(col1#x as double) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -620,7 +620,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO FLOAT8_TBL VALUES (double('-34.84'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float8_tbl], Append, `spark_catalog`.`default`.`float8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float8_tbl), [f1]
-+- Project [cast(col1#x as double) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -628,7 +628,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO FLOAT8_TBL VALUES (double('-1004.30'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float8_tbl], Append, `spark_catalog`.`default`.`float8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float8_tbl), [f1]
-+- Project [cast(col1#x as double) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -636,7 +636,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO FLOAT8_TBL VALUES (double('-1.2345678901234e+200'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float8_tbl], Append, `spark_catalog`.`default`.`float8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float8_tbl), [f1]
-+- Project [cast(col1#x as double) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -644,7 +644,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO FLOAT8_TBL VALUES (double('-1.2345678901234e-200'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/float8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/float8_tbl], Append, `spark_catalog`.`default`.`float8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/float8_tbl), [f1]
-+- Project [cast(col1#x as double) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/groupingsets.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/groupingsets.sql.out
index 82b03e986f9dd..84b280b5e9e73 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/groupingsets.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/groupingsets.sql.out
@@ -33,7 +33,7 @@ insert into gstest2 values
(2, 2, 2, 2, 2, 2, 2, 2)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/gstest2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/gstest2], Append, `spark_catalog`.`default`.`gstest2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/gstest2), [a, b, c, d, e, f, g, h]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x, cast(col4#x as int) AS d#x, cast(col5#x as int) AS e#x, cast(col6#x as int) AS f#x, cast(col7#x as int) AS g#x, cast(col8#x as int) AS h#x]
++- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS d#x, col5#x AS e#x, col6#x AS f#x, col7#x AS g#x, col8#x AS h#x]
+- LocalRelation [col1#x, col2#x, col3#x, col4#x, col5#x, col6#x, col7#x, col8#x]
@@ -49,7 +49,7 @@ insert into gstest3 values
(2, 2, 2, 2)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/gstest3, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/gstest3], Append, `spark_catalog`.`default`.`gstest3`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/gstest3), [a, b, c, d]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as int) AS c#x, cast(col4#x as int) AS d#x]
++- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS d#x]
+- LocalRelation [col1#x, col2#x, col3#x, col4#x]
@@ -68,7 +68,7 @@ values (1,1,tinyint('0'),1), (2,2,tinyint('1'),1),
(7,64,tinyint('2'),1), (8,128,tinyint('3'),1)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/gstest4, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/gstest4], Append, `spark_catalog`.`default`.`gstest4`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/gstest4), [id, v, unhashable_col, unsortable_col]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as int) AS v#x, cast(col3#x as tinyint) AS unhashable_col#x, cast(col4#x as int) AS unsortable_col#x]
++- Project [col1#x AS id#x, col2#x AS v#x, col3#x AS unhashable_col#x, col4#x AS unsortable_col#x]
+- LocalRelation [col1#x, col2#x, col3#x, col4#x]
@@ -401,13 +401,12 @@ group by grouping sets(ten) having grouping(ten) >= 0
order by 2,1
-- !query analysis
Sort [grouping(ten)#x ASC NULLS FIRST, ten#x ASC NULLS FIRST], true
-+- Project [ten#x, grouping(ten)#x]
- +- Filter (cast(cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) as int) >= 0)
- +- Aggregate [ten#x, spark_grouping_id#xL], [ten#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(ten)#x, spark_grouping_id#xL]
- +- Expand [[unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, 0]], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, spark_grouping_id#xL]
- +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x AS ten#x]
- +- SubqueryAlias spark_catalog.default.onek
- +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet
++- Filter (cast(grouping(ten)#x as int) >= 0)
+ +- Aggregate [ten#x, spark_grouping_id#xL], [ten#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(ten)#x]
+ +- Expand [[unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, 0]], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, spark_grouping_id#xL]
+ +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x AS ten#x]
+ +- SubqueryAlias spark_catalog.default.onek
+ +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet
-- !query
@@ -416,13 +415,12 @@ group by grouping sets(ten, four) having grouping(ten) > 0
order by 2,1
-- !query analysis
Sort [grouping(ten)#x ASC NULLS FIRST, ten#x ASC NULLS FIRST], true
-+- Project [ten#x, grouping(ten)#x]
- +- Filter (cast(cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) as int) > 0)
- +- Aggregate [ten#x, four#x, spark_grouping_id#xL], [ten#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(ten)#x, spark_grouping_id#xL]
- +- Expand [[unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, null, 1], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, null, four#x, 2]], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, four#x, spark_grouping_id#xL]
- +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x AS ten#x, four#x AS four#x]
- +- SubqueryAlias spark_catalog.default.onek
- +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet
++- Filter (cast(grouping(ten)#x as int) > 0)
+ +- Aggregate [ten#x, four#x, spark_grouping_id#xL], [ten#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(ten)#x]
+ +- Expand [[unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, null, 1], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, null, four#x, 2]], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, four#x, spark_grouping_id#xL]
+ +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x AS ten#x, four#x AS four#x]
+ +- SubqueryAlias spark_catalog.default.onek
+ +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet
-- !query
@@ -431,13 +429,12 @@ group by rollup(ten) having grouping(ten) > 0
order by 2,1
-- !query analysis
Sort [grouping(ten)#x ASC NULLS FIRST, ten#x ASC NULLS FIRST], true
-+- Project [ten#x, grouping(ten)#x]
- +- Filter (cast(cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) as int) > 0)
- +- Aggregate [ten#x, spark_grouping_id#xL], [ten#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(ten)#x, spark_grouping_id#xL]
- +- Expand [[unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, 0], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, null, 1]], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, spark_grouping_id#xL]
- +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x AS ten#x]
- +- SubqueryAlias spark_catalog.default.onek
- +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet
++- Filter (cast(grouping(ten)#x as int) > 0)
+ +- Aggregate [ten#x, spark_grouping_id#xL], [ten#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(ten)#x]
+ +- Expand [[unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, 0], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, null, 1]], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, spark_grouping_id#xL]
+ +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x AS ten#x]
+ +- SubqueryAlias spark_catalog.default.onek
+ +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet
-- !query
@@ -446,13 +443,12 @@ group by cube(ten) having grouping(ten) > 0
order by 2,1
-- !query analysis
Sort [grouping(ten)#x ASC NULLS FIRST, ten#x ASC NULLS FIRST], true
-+- Project [ten#x, grouping(ten)#x]
- +- Filter (cast(cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) as int) > 0)
- +- Aggregate [ten#x, spark_grouping_id#xL], [ten#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(ten)#x, spark_grouping_id#xL]
- +- Expand [[unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, 0], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, null, 1]], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, spark_grouping_id#xL]
- +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x AS ten#x]
- +- SubqueryAlias spark_catalog.default.onek
- +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet
++- Filter (cast(grouping(ten)#x as int) > 0)
+ +- Aggregate [ten#x, spark_grouping_id#xL], [ten#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(ten)#x]
+ +- Expand [[unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, 0], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, null, 1]], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, spark_grouping_id#xL]
+ +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x AS ten#x]
+ +- SubqueryAlias spark_catalog.default.onek
+ +- Relation spark_catalog.default.onek[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet
-- !query
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int2.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int2.sql.out
index 9dda3c0dc42d4..6720132a09acc 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int2.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int2.sql.out
@@ -9,7 +9,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`INT2_TBL`, false
INSERT INTO INT2_TBL VALUES (smallint(trim('0 ')))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int2_tbl], Append, `spark_catalog`.`default`.`int2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int2_tbl), [f1]
-+- Project [cast(col1#x as smallint) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -17,7 +17,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO INT2_TBL VALUES (smallint(trim(' 1234 ')))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int2_tbl], Append, `spark_catalog`.`default`.`int2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int2_tbl), [f1]
-+- Project [cast(col1#x as smallint) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -25,7 +25,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO INT2_TBL VALUES (smallint(trim(' -1234')))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int2_tbl], Append, `spark_catalog`.`default`.`int2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int2_tbl), [f1]
-+- Project [cast(col1#x as smallint) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -33,7 +33,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO INT2_TBL VALUES (smallint('32767'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int2_tbl], Append, `spark_catalog`.`default`.`int2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int2_tbl), [f1]
-+- Project [cast(col1#x as smallint) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -41,7 +41,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO INT2_TBL VALUES (smallint('-32767'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int2_tbl], Append, `spark_catalog`.`default`.`int2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int2_tbl), [f1]
-+- Project [cast(col1#x as smallint) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int4.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int4.sql.out
index d261b59a4c5e2..e19adeb5f0bf7 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int4.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int4.sql.out
@@ -9,7 +9,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`INT4_TBL`, false
INSERT INTO INT4_TBL VALUES (int(trim(' 0 ')))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int4_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int4_tbl], Append, `spark_catalog`.`default`.`int4_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int4_tbl), [f1]
-+- Project [cast(col1#x as int) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -17,7 +17,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO INT4_TBL VALUES (int(trim('123456 ')))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int4_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int4_tbl], Append, `spark_catalog`.`default`.`int4_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int4_tbl), [f1]
-+- Project [cast(col1#x as int) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -25,7 +25,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO INT4_TBL VALUES (int(trim(' -123456')))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int4_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int4_tbl], Append, `spark_catalog`.`default`.`int4_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int4_tbl), [f1]
-+- Project [cast(col1#x as int) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -33,7 +33,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO INT4_TBL VALUES (int('2147483647'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int4_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int4_tbl], Append, `spark_catalog`.`default`.`int4_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int4_tbl), [f1]
-+- Project [cast(col1#x as int) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -41,7 +41,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO INT4_TBL VALUES (int('-2147483647'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int4_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int4_tbl], Append, `spark_catalog`.`default`.`int4_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int4_tbl), [f1]
-+- Project [cast(col1#x as int) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int8.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int8.sql.out
index 72972469fa6ef..9546f1dae5e31 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int8.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int8.sql.out
@@ -9,7 +9,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`INT8_TBL`, false
INSERT INTO INT8_TBL VALUES(bigint(trim(' 123 ')),bigint(trim(' 456')))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int8_tbl], Append, `spark_catalog`.`default`.`int8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int8_tbl), [q1, q2]
-+- Project [cast(col1#xL as bigint) AS q1#xL, cast(col2#xL as bigint) AS q2#xL]
++- Project [col1#xL AS q1#xL, col2#xL AS q2#xL]
+- LocalRelation [col1#xL, col2#xL]
@@ -17,7 +17,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO INT8_TBL VALUES(bigint(trim('123 ')),bigint('4567890123456789'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int8_tbl], Append, `spark_catalog`.`default`.`int8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int8_tbl), [q1, q2]
-+- Project [cast(col1#xL as bigint) AS q1#xL, cast(col2#xL as bigint) AS q2#xL]
++- Project [col1#xL AS q1#xL, col2#xL AS q2#xL]
+- LocalRelation [col1#xL, col2#xL]
@@ -25,7 +25,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO INT8_TBL VALUES(bigint('4567890123456789'),bigint('123'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int8_tbl], Append, `spark_catalog`.`default`.`int8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int8_tbl), [q1, q2]
-+- Project [cast(col1#xL as bigint) AS q1#xL, cast(col2#xL as bigint) AS q2#xL]
++- Project [col1#xL AS q1#xL, col2#xL AS q2#xL]
+- LocalRelation [col1#xL, col2#xL]
@@ -33,7 +33,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO INT8_TBL VALUES(+4567890123456789,bigint('4567890123456789'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int8_tbl], Append, `spark_catalog`.`default`.`int8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int8_tbl), [q1, q2]
-+- Project [cast(col1#xL as bigint) AS q1#xL, cast(col2#xL as bigint) AS q2#xL]
++- Project [col1#xL AS q1#xL, col2#xL AS q2#xL]
+- LocalRelation [col1#xL, col2#xL]
@@ -41,7 +41,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO INT8_TBL VALUES(bigint('+4567890123456789'),bigint('-4567890123456789'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/int8_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/int8_tbl], Append, `spark_catalog`.`default`.`int8_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/int8_tbl), [q1, q2]
-+- Project [cast(col1#xL as bigint) AS q1#xL, cast(col2#xL as bigint) AS q2#xL]
++- Project [col1#xL AS q1#xL, col2#xL AS q2#xL]
+- LocalRelation [col1#xL, col2#xL]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/join.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/join.sql.out
index 144fb69812ee6..439094c112863 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/join.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/join.sql.out
@@ -116,7 +116,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`J2_TBL`, false
INSERT INTO J1_TBL VALUES (1, 4, 'one')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x]
++- Project [col1#x AS i#x, col2#x AS j#x, col3#x AS t#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -124,7 +124,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO J1_TBL VALUES (2, 3, 'two')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x]
++- Project [col1#x AS i#x, col2#x AS j#x, col3#x AS t#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -132,7 +132,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO J1_TBL VALUES (3, 2, 'three')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x]
++- Project [col1#x AS i#x, col2#x AS j#x, col3#x AS t#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -140,7 +140,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO J1_TBL VALUES (4, 1, 'four')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x]
++- Project [col1#x AS i#x, col2#x AS j#x, col3#x AS t#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -148,7 +148,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO J1_TBL VALUES (5, 0, 'five')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x]
++- Project [col1#x AS i#x, col2#x AS j#x, col3#x AS t#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -156,7 +156,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO J1_TBL VALUES (6, 6, 'six')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x]
++- Project [col1#x AS i#x, col2#x AS j#x, col3#x AS t#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -164,7 +164,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO J1_TBL VALUES (7, 7, 'seven')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x]
++- Project [col1#x AS i#x, col2#x AS j#x, col3#x AS t#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -172,7 +172,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO J1_TBL VALUES (8, 8, 'eight')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x]
++- Project [col1#x AS i#x, col2#x AS j#x, col3#x AS t#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -180,7 +180,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO J1_TBL VALUES (0, NULL, 'zero')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x]
++- Project [col1#x AS i#x, cast(col2#x as int) AS j#x, col3#x AS t#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -188,7 +188,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO J1_TBL VALUES (NULL, NULL, 'null')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x]
++- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, col3#x AS t#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -196,7 +196,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO J1_TBL VALUES (NULL, 0, 'zero')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1_tbl], Append, `spark_catalog`.`default`.`j1_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1_tbl), [i, j, t]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS j#x, cast(col3#x as string) AS t#x]
++- Project [cast(col1#x as int) AS i#x, col2#x AS j#x, col3#x AS t#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -204,7 +204,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO J2_TBL VALUES (1, -1)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2_tbl], Append, `spark_catalog`.`default`.`j2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2_tbl), [i, k]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS k#x]
++- Project [col1#x AS i#x, col2#x AS k#x]
+- LocalRelation [col1#x, col2#x]
@@ -212,7 +212,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO J2_TBL VALUES (2, 2)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2_tbl], Append, `spark_catalog`.`default`.`j2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2_tbl), [i, k]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS k#x]
++- Project [col1#x AS i#x, col2#x AS k#x]
+- LocalRelation [col1#x, col2#x]
@@ -220,7 +220,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO J2_TBL VALUES (3, -3)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2_tbl], Append, `spark_catalog`.`default`.`j2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2_tbl), [i, k]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS k#x]
++- Project [col1#x AS i#x, col2#x AS k#x]
+- LocalRelation [col1#x, col2#x]
@@ -228,7 +228,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO J2_TBL VALUES (2, 4)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2_tbl], Append, `spark_catalog`.`default`.`j2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2_tbl), [i, k]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS k#x]
++- Project [col1#x AS i#x, col2#x AS k#x]
+- LocalRelation [col1#x, col2#x]
@@ -236,7 +236,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO J2_TBL VALUES (5, -5)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2_tbl], Append, `spark_catalog`.`default`.`j2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2_tbl), [i, k]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS k#x]
++- Project [col1#x AS i#x, col2#x AS k#x]
+- LocalRelation [col1#x, col2#x]
@@ -244,7 +244,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO J2_TBL VALUES (5, -5)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2_tbl], Append, `spark_catalog`.`default`.`j2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2_tbl), [i, k]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS k#x]
++- Project [col1#x AS i#x, col2#x AS k#x]
+- LocalRelation [col1#x, col2#x]
@@ -252,7 +252,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO J2_TBL VALUES (0, NULL)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2_tbl], Append, `spark_catalog`.`default`.`j2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2_tbl), [i, k]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS k#x]
++- Project [col1#x AS i#x, cast(col2#x as int) AS k#x]
+- LocalRelation [col1#x, col2#x]
@@ -268,7 +268,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO J2_TBL VALUES (NULL, 0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2_tbl], Append, `spark_catalog`.`default`.`j2_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2_tbl), [i, k]
-+- Project [cast(col1#x as int) AS i#x, cast(col2#x as int) AS k#x]
++- Project [cast(col1#x as int) AS i#x, col2#x AS k#x]
+- LocalRelation [col1#x, col2#x]
@@ -674,7 +674,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`t3`, false
INSERT INTO t1 VALUES ( 'bb', 11 )
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [name, n]
-+- Project [cast(col1#x as string) AS name#x, cast(col2#x as int) AS n#x]
++- Project [col1#x AS name#x, col2#x AS n#x]
+- LocalRelation [col1#x, col2#x]
@@ -682,7 +682,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO t2 VALUES ( 'bb', 12 )
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t2], Append, `spark_catalog`.`default`.`t2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t2), [name, n]
-+- Project [cast(col1#x as string) AS name#x, cast(col2#x as int) AS n#x]
++- Project [col1#x AS name#x, col2#x AS n#x]
+- LocalRelation [col1#x, col2#x]
@@ -690,7 +690,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO t2 VALUES ( 'cc', 22 )
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t2], Append, `spark_catalog`.`default`.`t2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t2), [name, n]
-+- Project [cast(col1#x as string) AS name#x, cast(col2#x as int) AS n#x]
++- Project [col1#x AS name#x, col2#x AS n#x]
+- LocalRelation [col1#x, col2#x]
@@ -698,7 +698,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO t2 VALUES ( 'ee', 42 )
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t2], Append, `spark_catalog`.`default`.`t2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t2), [name, n]
-+- Project [cast(col1#x as string) AS name#x, cast(col2#x as int) AS n#x]
++- Project [col1#x AS name#x, col2#x AS n#x]
+- LocalRelation [col1#x, col2#x]
@@ -706,7 +706,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO t3 VALUES ( 'bb', 13 )
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t3, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t3], Append, `spark_catalog`.`default`.`t3`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t3), [name, n]
-+- Project [cast(col1#x as string) AS name#x, cast(col2#x as int) AS n#x]
++- Project [col1#x AS name#x, col2#x AS n#x]
+- LocalRelation [col1#x, col2#x]
@@ -714,7 +714,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO t3 VALUES ( 'cc', 23 )
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t3, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t3], Append, `spark_catalog`.`default`.`t3`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t3), [name, n]
-+- Project [cast(col1#x as string) AS name#x, cast(col2#x as int) AS n#x]
++- Project [col1#x AS name#x, col2#x AS n#x]
+- LocalRelation [col1#x, col2#x]
@@ -722,7 +722,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO t3 VALUES ( 'dd', 33 )
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t3, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t3], Append, `spark_catalog`.`default`.`t3`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t3), [name, n]
-+- Project [cast(col1#x as string) AS name#x, cast(col2#x as int) AS n#x]
++- Project [col1#x AS name#x, col2#x AS n#x]
+- LocalRelation [col1#x, col2#x]
@@ -1326,7 +1326,7 @@ Aggregate [count(1) AS count(1)#xL]
+- Filter unique1#x IN (list#x [])
: +- Project [unique1#x]
: +- Filter (unique2#x = 42)
- : +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, ... 6 more fields]
+ : +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x]
: +- Join Inner, (unique1#x = unique1#x)
: :- SubqueryAlias b
: : +- SubqueryAlias spark_catalog.default.tenk1
@@ -3844,7 +3844,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`j2`, false
INSERT INTO j1 values(1,1),(1,2)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j1], Append, `spark_catalog`.`default`.`j1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j1), [id1, id2]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x]
+- LocalRelation [col1#x, col2#x]
@@ -3852,7 +3852,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO j2 values(1,1)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2], Append, `spark_catalog`.`default`.`j2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2), [id1, id2]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x]
+- LocalRelation [col1#x, col2#x]
@@ -3860,7 +3860,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO j2 values(1,2)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/j2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/j2], Append, `spark_catalog`.`default`.`j2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/j2), [id1, id2]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x]
+- LocalRelation [col1#x, col2#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/numeric.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/numeric.sql.out
index 6c2ae23291755..d5e8edbf02060 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/numeric.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/numeric.sql.out
@@ -63,7 +63,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`num_result`, false
INSERT INTO num_exp_add VALUES (0,0,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -71,7 +71,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (0,0,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -79,7 +79,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (0,0,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -87,7 +87,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (0,0,double('NaN'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -95,7 +95,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (0,1,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -103,7 +103,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (0,1,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -111,7 +111,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (0,1,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -119,7 +119,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (0,1,double('NaN'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -127,7 +127,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (0,2,-34338492.215397047)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -135,7 +135,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (0,2,34338492.215397047)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -143,7 +143,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (0,2,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -151,7 +151,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (0,2,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -159,7 +159,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (0,3,4.31)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -167,7 +167,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (0,3,-4.31)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -175,7 +175,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (0,3,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -183,7 +183,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (0,3,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -191,7 +191,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (0,4,7799461.4119)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -199,7 +199,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (0,4,-7799461.4119)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -207,7 +207,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (0,4,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -215,7 +215,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (0,4,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -223,7 +223,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (0,5,16397.038491)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -231,7 +231,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (0,5,-16397.038491)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -239,7 +239,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (0,5,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -247,7 +247,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (0,5,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -255,7 +255,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (0,6,93901.57763026)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -263,7 +263,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (0,6,-93901.57763026)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -271,7 +271,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (0,6,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -279,7 +279,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (0,6,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -287,7 +287,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (0,7,-83028485)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -295,7 +295,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (0,7,83028485)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -303,7 +303,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (0,7,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -311,7 +311,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (0,7,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -319,7 +319,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (0,8,74881)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -327,7 +327,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (0,8,-74881)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -335,7 +335,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (0,8,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -343,7 +343,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (0,8,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -351,7 +351,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (0,9,-24926804.045047420)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -359,7 +359,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (0,9,24926804.045047420)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -367,7 +367,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (0,9,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -375,7 +375,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (0,9,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -383,7 +383,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (1,0,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -391,7 +391,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (1,0,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -399,7 +399,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (1,0,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -407,7 +407,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (1,0,double('NaN'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -415,7 +415,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (1,1,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -423,7 +423,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (1,1,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -431,7 +431,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (1,1,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -439,7 +439,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (1,1,double('NaN'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -447,7 +447,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (1,2,-34338492.215397047)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -455,7 +455,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (1,2,34338492.215397047)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -463,7 +463,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (1,2,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -471,7 +471,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (1,2,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -479,7 +479,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (1,3,4.31)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -487,7 +487,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (1,3,-4.31)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -495,7 +495,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (1,3,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -503,7 +503,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (1,3,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -511,7 +511,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (1,4,7799461.4119)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -519,7 +519,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (1,4,-7799461.4119)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -527,7 +527,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (1,4,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -535,7 +535,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (1,4,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -543,7 +543,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (1,5,16397.038491)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -551,7 +551,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (1,5,-16397.038491)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -559,7 +559,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (1,5,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -567,7 +567,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (1,5,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -575,7 +575,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (1,6,93901.57763026)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -583,7 +583,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (1,6,-93901.57763026)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -591,7 +591,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (1,6,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -599,7 +599,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (1,6,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -607,7 +607,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (1,7,-83028485)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -615,7 +615,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (1,7,83028485)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -623,7 +623,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (1,7,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -631,7 +631,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (1,7,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -639,7 +639,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (1,8,74881)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -647,7 +647,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (1,8,-74881)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -655,7 +655,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (1,8,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -663,7 +663,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (1,8,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -671,7 +671,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (1,9,-24926804.045047420)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -679,7 +679,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (1,9,24926804.045047420)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -687,7 +687,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (1,9,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -695,7 +695,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (1,9,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -703,7 +703,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (2,0,-34338492.215397047)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -711,7 +711,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (2,0,-34338492.215397047)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -719,7 +719,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (2,0,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -727,7 +727,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (2,0,double('NaN'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -735,7 +735,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (2,1,-34338492.215397047)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -743,7 +743,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (2,1,-34338492.215397047)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -751,7 +751,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (2,1,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -759,7 +759,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (2,1,double('NaN'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -767,7 +767,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (2,2,-68676984.430794094)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -775,7 +775,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (2,2,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -783,7 +783,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (2,2,1179132047626883.596862135856320209)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -791,7 +791,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (2,2,1.00000000000000000000)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -799,7 +799,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (2,3,-34338487.905397047)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -807,7 +807,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (2,3,-34338496.525397047)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -815,7 +815,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (2,3,-147998901.44836127257)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -823,7 +823,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (2,3,-7967167.56737750510440835266)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -831,7 +831,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (2,4,-26539030.803497047)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -839,7 +839,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (2,4,-42137953.627297047)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -847,7 +847,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (2,4,-267821744976817.8111137106593)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -855,7 +855,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (2,4,-4.40267480046830116685)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -863,7 +863,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (2,5,-34322095.176906047)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -871,7 +871,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (2,5,-34354889.253888047)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -879,7 +879,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (2,5,-563049578578.769242506736077)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -887,7 +887,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (2,5,-2094.18866914563535496429)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -895,7 +895,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (2,6,-34244590.637766787)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -903,7 +903,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (2,6,-34432393.793027307)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -911,7 +911,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (2,6,-3224438592470.18449811926184222)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -919,7 +919,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (2,6,-365.68599891479766440940)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -927,7 +927,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (2,7,-117366977.215397047)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -935,7 +935,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (2,7,48689992.784602953)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -943,7 +943,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (2,7,2851072985828710.485883795)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -951,7 +951,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (2,7,.41357483778485235518)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -959,7 +959,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (2,8,-34263611.215397047)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -967,7 +967,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (2,8,-34413373.215397047)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -975,7 +975,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (2,8,-2571300635581.146276407)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -983,7 +983,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (2,8,-458.57416721727870888476)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -991,7 +991,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (2,9,-59265296.260444467)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -999,7 +999,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (2,9,-9411688.170349627)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1007,7 +1007,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (2,9,855948866655588.453741509242968740)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1015,7 +1015,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (2,9,1.37757299946438931811)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1023,7 +1023,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (3,0,4.31)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1031,7 +1031,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (3,0,4.31)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1039,7 +1039,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (3,0,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1047,7 +1047,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (3,0,double('NaN'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1055,7 +1055,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (3,1,4.31)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1063,7 +1063,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (3,1,4.31)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1071,7 +1071,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (3,1,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1079,7 +1079,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (3,1,double('NaN'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1087,7 +1087,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (3,2,-34338487.905397047)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1095,7 +1095,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (3,2,34338496.525397047)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1103,7 +1103,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (3,2,-147998901.44836127257)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1111,7 +1111,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (3,2,-.00000012551512084352)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1119,7 +1119,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (3,3,8.62)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1127,7 +1127,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (3,3,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1135,7 +1135,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (3,3,18.5761)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1143,7 +1143,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (3,3,1.00000000000000000000)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1151,7 +1151,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (3,4,7799465.7219)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1159,7 +1159,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (3,4,-7799457.1019)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1167,7 +1167,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (3,4,33615678.685289)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1175,7 +1175,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (3,4,.00000055260225961552)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1183,7 +1183,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (3,5,16401.348491)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1191,7 +1191,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (3,5,-16392.728491)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1199,7 +1199,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (3,5,70671.23589621)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1207,7 +1207,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (3,5,.00026285234387695504)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1215,7 +1215,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (3,6,93905.88763026)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1223,7 +1223,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (3,6,-93897.26763026)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1231,7 +1231,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (3,6,404715.7995864206)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1239,7 +1239,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (3,6,.00004589912234457595)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1247,7 +1247,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (3,7,-83028480.69)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1255,7 +1255,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (3,7,83028489.31)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1263,7 +1263,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (3,7,-357852770.35)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1271,7 +1271,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (3,7,-.00000005190989574240)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1279,7 +1279,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (3,8,74885.31)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1287,7 +1287,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (3,8,-74876.69)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1295,7 +1295,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (3,8,322737.11)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1303,7 +1303,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (3,8,.00005755799201399553)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1311,7 +1311,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (3,9,-24926799.735047420)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1319,7 +1319,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (3,9,24926808.355047420)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1327,7 +1327,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (3,9,-107434525.43415438020)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1335,7 +1335,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (3,9,-.00000017290624149854)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1343,7 +1343,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (4,0,7799461.4119)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1351,7 +1351,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (4,0,7799461.4119)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1359,7 +1359,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (4,0,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1367,7 +1367,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (4,0,double('NaN'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1375,7 +1375,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (4,1,7799461.4119)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1383,7 +1383,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (4,1,7799461.4119)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1391,7 +1391,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (4,1,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1399,7 +1399,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (4,1,double('NaN'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1407,7 +1407,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (4,2,-26539030.803497047)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1415,7 +1415,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (4,2,42137953.627297047)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1423,7 +1423,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (4,2,-267821744976817.8111137106593)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1431,7 +1431,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (4,2,-.22713465002993920385)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1439,7 +1439,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (4,3,7799465.7219)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1447,7 +1447,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (4,3,7799457.1019)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1455,7 +1455,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (4,3,33615678.685289)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1463,7 +1463,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (4,3,1809619.81714617169373549883)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1471,7 +1471,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (4,4,15598922.8238)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1479,7 +1479,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (4,4,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1487,7 +1487,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (4,4,60831598315717.14146161)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1495,7 +1495,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (4,4,1.00000000000000000000)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1503,7 +1503,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (4,5,7815858.450391)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1511,7 +1511,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (4,5,7783064.373409)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1519,7 +1519,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (4,5,127888068979.9935054429)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1527,7 +1527,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (4,5,475.66281046305802686061)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1535,7 +1535,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (4,6,7893362.98953026)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1543,7 +1543,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (4,6,7705559.83426974)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1551,7 +1551,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (4,6,732381731243.745115764094)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1559,7 +1559,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (4,6,83.05996138436129499606)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1567,7 +1567,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (4,7,-75229023.5881)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1575,7 +1575,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (4,7,90827946.4119)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1583,7 +1583,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (4,7,-647577464846017.9715)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1591,7 +1591,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (4,7,-.09393717604145131637)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1599,7 +1599,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (4,8,7874342.4119)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1607,7 +1607,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (4,8,7724580.4119)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1615,7 +1615,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (4,8,584031469984.4839)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1623,7 +1623,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (4,8,104.15808298366741897143)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1631,7 +1631,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (4,9,-17127342.633147420)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1639,7 +1639,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (4,9,32726265.456947420)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1647,7 +1647,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (4,9,-194415646271340.1815956522980)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1655,7 +1655,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (4,9,-.31289456112403769409)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1663,7 +1663,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (5,0,16397.038491)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1671,7 +1671,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (5,0,16397.038491)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1679,7 +1679,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (5,0,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1687,7 +1687,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (5,0,double('NaN'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1695,7 +1695,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (5,1,16397.038491)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1703,7 +1703,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (5,1,16397.038491)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1711,7 +1711,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (5,1,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1719,7 +1719,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (5,1,double('NaN'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1727,7 +1727,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (5,2,-34322095.176906047)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1735,7 +1735,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (5,2,34354889.253888047)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1743,7 +1743,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (5,2,-563049578578.769242506736077)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1751,7 +1751,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (5,2,-.00047751189505192446)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1759,7 +1759,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (5,3,16401.348491)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1767,7 +1767,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (5,3,16392.728491)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1775,7 +1775,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (5,3,70671.23589621)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1783,7 +1783,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (5,3,3804.41728329466357308584)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1791,7 +1791,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (5,4,7815858.450391)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1799,7 +1799,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (5,4,-7783064.373409)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1807,7 +1807,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (5,4,127888068979.9935054429)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1815,7 +1815,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (5,4,.00210232958726897192)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1823,7 +1823,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (5,5,32794.076982)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1831,7 +1831,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (5,5,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1839,7 +1839,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (5,5,268862871.275335557081)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1847,7 +1847,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (5,5,1.00000000000000000000)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1855,7 +1855,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (5,6,110298.61612126)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1863,7 +1863,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (5,6,-77504.53913926)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1871,7 +1871,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (5,6,1539707782.76899778633766)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1879,7 +1879,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (5,6,.17461941433576102689)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1887,7 +1887,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (5,7,-83012087.961509)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1895,7 +1895,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (5,7,83044882.038491)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1903,7 +1903,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (5,7,-1361421264394.416135)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1911,7 +1911,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (5,7,-.00019748690453643710)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1919,7 +1919,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (5,8,91278.038491)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1927,7 +1927,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (5,8,-58483.961509)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1935,7 +1935,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (5,8,1227826639.244571)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1943,7 +1943,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (5,8,.21897461960978085228)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1951,7 +1951,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (5,9,-24910407.006556420)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1959,7 +1959,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (5,9,24943201.083538420)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1967,7 +1967,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (5,9,-408725765384.257043660243220)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1975,7 +1975,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (5,9,-.00065780749354660427)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1983,7 +1983,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (6,0,93901.57763026)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1991,7 +1991,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (6,0,93901.57763026)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1999,7 +1999,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (6,0,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2007,7 +2007,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (6,0,double('NaN'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2015,7 +2015,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (6,1,93901.57763026)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2023,7 +2023,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (6,1,93901.57763026)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2031,7 +2031,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (6,1,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2039,7 +2039,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (6,1,double('NaN'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2047,7 +2047,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (6,2,-34244590.637766787)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2055,7 +2055,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (6,2,34432393.793027307)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2063,7 +2063,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (6,2,-3224438592470.18449811926184222)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2071,7 +2071,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (6,2,-.00273458651128995823)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2079,7 +2079,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (6,3,93905.88763026)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2087,7 +2087,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (6,3,93897.26763026)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2095,7 +2095,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (6,3,404715.7995864206)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2103,7 +2103,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (6,3,21786.90896293735498839907)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2111,7 +2111,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (6,4,7893362.98953026)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2119,7 +2119,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (6,4,-7705559.83426974)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2127,7 +2127,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (6,4,732381731243.745115764094)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2135,7 +2135,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (6,4,.01203949512295682469)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2143,7 +2143,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (6,5,110298.61612126)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2151,7 +2151,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (6,5,77504.53913926)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2159,7 +2159,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (6,5,1539707782.76899778633766)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2167,7 +2167,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (6,5,5.72674008674192359679)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2175,7 +2175,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (6,6,187803.15526052)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2183,7 +2183,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (6,6,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2191,7 +2191,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (6,6,8817506281.4517452372676676)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2199,7 +2199,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (6,6,1.00000000000000000000)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2207,7 +2207,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (6,7,-82934583.42236974)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2215,7 +2215,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (6,7,83122386.57763026)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2223,7 +2223,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (6,7,-7796505729750.37795610)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2231,7 +2231,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (6,7,-.00113095617281538980)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2239,7 +2239,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (6,8,168782.57763026)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2247,7 +2247,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (6,8,19020.57763026)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2255,7 +2255,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (6,8,7031444034.53149906)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2263,7 +2263,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (6,8,1.25401073209839612184)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2271,7 +2271,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (6,9,-24832902.467417160)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2279,7 +2279,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (6,9,25020705.622677680)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2287,7 +2287,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (6,9,-2340666225110.29929521292692920)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2295,7 +2295,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (6,9,-.00376709254265256789)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2303,7 +2303,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (7,0,-83028485)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2311,7 +2311,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (7,0,-83028485)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2319,7 +2319,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (7,0,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2327,7 +2327,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (7,0,double('NaN'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2335,7 +2335,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (7,1,-83028485)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2343,7 +2343,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (7,1,-83028485)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2351,7 +2351,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (7,1,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2359,7 +2359,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (7,1,double('NaN'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2367,7 +2367,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (7,2,-117366977.215397047)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2375,7 +2375,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (7,2,-48689992.784602953)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2383,7 +2383,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (7,2,2851072985828710.485883795)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2391,7 +2391,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (7,2,2.41794207151503385700)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2399,7 +2399,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (7,3,-83028480.69)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2407,7 +2407,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (7,3,-83028489.31)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2415,7 +2415,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (7,3,-357852770.35)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2423,7 +2423,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (7,3,-19264149.65197215777262180974)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2431,7 +2431,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (7,4,-75229023.5881)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2439,7 +2439,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (7,4,-90827946.4119)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2447,7 +2447,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (7,4,-647577464846017.9715)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2455,7 +2455,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (7,4,-10.64541262725136247686)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2463,7 +2463,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (7,5,-83012087.961509)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2471,7 +2471,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (7,5,-83044882.038491)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2479,7 +2479,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (7,5,-1361421264394.416135)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2487,7 +2487,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (7,5,-5063.62688881730941836574)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2495,7 +2495,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (7,6,-82934583.42236974)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2503,7 +2503,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (7,6,-83122386.57763026)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2511,7 +2511,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (7,6,-7796505729750.37795610)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2519,7 +2519,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (7,6,-884.20756174009028770294)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2527,7 +2527,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (7,7,-166056970)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2535,7 +2535,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (7,7,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2543,7 +2543,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (7,7,6893729321395225)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#xL as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#xL as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#xL]
@@ -2551,7 +2551,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (7,7,1.00000000000000000000)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2559,7 +2559,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (7,8,-82953604)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2567,7 +2567,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (7,8,-83103366)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2575,7 +2575,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (7,8,-6217255985285)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#xL as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#xL as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#xL]
@@ -2583,7 +2583,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (7,8,-1108.80577182462841041118)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2591,7 +2591,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (7,9,-107955289.045047420)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2599,7 +2599,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (7,9,-58101680.954952580)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2607,7 +2607,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (7,9,2069634775752159.035758700)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2615,7 +2615,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (7,9,3.33089171198810413382)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2623,7 +2623,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (8,0,74881)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2631,7 +2631,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (8,0,74881)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2639,7 +2639,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (8,0,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2647,7 +2647,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (8,0,double('NaN'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2655,7 +2655,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (8,1,74881)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2663,7 +2663,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (8,1,74881)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2671,7 +2671,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (8,1,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2679,7 +2679,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (8,1,double('NaN'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2687,7 +2687,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (8,2,-34263611.215397047)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2695,7 +2695,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (8,2,34413373.215397047)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2703,7 +2703,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (8,2,-2571300635581.146276407)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2711,7 +2711,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (8,2,-.00218067233500788615)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2719,7 +2719,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (8,3,74885.31)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2727,7 +2727,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (8,3,74876.69)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2735,7 +2735,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (8,3,322737.11)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2743,7 +2743,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (8,3,17373.78190255220417633410)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2751,7 +2751,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (8,4,7874342.4119)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2759,7 +2759,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (8,4,-7724580.4119)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2767,7 +2767,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (8,4,584031469984.4839)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2775,7 +2775,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (8,4,.00960079113741758956)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2783,7 +2783,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (8,5,91278.038491)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2791,7 +2791,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (8,5,58483.961509)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2799,7 +2799,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (8,5,1227826639.244571)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2807,7 +2807,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (8,5,4.56673929509287019456)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2815,7 +2815,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (8,6,168782.57763026)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2823,7 +2823,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (8,6,-19020.57763026)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2831,7 +2831,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (8,6,7031444034.53149906)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2839,7 +2839,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (8,6,.79744134113322314424)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2847,7 +2847,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (8,7,-82953604)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2855,7 +2855,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (8,7,83103366)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2863,7 +2863,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (8,7,-6217255985285)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#xL as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#xL as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#xL]
@@ -2871,7 +2871,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (8,7,-.00090187120721280172)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2879,7 +2879,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (8,8,149762)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2887,7 +2887,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (8,8,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2895,7 +2895,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (8,8,5607164161)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#xL as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#xL as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#xL]
@@ -2903,7 +2903,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (8,8,1.00000000000000000000)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2911,7 +2911,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (8,9,-24851923.045047420)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2919,7 +2919,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (8,9,25001685.045047420)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2927,7 +2927,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (8,9,-1866544013697.195857020)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2935,7 +2935,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (8,9,-.00300403532938582735)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2943,7 +2943,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (9,0,-24926804.045047420)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2951,7 +2951,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (9,0,-24926804.045047420)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2959,7 +2959,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (9,0,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2967,7 +2967,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (9,0,double('NaN'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2975,7 +2975,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (9,1,-24926804.045047420)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2983,7 +2983,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (9,1,-24926804.045047420)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2991,7 +2991,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (9,1,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -2999,7 +2999,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (9,1,double('NaN'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3007,7 +3007,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (9,2,-59265296.260444467)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3015,7 +3015,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (9,2,9411688.170349627)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3023,7 +3023,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (9,2,855948866655588.453741509242968740)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3031,7 +3031,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (9,2,.72591434384152961526)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3039,7 +3039,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (9,3,-24926799.735047420)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3047,7 +3047,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (9,3,-24926808.355047420)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3055,7 +3055,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (9,3,-107434525.43415438020)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3063,7 +3063,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (9,3,-5783481.21694835730858468677)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3071,7 +3071,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (9,4,-17127342.633147420)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3079,7 +3079,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (9,4,-32726265.456947420)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3087,7 +3087,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (9,4,-194415646271340.1815956522980)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3095,7 +3095,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (9,4,-3.19596478892958416484)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3103,7 +3103,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (9,5,-24910407.006556420)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3111,7 +3111,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (9,5,-24943201.083538420)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3119,7 +3119,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (9,5,-408725765384.257043660243220)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3127,7 +3127,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (9,5,-1520.20159364322004505807)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3135,7 +3135,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (9,6,-24832902.467417160)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3143,7 +3143,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (9,6,-25020705.622677680)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3151,7 +3151,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (9,6,-2340666225110.29929521292692920)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3159,7 +3159,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (9,6,-265.45671195426965751280)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3167,7 +3167,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (9,7,-107955289.045047420)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3175,7 +3175,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (9,7,58101680.954952580)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3183,7 +3183,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (9,7,2069634775752159.035758700)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3191,7 +3191,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (9,7,.30021990699995814689)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3199,7 +3199,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (9,8,-24851923.045047420)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3207,7 +3207,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (9,8,-25001685.045047420)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3215,7 +3215,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (9,8,-1866544013697.195857020)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3223,7 +3223,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (9,8,-332.88556569820675471748)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3231,7 +3231,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_add VALUES (9,9,-49853608.090094840)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_add, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_add], Append, `spark_catalog`.`default`.`num_exp_add`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_add), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3239,7 +3239,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sub VALUES (9,9,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sub, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sub], Append, `spark_catalog`.`default`.`num_exp_sub`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sub), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3247,7 +3247,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_mul VALUES (9,9,621345559900192.420120630048656400)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_mul, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_mul], Append, `spark_catalog`.`default`.`num_exp_mul`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_mul), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3255,7 +3255,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_div VALUES (9,9,1.00000000000000000000)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_div, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_div], Append, `spark_catalog`.`default`.`num_exp_div`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_div), [id1, id2, expected]
-+- Project [cast(col1#x as int) AS id1#x, cast(col2#x as int) AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id1#x, col2#x AS id2#x, cast(col3#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -3263,7 +3263,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sqrt VALUES (0,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt], Append, `spark_catalog`.`default`.`num_exp_sqrt`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3271,7 +3271,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sqrt VALUES (1,0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt], Append, `spark_catalog`.`default`.`num_exp_sqrt`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3279,7 +3279,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sqrt VALUES (2,5859.90547836712524903505)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt], Append, `spark_catalog`.`default`.`num_exp_sqrt`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3287,7 +3287,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sqrt VALUES (3,2.07605394920266944396)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt], Append, `spark_catalog`.`default`.`num_exp_sqrt`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3295,7 +3295,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sqrt VALUES (4,2792.75158435189147418923)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt], Append, `spark_catalog`.`default`.`num_exp_sqrt`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3303,7 +3303,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sqrt VALUES (5,128.05092147657509145473)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt], Append, `spark_catalog`.`default`.`num_exp_sqrt`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3311,7 +3311,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sqrt VALUES (6,306.43364311096782703406)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt], Append, `spark_catalog`.`default`.`num_exp_sqrt`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3319,7 +3319,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sqrt VALUES (7,9111.99676251039939975230)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt], Append, `spark_catalog`.`default`.`num_exp_sqrt`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3327,7 +3327,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sqrt VALUES (8,273.64392922189960397542)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt], Append, `spark_catalog`.`default`.`num_exp_sqrt`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3335,7 +3335,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_sqrt VALUES (9,4992.67503899937593364766)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt], Append, `spark_catalog`.`default`.`num_exp_sqrt`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_sqrt), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3343,7 +3343,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_ln VALUES (0,double('NaN'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_ln], Append, `spark_catalog`.`default`.`num_exp_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_ln), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3351,7 +3351,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_ln VALUES (1,double('NaN'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_ln], Append, `spark_catalog`.`default`.`num_exp_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_ln), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3359,7 +3359,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_ln VALUES (2,17.35177750493897715514)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_ln], Append, `spark_catalog`.`default`.`num_exp_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_ln), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3367,7 +3367,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_ln VALUES (3,1.46093790411565641971)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_ln], Append, `spark_catalog`.`default`.`num_exp_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_ln), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3375,7 +3375,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_ln VALUES (4,15.86956523951936572464)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_ln], Append, `spark_catalog`.`default`.`num_exp_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_ln), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3383,7 +3383,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_ln VALUES (5,9.70485601768871834038)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_ln], Append, `spark_catalog`.`default`.`num_exp_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_ln), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3391,7 +3391,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_ln VALUES (6,11.45000246622944403127)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_ln], Append, `spark_catalog`.`default`.`num_exp_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_ln), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3399,7 +3399,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_ln VALUES (7,18.23469429965478772991)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_ln], Append, `spark_catalog`.`default`.`num_exp_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_ln), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3407,7 +3407,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_ln VALUES (8,11.22365546576315513668)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_ln], Append, `spark_catalog`.`default`.`num_exp_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_ln), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3415,7 +3415,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_ln VALUES (9,17.03145425013166006962)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_ln], Append, `spark_catalog`.`default`.`num_exp_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_ln), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3423,7 +3423,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_log10 VALUES (0,double('NaN'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_log10, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_log10], Append, `spark_catalog`.`default`.`num_exp_log10`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_log10), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3431,7 +3431,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_log10 VALUES (1,double('NaN'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_log10, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_log10], Append, `spark_catalog`.`default`.`num_exp_log10`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_log10), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3439,7 +3439,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_log10 VALUES (2,7.53578122160797276459)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_log10, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_log10], Append, `spark_catalog`.`default`.`num_exp_log10`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_log10), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3447,7 +3447,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_log10 VALUES (3,.63447727016073160075)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_log10, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_log10], Append, `spark_catalog`.`default`.`num_exp_log10`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_log10), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3455,7 +3455,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_log10 VALUES (4,6.89206461372691743345)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_log10, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_log10], Append, `spark_catalog`.`default`.`num_exp_log10`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_log10), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3463,7 +3463,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_log10 VALUES (5,4.21476541614777768626)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_log10, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_log10], Append, `spark_catalog`.`default`.`num_exp_log10`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_log10), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3471,7 +3471,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_log10 VALUES (6,4.97267288886207207671)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_log10, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_log10], Append, `spark_catalog`.`default`.`num_exp_log10`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_log10), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3479,7 +3479,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_log10 VALUES (7,7.91922711353275546914)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_log10, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_log10], Append, `spark_catalog`.`default`.`num_exp_log10`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_log10), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3487,7 +3487,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_log10 VALUES (8,4.87437163556421004138)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_log10, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_log10], Append, `spark_catalog`.`default`.`num_exp_log10`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_log10), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3495,7 +3495,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_log10 VALUES (9,7.39666659961986567059)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_log10, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_log10], Append, `spark_catalog`.`default`.`num_exp_log10`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_log10), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3503,7 +3503,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_power_10_ln VALUES (0,double('NaN'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln], Append, `spark_catalog`.`default`.`num_exp_power_10_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3511,7 +3511,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_power_10_ln VALUES (1,double('NaN'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln], Append, `spark_catalog`.`default`.`num_exp_power_10_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3519,7 +3519,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_power_10_ln VALUES (2,224790267919917955.13261618583642653184)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln], Append, `spark_catalog`.`default`.`num_exp_power_10_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3527,7 +3527,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_power_10_ln VALUES (3,28.90266599445155957393)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln], Append, `spark_catalog`.`default`.`num_exp_power_10_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3535,7 +3535,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_power_10_ln VALUES (4,7405685069594999.07733999469386277636)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln], Append, `spark_catalog`.`default`.`num_exp_power_10_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3543,7 +3543,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_power_10_ln VALUES (5,5068226527.32127265408584640098)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln], Append, `spark_catalog`.`default`.`num_exp_power_10_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3551,7 +3551,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_power_10_ln VALUES (6,281839893606.99372343357047819067)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln], Append, `spark_catalog`.`default`.`num_exp_power_10_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3573,7 +3573,7 @@ org.apache.spark.SparkArithmeticException
INSERT INTO num_exp_power_10_ln VALUES (8,167361463828.07491320069016125952)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln], Append, `spark_catalog`.`default`.`num_exp_power_10_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3581,7 +3581,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_exp_power_10_ln VALUES (9,107511333880052007.04141124673540337457)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln], Append, `spark_catalog`.`default`.`num_exp_power_10_ln`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_exp_power_10_ln), [id, expected]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS expected#x]
+- LocalRelation [col1#x, col2#x]
@@ -3589,7 +3589,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_data VALUES (0, 0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_data, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_data], Append, `spark_catalog`.`default`.`num_data`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_data), [id, val]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS val#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS val#x]
+- LocalRelation [col1#x, col2#x]
@@ -3597,7 +3597,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_data VALUES (1, 0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_data, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_data], Append, `spark_catalog`.`default`.`num_data`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_data), [id, val]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS val#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS val#x]
+- LocalRelation [col1#x, col2#x]
@@ -3605,7 +3605,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_data VALUES (2, -34338492.215397047)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_data, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_data], Append, `spark_catalog`.`default`.`num_data`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_data), [id, val]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS val#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS val#x]
+- LocalRelation [col1#x, col2#x]
@@ -3613,7 +3613,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_data VALUES (3, 4.31)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_data, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_data], Append, `spark_catalog`.`default`.`num_data`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_data), [id, val]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS val#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS val#x]
+- LocalRelation [col1#x, col2#x]
@@ -3621,7 +3621,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_data VALUES (4, 7799461.4119)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_data, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_data], Append, `spark_catalog`.`default`.`num_data`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_data), [id, val]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS val#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS val#x]
+- LocalRelation [col1#x, col2#x]
@@ -3629,7 +3629,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_data VALUES (5, 16397.038491)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_data, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_data], Append, `spark_catalog`.`default`.`num_data`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_data), [id, val]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS val#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS val#x]
+- LocalRelation [col1#x, col2#x]
@@ -3637,7 +3637,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_data VALUES (6, 93901.57763026)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_data, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_data], Append, `spark_catalog`.`default`.`num_data`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_data), [id, val]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS val#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS val#x]
+- LocalRelation [col1#x, col2#x]
@@ -3645,7 +3645,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_data VALUES (7, -83028485)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_data, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_data], Append, `spark_catalog`.`default`.`num_data`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_data), [id, val]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS val#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS val#x]
+- LocalRelation [col1#x, col2#x]
@@ -3653,7 +3653,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_data VALUES (8, 74881)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_data, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_data], Append, `spark_catalog`.`default`.`num_data`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_data), [id, val]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS val#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS val#x]
+- LocalRelation [col1#x, col2#x]
@@ -3661,7 +3661,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO num_data VALUES (9, -24926804.045047420)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_data, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_data], Append, `spark_catalog`.`default`.`num_data`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_data), [id, val]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(38,10)) AS val#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(38,10)) AS val#x]
+- LocalRelation [col1#x, col2#x]
@@ -3684,7 +3684,7 @@ INSERT INTO num_result SELECT t1.id, t2.id, t1.val + t2.val
FROM num_data t1, num_data t2
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_result, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_result], Append, `spark_catalog`.`default`.`num_result`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_result), [id1, id2, result]
-+- Project [cast(id#x as int) AS id1#x, cast(id#x as int) AS id2#x, cast((val + val)#x as decimal(38,10)) AS result#x]
++- Project [id#x AS id1#x, id#x AS id2#x, cast((val + val)#x as decimal(38,10)) AS result#x]
+- Project [id#x, id#x, (val#x + val#x) AS (val + val)#x]
+- Join Inner
:- SubqueryAlias t1
@@ -3723,7 +3723,7 @@ INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val + t2.val, 10)
FROM num_data t1, num_data t2
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_result, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_result], Append, `spark_catalog`.`default`.`num_result`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_result), [id1, id2, result]
-+- Project [cast(id#x as int) AS id1#x, cast(id#x as int) AS id2#x, cast(round((val + val), 10)#x as decimal(38,10)) AS result#x]
++- Project [id#x AS id1#x, id#x AS id2#x, cast(round((val + val), 10)#x as decimal(38,10)) AS result#x]
+- Project [id#x, id#x, round((val#x + val#x), 10) AS round((val + val), 10)#x]
+- Join Inner
:- SubqueryAlias t1
@@ -3762,7 +3762,7 @@ INSERT INTO num_result SELECT t1.id, t2.id, t1.val - t2.val
FROM num_data t1, num_data t2
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_result, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_result], Append, `spark_catalog`.`default`.`num_result`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_result), [id1, id2, result]
-+- Project [cast(id#x as int) AS id1#x, cast(id#x as int) AS id2#x, cast((val - val)#x as decimal(38,10)) AS result#x]
++- Project [id#x AS id1#x, id#x AS id2#x, cast((val - val)#x as decimal(38,10)) AS result#x]
+- Project [id#x, id#x, (val#x - val#x) AS (val - val)#x]
+- Join Inner
:- SubqueryAlias t1
@@ -3801,7 +3801,7 @@ INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val - t2.val, 40)
FROM num_data t1, num_data t2
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_result, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_result], Append, `spark_catalog`.`default`.`num_result`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_result), [id1, id2, result]
-+- Project [cast(id#x as int) AS id1#x, cast(id#x as int) AS id2#x, cast(round((val - val), 40)#x as decimal(38,10)) AS result#x]
++- Project [id#x AS id1#x, id#x AS id2#x, cast(round((val - val), 40)#x as decimal(38,10)) AS result#x]
+- Project [id#x, id#x, round((val#x - val#x), 40) AS round((val - val), 40)#x]
+- Join Inner
:- SubqueryAlias t1
@@ -3879,7 +3879,7 @@ INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val * t2.val, 30)
FROM num_data t1, num_data t2
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_result, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_result], Append, `spark_catalog`.`default`.`num_result`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_result), [id1, id2, result]
-+- Project [cast(id#x as int) AS id1#x, cast(id#x as int) AS id2#x, cast(round((val * val), 30)#x as decimal(38,10)) AS result#x]
++- Project [id#x AS id1#x, id#x AS id2#x, cast(round((val * val), 30)#x as decimal(38,10)) AS result#x]
+- Project [id#x, id#x, round((val#x * val#x), 30) AS round((val * val), 30)#x]
+- Join Inner
:- SubqueryAlias t1
@@ -3919,7 +3919,7 @@ INSERT INTO num_result SELECT t1.id, t2.id, t1.val / t2.val
WHERE t2.val != '0.0'
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_result, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_result], Append, `spark_catalog`.`default`.`num_result`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_result), [id1, id2, result]
-+- Project [cast(id#x as int) AS id1#x, cast(id#x as int) AS id2#x, cast((val / val)#x as decimal(38,10)) AS result#x]
++- Project [id#x AS id1#x, id#x AS id2#x, cast((val / val)#x as decimal(38,10)) AS result#x]
+- Project [id#x, id#x, (val#x / val#x) AS (val / val)#x]
+- Filter NOT (cast(val#x as double) = cast(0.0 as double))
+- Join Inner
@@ -3960,7 +3960,7 @@ INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val / t2.val, 80)
WHERE t2.val != '0.0'
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_result, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_result], Append, `spark_catalog`.`default`.`num_result`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_result), [id1, id2, result]
-+- Project [cast(id#x as int) AS id1#x, cast(id#x as int) AS id2#x, cast(round((val / val), 80)#x as decimal(38,10)) AS result#x]
++- Project [id#x AS id1#x, id#x AS id2#x, cast(round((val / val), 80)#x as decimal(38,10)) AS result#x]
+- Project [id#x, id#x, round((val#x / val#x), 80) AS round((val / val), 80)#x]
+- Filter NOT (cast(val#x as double) = cast(0.0 as double))
+- Join Inner
@@ -4000,7 +4000,7 @@ INSERT INTO num_result SELECT id, 0, SQRT(ABS(val))
FROM num_data
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_result, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_result], Append, `spark_catalog`.`default`.`num_result`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_result), [id1, id2, result]
-+- Project [cast(id#x as int) AS id1#x, cast(0#x as int) AS id2#x, cast(SQRT(abs(val))#x as decimal(38,10)) AS result#x]
++- Project [id#x AS id1#x, 0#x AS id2#x, cast(SQRT(abs(val))#x as decimal(38,10)) AS result#x]
+- Project [id#x, 0 AS 0#x, SQRT(cast(abs(val#x) as double)) AS SQRT(abs(val))#x]
+- SubqueryAlias spark_catalog.default.num_data
+- Relation spark_catalog.default.num_data[id#x,val#x] parquet
@@ -4035,7 +4035,7 @@ INSERT INTO num_result SELECT id, 0, LN(ABS(val))
WHERE val != '0.0'
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_result, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_result], Append, `spark_catalog`.`default`.`num_result`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_result), [id1, id2, result]
-+- Project [cast(id#x as int) AS id1#x, cast(0#x as int) AS id2#x, cast(ln(abs(val))#x as decimal(38,10)) AS result#x]
++- Project [id#x AS id1#x, 0#x AS id2#x, cast(ln(abs(val))#x as decimal(38,10)) AS result#x]
+- Project [id#x, 0 AS 0#x, ln(cast(abs(val#x) as double)) AS ln(abs(val))#x]
+- Filter NOT (cast(val#x as double) = cast(0.0 as double))
+- SubqueryAlias spark_catalog.default.num_data
@@ -4071,7 +4071,7 @@ INSERT INTO num_result SELECT id, 0, LOG(cast('10' as decimal(38, 18)), ABS(val)
WHERE val != '0.0'
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_result, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_result], Append, `spark_catalog`.`default`.`num_result`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_result), [id1, id2, result]
-+- Project [cast(id#x as int) AS id1#x, cast(0#x as int) AS id2#x, cast(LOG(CAST(10 AS DECIMAL(38,18)), abs(val))#x as decimal(38,10)) AS result#x]
++- Project [id#x AS id1#x, 0#x AS id2#x, cast(LOG(CAST(10 AS DECIMAL(38,18)), abs(val))#x as decimal(38,10)) AS result#x]
+- Project [id#x, 0 AS 0#x, LOG(cast(cast(10 as decimal(38,18)) as double), cast(abs(val#x) as double)) AS LOG(CAST(10 AS DECIMAL(38,18)), abs(val))#x]
+- Filter NOT (cast(val#x as double) = cast(0.0 as double))
+- SubqueryAlias spark_catalog.default.num_data
@@ -4107,7 +4107,7 @@ INSERT INTO num_result SELECT id, 0, POWER(cast('10' as decimal(38, 18)), LN(ABS
WHERE val != '0.0'
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/num_result, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/num_result], Append, `spark_catalog`.`default`.`num_result`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/num_result), [id1, id2, result]
-+- Project [cast(id#x as int) AS id1#x, cast(0#x as int) AS id2#x, cast(POWER(CAST(10 AS DECIMAL(38,18)), ln(abs(round(val, 200))))#x as decimal(38,10)) AS result#x]
++- Project [id#x AS id1#x, 0#x AS id2#x, cast(POWER(CAST(10 AS DECIMAL(38,18)), ln(abs(round(val, 200))))#x as decimal(38,10)) AS result#x]
+- Project [id#x, 0 AS 0#x, POWER(cast(cast(10 as decimal(38,18)) as double), ln(cast(abs(round(val#x, 200)) as double))) AS POWER(CAST(10 AS DECIMAL(38,18)), ln(abs(round(val, 200))))#x]
+- Filter NOT (cast(val#x as double) = cast(0.0 as double))
+- SubqueryAlias spark_catalog.default.num_data
@@ -4149,7 +4149,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`fract_only`, false
INSERT INTO fract_only VALUES (1, 0.0)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/fract_only, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/fract_only], Append, `spark_catalog`.`default`.`fract_only`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/fract_only), [id, val]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(4,4)) AS val#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(4,4)) AS val#x]
+- LocalRelation [col1#x, col2#x]
@@ -4157,7 +4157,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO fract_only VALUES (2, 0.1)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/fract_only, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/fract_only], Append, `spark_catalog`.`default`.`fract_only`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/fract_only), [id, val]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(4,4)) AS val#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(4,4)) AS val#x]
+- LocalRelation [col1#x, col2#x]
@@ -4165,7 +4165,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO fract_only VALUES (4, -0.9999)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/fract_only, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/fract_only], Append, `spark_catalog`.`default`.`fract_only`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/fract_only), [id, val]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(4,4)) AS val#x]
++- Project [col1#x AS id#x, col2#x AS val#x]
+- LocalRelation [col1#x, col2#x]
@@ -4173,7 +4173,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO fract_only VALUES (5, 0.99994)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/fract_only, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/fract_only], Append, `spark_catalog`.`default`.`fract_only`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/fract_only), [id, val]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(4,4)) AS val#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(4,4)) AS val#x]
+- LocalRelation [col1#x, col2#x]
@@ -4181,7 +4181,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO fract_only VALUES (7, 0.00001)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/fract_only, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/fract_only], Append, `spark_catalog`.`default`.`fract_only`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/fract_only), [id, val]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(4,4)) AS val#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(4,4)) AS val#x]
+- LocalRelation [col1#x, col2#x]
@@ -4189,7 +4189,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO fract_only VALUES (8, 0.00017)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/fract_only, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/fract_only], Append, `spark_catalog`.`default`.`fract_only`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/fract_only), [id, val]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as decimal(4,4)) AS val#x]
++- Project [col1#x AS id#x, cast(col2#x as decimal(4,4)) AS val#x]
+- LocalRelation [col1#x, col2#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_having.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_having.sql.out
index 640ab20757936..72b607951ec1d 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_having.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_having.sql.out
@@ -9,7 +9,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`test_having`, false
INSERT INTO test_having VALUES (0, 1, 'XXXX', 'A')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x]
++- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS d#x]
+- LocalRelation [col1#x, col2#x, col3#x, col4#x]
@@ -17,7 +17,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO test_having VALUES (1, 2, 'AAAA', 'b')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x]
++- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS d#x]
+- LocalRelation [col1#x, col2#x, col3#x, col4#x]
@@ -25,7 +25,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO test_having VALUES (2, 2, 'AAAA', 'c')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x]
++- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS d#x]
+- LocalRelation [col1#x, col2#x, col3#x, col4#x]
@@ -33,7 +33,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO test_having VALUES (3, 3, 'BBBB', 'D')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x]
++- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS d#x]
+- LocalRelation [col1#x, col2#x, col3#x, col4#x]
@@ -41,7 +41,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO test_having VALUES (4, 3, 'BBBB', 'e')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x]
++- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS d#x]
+- LocalRelation [col1#x, col2#x, col3#x, col4#x]
@@ -49,7 +49,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO test_having VALUES (5, 3, 'bbbb', 'F')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x]
++- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS d#x]
+- LocalRelation [col1#x, col2#x, col3#x, col4#x]
@@ -57,7 +57,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO test_having VALUES (6, 4, 'cccc', 'g')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x]
++- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS d#x]
+- LocalRelation [col1#x, col2#x, col3#x, col4#x]
@@ -65,7 +65,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO test_having VALUES (7, 4, 'cccc', 'h')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x]
++- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS d#x]
+- LocalRelation [col1#x, col2#x, col3#x, col4#x]
@@ -73,7 +73,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO test_having VALUES (8, 4, 'CCCC', 'I')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x]
++- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS d#x]
+- LocalRelation [col1#x, col2#x, col3#x, col4#x]
@@ -81,7 +81,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO test_having VALUES (9, 4, 'CCCC', 'j')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_having, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_having], Append, `spark_catalog`.`default`.`test_having`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_having), [a, b, c, d]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x]
++- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS d#x]
+- LocalRelation [col1#x, col2#x, col3#x, col4#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_implicit.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_implicit.sql.out
index 83b10b3cb67c4..e456a80d5e049 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_implicit.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_implicit.sql.out
@@ -9,7 +9,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`test_missing_target`, fa
INSERT INTO test_missing_target VALUES (0, 1, 'XXXX', 'A')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x]
++- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS d#x]
+- LocalRelation [col1#x, col2#x, col3#x, col4#x]
@@ -17,7 +17,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO test_missing_target VALUES (1, 2, 'ABAB', 'b')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x]
++- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS d#x]
+- LocalRelation [col1#x, col2#x, col3#x, col4#x]
@@ -25,7 +25,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO test_missing_target VALUES (2, 2, 'ABAB', 'c')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x]
++- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS d#x]
+- LocalRelation [col1#x, col2#x, col3#x, col4#x]
@@ -33,7 +33,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO test_missing_target VALUES (3, 3, 'BBBB', 'D')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x]
++- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS d#x]
+- LocalRelation [col1#x, col2#x, col3#x, col4#x]
@@ -41,7 +41,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO test_missing_target VALUES (4, 3, 'BBBB', 'e')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x]
++- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS d#x]
+- LocalRelation [col1#x, col2#x, col3#x, col4#x]
@@ -49,7 +49,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO test_missing_target VALUES (5, 3, 'bbbb', 'F')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x]
++- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS d#x]
+- LocalRelation [col1#x, col2#x, col3#x, col4#x]
@@ -57,7 +57,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO test_missing_target VALUES (6, 4, 'cccc', 'g')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x]
++- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS d#x]
+- LocalRelation [col1#x, col2#x, col3#x, col4#x]
@@ -65,7 +65,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO test_missing_target VALUES (7, 4, 'cccc', 'h')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x]
++- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS d#x]
+- LocalRelation [col1#x, col2#x, col3#x, col4#x]
@@ -73,7 +73,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO test_missing_target VALUES (8, 4, 'CCCC', 'I')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x]
++- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS d#x]
+- LocalRelation [col1#x, col2#x, col3#x, col4#x]
@@ -81,7 +81,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO test_missing_target VALUES (9, 4, 'CCCC', 'j')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test_missing_target, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test_missing_target], Append, `spark_catalog`.`default`.`test_missing_target`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test_missing_target), [a, b, c, d]
-+- Project [cast(col1#x as int) AS a#x, cast(col2#x as int) AS b#x, cast(col3#x as string) AS c#x, cast(col4#x as string) AS d#x]
++- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS d#x]
+- LocalRelation [col1#x, col2#x, col3#x, col4#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/strings.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/strings.sql.out
index 2a4d79360b5c1..57d6a813c25d3 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/strings.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/strings.sql.out
@@ -512,7 +512,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`toasttest`, false
insert into toasttest values(repeat('1234567890',10000))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/toasttest, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/toasttest], Append, `spark_catalog`.`default`.`toasttest`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/toasttest), [f1]
-+- Project [cast(col1#x as string) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -520,7 +520,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
insert into toasttest values(repeat('1234567890',10000))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/toasttest, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/toasttest], Append, `spark_catalog`.`default`.`toasttest`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/toasttest), [f1]
-+- Project [cast(col1#x as string) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -528,7 +528,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
insert into toasttest values(repeat('1234567890',10000))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/toasttest, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/toasttest], Append, `spark_catalog`.`default`.`toasttest`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/toasttest), [f1]
-+- Project [cast(col1#x as string) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -536,7 +536,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
insert into toasttest values(repeat('1234567890',10000))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/toasttest, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/toasttest], Append, `spark_catalog`.`default`.`toasttest`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/toasttest), [f1]
-+- Project [cast(col1#x as string) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/text.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/text.sql.out
index 474c2401f40d0..ef7b7a4180ba1 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/text.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/text.sql.out
@@ -23,7 +23,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`TEXT_TBL`, false
INSERT INTO TEXT_TBL VALUES ('doh!')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/text_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/text_tbl], Append, `spark_catalog`.`default`.`text_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/text_tbl), [f1]
-+- Project [cast(col1#x as string) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
@@ -31,7 +31,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO TEXT_TBL VALUES ('hi de ho neighbor')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/text_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/text_tbl], Append, `spark_catalog`.`default`.`text_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/text_tbl), [f1]
-+- Project [cast(col1#x as string) AS f1#x]
++- Project [col1#x AS f1#x]
+- LocalRelation [col1#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/timestamp.sql.out
index a6c3c27829698..3bc151fe61956 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/timestamp.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/timestamp.sql.out
@@ -9,7 +9,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`TIMESTAMP_TBL`, false
INSERT INTO TIMESTAMP_TBL VALUES (timestamp'now')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/timestamp_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/timestamp_tbl], Append, `spark_catalog`.`default`.`timestamp_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/timestamp_tbl), [d1]
-+- Project [cast(col1#x as timestamp) AS d1#x]
++- Project [col1#x AS d1#x]
+- LocalRelation [col1#x]
@@ -17,7 +17,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO TIMESTAMP_TBL VALUES (timestamp'now')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/timestamp_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/timestamp_tbl], Append, `spark_catalog`.`default`.`timestamp_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/timestamp_tbl), [d1]
-+- Project [cast(col1#x as timestamp) AS d1#x]
++- Project [col1#x AS d1#x]
+- LocalRelation [col1#x]
@@ -25,7 +25,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO TIMESTAMP_TBL VALUES (timestamp'today')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/timestamp_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/timestamp_tbl], Append, `spark_catalog`.`default`.`timestamp_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/timestamp_tbl), [d1]
-+- Project [cast(col1#x as timestamp) AS d1#x]
++- Project [col1#x AS d1#x]
+- LocalRelation [col1#x]
@@ -33,7 +33,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO TIMESTAMP_TBL VALUES (timestamp'yesterday')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/timestamp_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/timestamp_tbl], Append, `spark_catalog`.`default`.`timestamp_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/timestamp_tbl), [d1]
-+- Project [cast(col1#x as timestamp) AS d1#x]
++- Project [col1#x AS d1#x]
+- LocalRelation [col1#x]
@@ -41,7 +41,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO TIMESTAMP_TBL VALUES (timestamp'tomorrow')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/timestamp_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/timestamp_tbl], Append, `spark_catalog`.`default`.`timestamp_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/timestamp_tbl), [d1]
-+- Project [cast(col1#x as timestamp) AS d1#x]
++- Project [col1#x AS d1#x]
+- LocalRelation [col1#x]
@@ -49,7 +49,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO TIMESTAMP_TBL VALUES (timestamp'tomorrow EST')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/timestamp_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/timestamp_tbl], Append, `spark_catalog`.`default`.`timestamp_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/timestamp_tbl), [d1]
-+- Project [cast(col1#x as timestamp) AS d1#x]
++- Project [col1#x AS d1#x]
+- LocalRelation [col1#x]
@@ -57,7 +57,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO TIMESTAMP_TBL VALUES (timestamp'tomorrow Zulu')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/timestamp_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/timestamp_tbl], Append, `spark_catalog`.`default`.`timestamp_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/timestamp_tbl), [d1]
-+- Project [cast(col1#x as timestamp) AS d1#x]
++- Project [col1#x AS d1#x]
+- LocalRelation [col1#x]
@@ -89,7 +89,7 @@ TruncateTableCommand `spark_catalog`.`default`.`timestamp_tbl`
INSERT INTO TIMESTAMP_TBL VALUES (timestamp'epoch')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/timestamp_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/timestamp_tbl], Append, `spark_catalog`.`default`.`timestamp_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/timestamp_tbl), [d1]
-+- Project [cast(col1#x as timestamp) AS d1#x]
++- Project [col1#x AS d1#x]
+- LocalRelation [col1#x]
@@ -97,7 +97,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO TIMESTAMP_TBL VALUES (timestamp('1997-01-02'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/timestamp_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/timestamp_tbl], Append, `spark_catalog`.`default`.`timestamp_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/timestamp_tbl), [d1]
-+- Project [cast(col1#x as timestamp) AS d1#x]
++- Project [col1#x AS d1#x]
+- LocalRelation [col1#x]
@@ -105,7 +105,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO TIMESTAMP_TBL VALUES (timestamp('1997-01-02 03:04:05'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/timestamp_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/timestamp_tbl], Append, `spark_catalog`.`default`.`timestamp_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/timestamp_tbl), [d1]
-+- Project [cast(col1#x as timestamp) AS d1#x]
++- Project [col1#x AS d1#x]
+- LocalRelation [col1#x]
@@ -113,7 +113,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO TIMESTAMP_TBL VALUES (timestamp('1997-02-10 17:32:01-08'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/timestamp_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/timestamp_tbl], Append, `spark_catalog`.`default`.`timestamp_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/timestamp_tbl), [d1]
-+- Project [cast(col1#x as timestamp) AS d1#x]
++- Project [col1#x AS d1#x]
+- LocalRelation [col1#x]
@@ -121,7 +121,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO TIMESTAMP_TBL VALUES (timestamp('2001-09-22T18:19:20'))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/timestamp_tbl, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/timestamp_tbl], Append, `spark_catalog`.`default`.`timestamp_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/timestamp_tbl), [d1]
-+- Project [cast(col1#x as timestamp) AS d1#x]
++- Project [col1#x AS d1#x]
+- LocalRelation [col1#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part2.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part2.sql.out
index cdcd563de4f6a..5281494bece15 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part2.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part2.sql.out
@@ -24,7 +24,7 @@ INSERT INTO empsalary VALUES
('develop', 11, 5200, date '2007-08-15')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/empsalary, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/empsalary], Append, `spark_catalog`.`default`.`empsalary`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/empsalary), [depname, empno, salary, enroll_date]
-+- Project [cast(col1#x as string) AS depname#x, cast(col2#x as int) AS empno#x, cast(col3#x as int) AS salary#x, cast(col4#x as date) AS enroll_date#x]
++- Project [col1#x AS depname#x, col2#x AS empno#x, col3#x AS salary#x, col4#x AS enroll_date#x]
+- LocalRelation [col1#x, col2#x, col3#x, col4#x]
@@ -280,7 +280,7 @@ insert into numerics values
(7, 100, 100, 100)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/numerics, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/numerics], Append, `spark_catalog`.`default`.`numerics`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/numerics), [id, f_float4, f_float8, f_numeric]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as float) AS f_float4#x, cast(col3#x as float) AS f_float8#x, cast(col4#x as int) AS f_numeric#x]
++- Project [col1#x AS id#x, cast(col2#x as float) AS f_float4#x, cast(col3#x as float) AS f_float8#x, cast(col4#x as int) AS f_numeric#x]
+- LocalRelation [col1#x, col2#x, col3#x, col4#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out
index d552c108d4612..bc6dc828ad857 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out
@@ -33,7 +33,7 @@ INSERT INTO empsalary VALUES
('develop', 11, 5200, date '2007-08-15')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/empsalary, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/empsalary], Append, `spark_catalog`.`default`.`empsalary`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/empsalary), [depname, empno, salary, enroll_date]
-+- Project [cast(col1#x as string) AS depname#x, cast(col2#x as int) AS empno#x, cast(col3#x as int) AS salary#x, cast(col4#x as date) AS enroll_date#x]
++- Project [col1#x AS depname#x, col2#x AS empno#x, col3#x AS salary#x, col4#x AS enroll_date#x]
+- LocalRelation [col1#x, col2#x, col3#x, col4#x]
@@ -223,7 +223,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false
insert into t1 values (1,1),(1,2),(2,2)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [f1, f2]
-+- Project [cast(col1#x as int) AS f1#x, cast(col2#x as int) AS f2#x]
++- Project [col1#x AS f1#x, col2#x AS f2#x]
+- LocalRelation [col1#x, col2#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out
index de86c88f6d1b9..edbdcba851483 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out
@@ -325,7 +325,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`department`, false
INSERT INTO department VALUES (0, NULL, 'ROOT')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/department, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/department], Append, `spark_catalog`.`default`.`department`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/department), [id, parent_department, name]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as int) AS parent_department#x, cast(col3#x as string) AS name#x]
++- Project [col1#x AS id#x, cast(col2#x as int) AS parent_department#x, col3#x AS name#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -333,7 +333,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO department VALUES (1, 0, 'A')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/department, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/department], Append, `spark_catalog`.`default`.`department`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/department), [id, parent_department, name]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as int) AS parent_department#x, cast(col3#x as string) AS name#x]
++- Project [col1#x AS id#x, col2#x AS parent_department#x, col3#x AS name#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -341,7 +341,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO department VALUES (2, 1, 'B')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/department, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/department], Append, `spark_catalog`.`default`.`department`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/department), [id, parent_department, name]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as int) AS parent_department#x, cast(col3#x as string) AS name#x]
++- Project [col1#x AS id#x, col2#x AS parent_department#x, col3#x AS name#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -349,7 +349,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO department VALUES (3, 2, 'C')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/department, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/department], Append, `spark_catalog`.`default`.`department`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/department), [id, parent_department, name]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as int) AS parent_department#x, cast(col3#x as string) AS name#x]
++- Project [col1#x AS id#x, col2#x AS parent_department#x, col3#x AS name#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -357,7 +357,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO department VALUES (4, 2, 'D')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/department, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/department], Append, `spark_catalog`.`default`.`department`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/department), [id, parent_department, name]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as int) AS parent_department#x, cast(col3#x as string) AS name#x]
++- Project [col1#x AS id#x, col2#x AS parent_department#x, col3#x AS name#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -365,7 +365,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO department VALUES (5, 0, 'E')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/department, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/department], Append, `spark_catalog`.`default`.`department`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/department), [id, parent_department, name]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as int) AS parent_department#x, cast(col3#x as string) AS name#x]
++- Project [col1#x AS id#x, col2#x AS parent_department#x, col3#x AS name#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -373,7 +373,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO department VALUES (6, 4, 'F')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/department, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/department], Append, `spark_catalog`.`default`.`department`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/department), [id, parent_department, name]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as int) AS parent_department#x, cast(col3#x as string) AS name#x]
++- Project [col1#x AS id#x, col2#x AS parent_department#x, col3#x AS name#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -381,7 +381,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
INSERT INTO department VALUES (7, 5, 'G')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/department, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/department], Append, `spark_catalog`.`default`.`department`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/department), [id, parent_department, name]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as int) AS parent_department#x, cast(col3#x as string) AS name#x]
++- Project [col1#x AS id#x, col2#x AS parent_department#x, col3#x AS name#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -774,7 +774,7 @@ VALUES (1, NULL), (2, 1), (3,1), (4,2), (5,2), (6,2), (7,3), (8,3),
(9,4), (10,4), (11,7), (12,7), (13,7), (14, 9), (15,11), (16,11)
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/tree, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/tree], Append, `spark_catalog`.`default`.`tree`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/tree), [id, parent_id]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as int) AS parent_id#x]
++- Project [col1#x AS id#x, col2#x AS parent_id#x]
+- LocalRelation [col1#x, col2#x]
@@ -902,7 +902,7 @@ insert into graph values
(5, 1, 'arc 5 -> 1')
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/graph, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/graph], Append, `spark_catalog`.`default`.`graph`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/graph), [f, t, label]
-+- Project [cast(col1#x as int) AS f#x, cast(col2#x as int) AS t#x, cast(col3#x as string) AS label#x]
++- Project [col1#x AS f#x, col2#x AS t#x, col3#x AS label#x]
+- LocalRelation [col1#x, col2#x, col3#x]
@@ -1111,7 +1111,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`y`, false
INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/y, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/y], Append, `spark_catalog`.`default`.`y`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/y), [a]
-+- Project [cast(col#x as int) AS a#x]
++- Project [col#x AS a#x]
+- Project [col#x]
+- Generate explode(sequence(1, 10, None, Some(America/Los_Angeles))), false, [col#x]
+- OneRowRelation
@@ -1183,20 +1183,10 @@ org.apache.spark.sql.AnalysisException
WITH RECURSIVE x(n) AS (SELECT n FROM x UNION ALL SELECT 1)
SELECT * FROM x
-- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
+org.apache.spark.sql.AnalysisException
{
- "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION",
- "sqlState" : "42703",
- "messageParameters" : {
- "objectName" : "`n`"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 32,
- "stopIndex" : 32,
- "fragment" : "n"
- } ]
+ "errorClass" : "INVALID_RECURSIVE_CTE",
+ "sqlState" : "42836"
}
@@ -1210,7 +1200,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`y`, false
INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/y, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/y], Append, `spark_catalog`.`default`.`y`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/y), [a]
-+- Project [cast(col#x as int) AS a#x]
++- Project [col#x AS a#x]
+- Project [col#x]
+- Generate explode(sequence(1, 10, None, Some(America/Los_Angeles))), false, [col#x]
+- OneRowRelation
@@ -1670,15 +1660,8 @@ SELECT * FROM outermost ORDER BY 1
-- !query analysis
org.apache.spark.sql.AnalysisException
{
- "errorClass" : "UNION_NOT_SUPPORTED_IN_RECURSIVE_CTE",
- "sqlState" : "42836",
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 1,
- "stopIndex" : 185,
- "fragment" : "WITH RECURSIVE outermost(x) AS (\n WITH innermost as (SELECT 2 FROM outermost) -- fail\n SELECT * FROM innermost\n UNION SELECT * from outermost\n)\nSELECT * FROM outermost ORDER BY 1"
- } ]
+ "errorClass" : "INVALID_RECURSIVE_CTE",
+ "sqlState" : "42836"
}
@@ -1719,7 +1702,7 @@ TruncateTableCommand `spark_catalog`.`default`.`y`
INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 3))
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/y, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/y], Append, `spark_catalog`.`default`.`y`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/y), [a]
-+- Project [cast(col#x as int) AS a#x]
++- Project [col#x AS a#x]
+- Project [col#x]
+- Generate explode(sequence(1, 3, None, Some(America/Los_Angeles))), false, [col#x]
+- OneRowRelation
@@ -1773,7 +1756,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`parent`, false
INSERT INTO parent VALUES ( 1, 'p1' )
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/parent, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/parent], Append, `spark_catalog`.`default`.`parent`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/parent), [id, val]
-+- Project [cast(col1#x as int) AS id#x, cast(col2#x as string) AS val#x]
++- Project [col1#x AS id#x, col2#x AS val#x]
+- LocalRelation [col1#x, col2#x]
@@ -1876,7 +1859,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`test`, false
with test as (select 42) insert into test select * from test
-- !query analysis
InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test], Append, `spark_catalog`.`default`.`test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test), [i]
-+- Project [cast(42#x as int) AS i#x]
++- Project [42#x AS i#x]
+- WithCTE
:- CTERelationDef xxxx, false
: +- SubqueryAlias test
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/replacing-missing-expression-with-alias.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/replacing-missing-expression-with-alias.sql.out
new file mode 100644
index 0000000000000..a353c9e8ed018
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/replacing-missing-expression-with-alias.sql.out
@@ -0,0 +1,204 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+SELECT col1 + 1 AS a FROM VALUES(1) GROUP BY a ORDER BY col1 + 1
+-- !query analysis
+Sort [a#x ASC NULLS FIRST], true
++- Aggregate [(col1#x + 1)], [(col1#x + 1) AS a#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT col1 + 1 AS a, a AS b FROM VALUES(1) GROUP BY a ORDER BY col1 + 1
+-- !query analysis
+Sort [a#x ASC NULLS FIRST], true
++- Project [a#x, a#x AS b#x]
+ +- Project [(col1 + 1)#x, (col1 + 1)#x AS a#x]
+ +- Aggregate [(col1#x + 1)], [(col1#x + 1) AS (col1 + 1)#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT col1 + 1 AS a FROM VALUES(1) GROUP BY a HAVING col1 + 1 > 0
+-- !query analysis
+Filter (a#x > 0)
++- Aggregate [(col1#x + 1)], [(col1#x + 1) AS a#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT col1 + 1 AS a, a AS b FROM VALUES(1) GROUP BY a HAVING col1 + 1 > 0
+-- !query analysis
+Filter (a#x > 0)
++- Project [a#x, a#x AS b#x]
+ +- Project [(col1 + 1)#x, (col1 + 1)#x AS a#x]
+ +- Aggregate [(col1#x + 1)], [(col1#x + 1) AS (col1 + 1)#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT col1, col2, GROUPING(col1) FROM VALUES("abc", 1) GROUP BY CUBE(col1, col2) ORDER BY GROUPING(col1)
+-- !query analysis
+Sort [grouping(col1)#x ASC NULLS FIRST], true
++- Aggregate [col1#x, col2#x, spark_grouping_id#xL], [col1#x, col2#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(col1)#x]
+ +- Expand [[col1#x, col2#x, col1#x, col2#x, 0], [col1#x, col2#x, col1#x, null, 1], [col1#x, col2#x, null, col2#x, 2], [col1#x, col2#x, null, null, 3]], [col1#x, col2#x, col1#x, col2#x, spark_grouping_id#xL]
+ +- Project [col1#x, col2#x, col1#x AS col1#x, col2#x AS col2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT col1, col2, GROUPING(col1) FROM VALUES("abc", 1) GROUP BY CUBE(col1, col2) HAVING GROUPING(col1) != NULL
+-- !query analysis
+Filter NOT (grouping(col1)#x = cast(null as tinyint))
++- Aggregate [col1#x, col2#x, spark_grouping_id#xL], [col1#x, col2#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(col1)#x]
+ +- Expand [[col1#x, col2#x, col1#x, col2#x, 0], [col1#x, col2#x, col1#x, null, 1], [col1#x, col2#x, null, col2#x, 2], [col1#x, col2#x, null, null, 3]], [col1#x, col2#x, col1#x, col2#x, spark_grouping_id#xL]
+ +- Project [col1#x, col2#x, col1#x AS col1#x, col2#x AS col2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT make_date(col1, col2, col3) AS a FROM VALUES(1,2,3) GROUP BY make_date(col1, col2, col3) ORDER BY make_date(col1, col2, col3)
+-- !query analysis
+Sort [a#x ASC NULLS FIRST], true
++- Aggregate [make_date(col1#x, col2#x, col3#x, true)], [make_date(col1#x, col2#x, col3#x, true) AS a#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+SELECT make_date(col1, col2, col3) AS a, a AS b FROM VALUES(1,2,3) GROUP BY make_date(col1, col2, col3) ORDER BY make_date(col1, col2, col3)
+-- !query analysis
+Sort [a#x ASC NULLS FIRST], true
++- Project [a#x, a#x AS b#x]
+ +- Project [make_date(col1, col2, col3)#x, make_date(col1, col2, col3)#x AS a#x]
+ +- Aggregate [make_date(col1#x, col2#x, col3#x, true)], [make_date(col1#x, col2#x, col3#x, true) AS make_date(col1, col2, col3)#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+SELECT make_date(col1, col2, col3) AS a FROM VALUES(1,2,3) GROUP BY make_date(col1, col2, col3) HAVING make_date(col1, col2, col3) > '2025-01-01'
+-- !query analysis
+Filter (a#x > cast(2025-01-01 as date))
++- Aggregate [make_date(col1#x, col2#x, col3#x, true)], [make_date(col1#x, col2#x, col3#x, true) AS a#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+SELECT make_date(col1, col2, col3) AS a, a AS b FROM VALUES(1,2,3) GROUP BY make_date(col1, col2, col3) HAVING make_date(col1, col2, col3) > '2025-01-01'
+-- !query analysis
+Filter (a#x > cast(2025-01-01 as date))
++- Project [a#x, a#x AS b#x]
+ +- Project [make_date(col1, col2, col3)#x, make_date(col1, col2, col3)#x AS a#x]
+ +- Aggregate [make_date(col1#x, col2#x, col3#x, true)], [make_date(col1#x, col2#x, col3#x, true) AS make_date(col1, col2, col3)#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+SELECT make_date(col1, col2, col3) AS a FROM VALUES(1,2,3) ORDER BY make_date(col1, col2, col3)
+-- !query analysis
+Project [a#x]
++- Sort [make_date(col1#x, col2#x, col3#x, true) ASC NULLS FIRST], true
+ +- Project [make_date(col1#x, col2#x, col3#x, true) AS a#x, col1#x, col2#x, col3#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+SELECT make_date(col1, col2, col3) AS a, a AS b FROM VALUES(1,2,3) ORDER BY make_date(col1, col2, col3)
+-- !query analysis
+Project [a#x, b#x]
++- Sort [make_date(col1#x, col2#x, col3#x, true) ASC NULLS FIRST], true
+ +- Project [a#x, a#x AS b#x, col1#x, col2#x, col3#x]
+ +- Project [col1#x, col2#x, col3#x, make_date(col1#x, col2#x, col3#x, true) AS a#x]
+ +- LocalRelation [col1#x, col2#x, col3#x]
+
+
+-- !query
+SELECT col1, col1 AS a FROM VALUES(1) GROUP BY col1 ORDER BY col1 ASC
+-- !query analysis
+Sort [col1#x ASC NULLS FIRST], true
++- Aggregate [col1#x], [col1#x, col1#x AS a#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT col1 AS a, col1 FROM VALUES(1) GROUP BY col1 ORDER BY col1 ASC
+-- !query analysis
+Sort [col1#x ASC NULLS FIRST], true
++- Aggregate [col1#x], [col1#x AS a#x, col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT col1, col1 AS a FROM VALUES(1) GROUP BY col1 HAVING col1 > 0
+-- !query analysis
+Filter (col1#x > 0)
++- Aggregate [col1#x], [col1#x, col1#x AS a#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT col1 AS a, col1 FROM VALUES(1) GROUP BY col1 HAVING col1 > 0
+-- !query analysis
+Filter (col1#x > 0)
++- Aggregate [col1#x], [col1#x AS a#x, col1#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT col2 AS b, col2 FROM VALUES(1,2) GROUP BY 1,2 ORDER BY ALL
+-- !query analysis
+Sort [b#x ASC NULLS FIRST, col2#x ASC NULLS FIRST], true
++- Aggregate [col2#x, col2#x], [col2#x AS b#x, col2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT col2 AS b, col2 FROM VALUES(1,2) GROUP BY 1,2 HAVING col2 > 0 ORDER BY ALL
+-- !query analysis
+Sort [b#x ASC NULLS FIRST, col2#x ASC NULLS FIRST], true
++- Filter (col2#x > 0)
+ +- Aggregate [col2#x, col2#x], [col2#x AS b#x, col2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT col2 AS b, col2, b as c FROM VALUES(1,2) GROUP BY 1,2 ORDER BY ALL
+-- !query analysis
+Sort [b#x ASC NULLS FIRST, col2#x ASC NULLS FIRST, c#x ASC NULLS FIRST], true
++- Project [b#x, col2#x, b#x AS c#x]
+ +- Project [col2#x, col2#x AS b#x]
+ +- Aggregate [col2#x, col2#x], [col2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT col2 AS b, col2, b as c FROM VALUES(1,2) GROUP BY 1,2 HAVING col2 > 0 ORDER BY ALL
+-- !query analysis
+Sort [b#x ASC NULLS FIRST, col2#x ASC NULLS FIRST, c#x ASC NULLS FIRST], true
++- Filter (col2#x > 0)
+ +- Project [b#x, col2#x, b#x AS c#x]
+ +- Project [col2#x, col2#x AS b#x]
+ +- Aggregate [col2#x, col2#x], [col2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT col1 AS a FROM VALUES(1,2) GROUP BY col1, col2 HAVING col2 > 1 ORDER BY col1
+-- !query analysis
+Project [a#x]
++- Sort [col1#x ASC NULLS FIRST], true
+ +- Project [a#x, col1#x]
+ +- Filter (col2#x > 1)
+ +- Aggregate [col1#x, col2#x], [col1#x AS a#x, col2#x, col1#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT col1 AS a, a AS b FROM VALUES(1,2) GROUP BY col1, col2 HAVING col2 > 1 ORDER BY col1
+-- !query analysis
+Project [a#x, b#x]
++- Sort [col1#x ASC NULLS FIRST], true
+ +- Project [a#x, b#x, col1#x]
+ +- Filter (col2#x > 1)
+ +- Project [a#x, a#x AS b#x, col2#x, col1#x]
+ +- Project [col1#x, col2#x, col1#x AS a#x]
+ +- Aggregate [col1#x, col2#x], [col1#x, col2#x]
+ +- LocalRelation [col1#x, col2#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-session-variables.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-session-variables.sql.out
index 6f22f598498c6..7d23f27cdf5d6 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-session-variables.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-session-variables.sql.out
@@ -1247,9 +1247,176 @@ DropVariable false
-- !query
-SET VARIABLE title = 'SET VARIABLE - row assignment'
+SET VARIABLE title = 'DECLARE VARIABLE - multiple variables declared at once'
-- !query analysis
SetVariable [variablereference(system.session.title='SET VARIABLE - comma separated target')]
++- Project [DECLARE VARIABLE - multiple variables declared at once AS title#x]
+ +- OneRowRelation
+
+
+-- !query
+DECLARE VARIABLE var1, var2, var3 INT
+-- !query analysis
+CreateVariable defaultvalueexpression(null, null), false
+:- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.var1
+:- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.var2
++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.var3
+
+
+-- !query
+DECLARE VARIABLE var4, var5, var6 INT DEFAULT CAST(RAND(0) * 10 AS INT)
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+DECLARE VARIABLE var7, var8, var9 DEFAULT 5
+-- !query analysis
+CreateVariable defaultvalueexpression(5, 5), false
+:- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.var7
+:- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.var8
++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.var9
+
+
+-- !query
+SELECT var4 = var5, var4 = var6, var5 = var6
+-- !query analysis
+Project [(variablereference(system.session.var4=7) = variablereference(system.session.var5=7)) AS (variablereference(system.session.var4=7) = variablereference(system.session.var5=7))#x, (variablereference(system.session.var4=7) = variablereference(system.session.var6=7)) AS (variablereference(system.session.var4=7) = variablereference(system.session.var6=7))#x, (variablereference(system.session.var5=7) = variablereference(system.session.var6=7)) AS (variablereference(system.session.var5=7) = variablereference(system.session.var6=7))#x]
++- OneRowRelation
+
+
+-- !query
+DECLARE OR REPLACE VARIABLE var1, var2, var3 DOUBLE
+-- !query analysis
+CreateVariable defaultvalueexpression(null, null), true
+:- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.var1
+:- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.var2
++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.var3
+
+
+-- !query
+DECLARE OR REPLACE VARIABLE var4, var5, var6 DOUBLE DEFAULT RAND(0)
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+DECLARE OR REPLACE VARIABLE var7, var8, var9 DEFAULT 1.5
+-- !query analysis
+CreateVariable defaultvalueexpression(1.5, 1.5), true
+:- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.var7
+:- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.var8
++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.var9
+
+
+-- !query
+SELECT var4 = var5, var4 = var6, var5 = var6
+-- !query analysis
+Project [(variablereference(system.session.var4=0.7604953758285915D) = variablereference(system.session.var5=0.7604953758285915D)) AS (variablereference(system.session.var4=0.7604953758285915D) = variablereference(system.session.var5=0.7604953758285915D))#x, (variablereference(system.session.var4=0.7604953758285915D) = variablereference(system.session.var6=0.7604953758285915D)) AS (variablereference(system.session.var4=0.7604953758285915D) = variablereference(system.session.var6=0.7604953758285915D))#x, (variablereference(system.session.var5=0.7604953758285915D) = variablereference(system.session.var6=0.7604953758285915D)) AS (variablereference(system.session.var5=0.7604953758285915D) = variablereference(system.session.var6=0.7604953758285915D))#x]
++- OneRowRelation
+
+
+-- !query
+DROP TEMPORARY VARIABLE var1
+-- !query analysis
+DropVariable false
++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.var1
+
+
+-- !query
+DROP TEMPORARY VARIABLE var2
+-- !query analysis
+DropVariable false
++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.var2
+
+
+-- !query
+DROP TEMPORARY VARIABLE var3
+-- !query analysis
+DropVariable false
++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.var3
+
+
+-- !query
+DROP TEMPORARY VARIABLE var4
+-- !query analysis
+DropVariable false
++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.var4
+
+
+-- !query
+DROP TEMPORARY VARIABLE var5
+-- !query analysis
+DropVariable false
++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.var5
+
+
+-- !query
+DROP TEMPORARY VARIABLE var6
+-- !query analysis
+DropVariable false
++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.var6
+
+
+-- !query
+DROP TEMPORARY VARIABLE var7
+-- !query analysis
+DropVariable false
++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.var7
+
+
+-- !query
+DROP TEMPORARY VARIABLE var8
+-- !query analysis
+DropVariable false
++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.var8
+
+
+-- !query
+DROP TEMPORARY VARIABLE var9
+-- !query analysis
+DropVariable false
++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.var9
+
+
+-- !query
+SET VARIABLE title = 'DECLARE VARIABLE - duplicate names'
+-- !query analysis
+SetVariable [variablereference(system.session.title='DECLARE VARIABLE - multiple variables declared at once')]
++- Project [DECLARE VARIABLE - duplicate names AS title#x]
+ +- OneRowRelation
+
+
+-- !query
+DECLARE VARIABLE var1, var2, vAr1 INT
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "DUPLICATE_VARIABLE_NAME_INSIDE_DECLARE",
+ "sqlState" : "42734",
+ "messageParameters" : {
+ "variableName" : "`system`.`session`.`var1`"
+ }
+}
+
+
+-- !query
+DECLARE OR REPLACE VARIABLE var1, var2, vAr1 INT
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "DUPLICATE_VARIABLE_NAME_INSIDE_DECLARE",
+ "sqlState" : "42734",
+ "messageParameters" : {
+ "variableName" : "`system`.`session`.`var1`"
+ }
+}
+
+
+-- !query
+SET VARIABLE title = 'SET VARIABLE - row assignment'
+-- !query analysis
+SetVariable [variablereference(system.session.title='DECLARE VARIABLE - duplicate names')]
+- Project [SET VARIABLE - row assignment AS title#x]
+- OneRowRelation
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-udf.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-udf.sql.out
index d24ab2c760fbc..2b8a47c9ca633 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-udf.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-udf.sql.out
@@ -2,16 +2,7 @@
-- !query
CREATE FUNCTION foo1a0() RETURNS INT RETURN 1
-- !query analysis
-org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException
-{
- "errorClass" : "ROUTINE_ALREADY_EXISTS",
- "sqlState" : "42723",
- "messageParameters" : {
- "existingRoutineType" : "routine",
- "newRoutineType" : "routine",
- "routineName" : "`default`.`foo1a0`"
- }
-}
+CreateSQLFunctionCommand spark_catalog.default.foo1a0, INT, 1, false, false, false, false
-- !query
@@ -48,16 +39,7 @@ org.apache.spark.sql.AnalysisException
-- !query
CREATE FUNCTION foo1a1(a INT) RETURNS INT RETURN 1
-- !query analysis
-org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException
-{
- "errorClass" : "ROUTINE_ALREADY_EXISTS",
- "sqlState" : "42723",
- "messageParameters" : {
- "existingRoutineType" : "routine",
- "newRoutineType" : "routine",
- "routineName" : "`default`.`foo1a1`"
- }
-}
+CreateSQLFunctionCommand spark_catalog.default.foo1a1, a INT, INT, 1, false, false, false, false
-- !query
@@ -94,16 +76,7 @@ org.apache.spark.sql.AnalysisException
-- !query
CREATE FUNCTION foo1a2(a INT, b INT, c INT, d INT) RETURNS INT RETURN 1
-- !query analysis
-org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException
-{
- "errorClass" : "ROUTINE_ALREADY_EXISTS",
- "sqlState" : "42723",
- "messageParameters" : {
- "existingRoutineType" : "routine",
- "newRoutineType" : "routine",
- "routineName" : "`default`.`foo1a2`"
- }
-}
+CreateSQLFunctionCommand spark_catalog.default.foo1a2, a INT, b INT, c INT, d INT, INT, 1, false, false, false, false
-- !query
@@ -117,16 +90,7 @@ Project [spark_catalog.default.foo1a2(a#x, b#x, c#x, d#x) AS spark_catalog.defau
-- !query
CREATE FUNCTION foo1b0() RETURNS TABLE (c1 INT) RETURN SELECT 1
-- !query analysis
-org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException
-{
- "errorClass" : "ROUTINE_ALREADY_EXISTS",
- "sqlState" : "42723",
- "messageParameters" : {
- "existingRoutineType" : "routine",
- "newRoutineType" : "routine",
- "routineName" : "`default`.`foo1b0`"
- }
-}
+CreateSQLFunctionCommand spark_catalog.default.foo1b0, c1 INT, SELECT 1, true, false, false, false
-- !query
@@ -143,16 +107,7 @@ Project [c1#x]
-- !query
CREATE FUNCTION foo1b1(a INT) RETURNS TABLE (c1 INT) RETURN SELECT 1
-- !query analysis
-org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException
-{
- "errorClass" : "ROUTINE_ALREADY_EXISTS",
- "sqlState" : "42723",
- "messageParameters" : {
- "existingRoutineType" : "routine",
- "newRoutineType" : "routine",
- "routineName" : "`default`.`foo1b1`"
- }
-}
+CreateSQLFunctionCommand spark_catalog.default.foo1b1, a INT, c1 INT, SELECT 1, true, false, false, false
-- !query
@@ -169,16 +124,7 @@ Project [c1#x]
-- !query
CREATE FUNCTION foo1b2(a INT, b INT, c INT, d INT) RETURNS TABLE(c1 INT) RETURN SELECT 1
-- !query analysis
-org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException
-{
- "errorClass" : "ROUTINE_ALREADY_EXISTS",
- "sqlState" : "42723",
- "messageParameters" : {
- "existingRoutineType" : "routine",
- "newRoutineType" : "routine",
- "routineName" : "`default`.`foo1b2`"
- }
-}
+CreateSQLFunctionCommand spark_catalog.default.foo1b2, a INT, b INT, c INT, d INT, c1 INT, SELECT 1, true, false, false, false
-- !query
@@ -643,16 +589,7 @@ org.apache.spark.sql.catalyst.parser.ParseException
-- !query
CREATE FUNCTION foo2a2() RETURNS TABLE(c1 INT, c2 INT) RETURN SELECT 1, 2
-- !query analysis
-org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException
-{
- "errorClass" : "ROUTINE_ALREADY_EXISTS",
- "sqlState" : "42723",
- "messageParameters" : {
- "existingRoutineType" : "routine",
- "newRoutineType" : "routine",
- "routineName" : "`default`.`foo2a2`"
- }
-}
+CreateSQLFunctionCommand spark_catalog.default.foo2a2, c1 INT, c2 INT, SELECT 1, 2, true, false, false, false
-- !query
@@ -669,16 +606,7 @@ Project [c1#x, c2#x]
-- !query
CREATE FUNCTION foo2a4() RETURNS TABLE(c1 INT, c2 INT, c3 INT, c4 INT) RETURN SELECT 1, 2, 3, 4
-- !query analysis
-org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException
-{
- "errorClass" : "ROUTINE_ALREADY_EXISTS",
- "sqlState" : "42723",
- "messageParameters" : {
- "existingRoutineType" : "routine",
- "newRoutineType" : "routine",
- "routineName" : "`default`.`foo2a4`"
- }
-}
+CreateSQLFunctionCommand spark_catalog.default.foo2a4, c1 INT, c2 INT, c3 INT, c4 INT, SELECT 1, 2, 3, 4, true, false, false, false
-- !query
@@ -788,16 +716,7 @@ org.apache.spark.sql.AnalysisException
-- !query
CREATE FUNCTION foo41() RETURNS INT RETURN SELECT 1
-- !query analysis
-org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException
-{
- "errorClass" : "ROUTINE_ALREADY_EXISTS",
- "sqlState" : "42723",
- "messageParameters" : {
- "existingRoutineType" : "routine",
- "newRoutineType" : "routine",
- "routineName" : "`default`.`foo41`"
- }
-}
+CreateSQLFunctionCommand spark_catalog.default.foo41, INT, SELECT 1, false, false, false, false
-- !query
@@ -814,2791 +733,4640 @@ org.apache.spark.sql.AnalysisException
-- !query
-CREATE FUNCTION foo2_1a(a INT) RETURNS INT RETURN a
+CREATE FUNCTION foo51() RETURNS INT RETURN (SELECT a FROM VALUES(1), (2) AS T(a))
-- !query analysis
-org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException
-{
- "errorClass" : "ROUTINE_ALREADY_EXISTS",
- "sqlState" : "42723",
- "messageParameters" : {
- "existingRoutineType" : "routine",
- "newRoutineType" : "routine",
- "routineName" : "`default`.`foo2_1a`"
- }
-}
+CreateSQLFunctionCommand spark_catalog.default.foo51, INT, (SELECT a FROM VALUES(1), (2) AS T(a)), false, false, false, false
-- !query
-SELECT foo2_1a(5)
+SELECT foo51()
-- !query analysis
-Project [spark_catalog.default.foo2_1a(a#x) AS spark_catalog.default.foo2_1a(5)#x]
-+- Project [cast(5 as int) AS a#x]
+Project [spark_catalog.default.foo51() AS spark_catalog.default.foo51()#x]
+: +- Project [a#x]
+: +- SubqueryAlias T
+: +- LocalRelation [a#x]
++- Project
+- OneRowRelation
-- !query
-CREATE FUNCTION foo2_1b(a INT, b INT) RETURNS INT RETURN a + b
+CREATE FUNCTION foo52() RETURNS INT RETURN (SELECT 1 FROM VALUES(1) WHERE 1 = 0)
-- !query analysis
-org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException
-{
- "errorClass" : "ROUTINE_ALREADY_EXISTS",
- "sqlState" : "42723",
- "messageParameters" : {
- "existingRoutineType" : "routine",
- "newRoutineType" : "routine",
- "routineName" : "`default`.`foo2_1b`"
- }
-}
+CreateSQLFunctionCommand spark_catalog.default.foo52, INT, (SELECT 1 FROM VALUES(1) WHERE 1 = 0), false, false, false, false
-- !query
-SELECT foo2_1b(5, 6)
+SELECT foo52()
-- !query analysis
-Project [spark_catalog.default.foo2_1b(a#x, b#x) AS spark_catalog.default.foo2_1b(5, 6)#x]
-+- Project [cast(5 as int) AS a#x, cast(6 as int) AS b#x]
+Project [spark_catalog.default.foo52() AS spark_catalog.default.foo52()#x]
+: +- Project [1 AS 1#x]
+: +- Filter (1 = 0)
+: +- LocalRelation [col1#x]
++- Project
+- OneRowRelation
-- !query
-CREATE FUNCTION foo2_1c(a INT, b INT) RETURNS INT RETURN 10 * (a + b) + 100 * (a -b)
+CREATE FUNCTION foo6c(` a` INT, a INT, `a b` INT) RETURNS INT RETURN 1
-- !query analysis
-org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException
-{
- "errorClass" : "ROUTINE_ALREADY_EXISTS",
- "sqlState" : "42723",
- "messageParameters" : {
- "existingRoutineType" : "routine",
- "newRoutineType" : "routine",
- "routineName" : "`default`.`foo2_1c`"
- }
-}
+CreateSQLFunctionCommand spark_catalog.default.foo6c, ` a` INT, a INT, `a b` INT, INT, 1, false, false, false, false
-- !query
-SELECT foo2_1c(5, 6)
+SELECT foo6c(1, 2, 3)
-- !query analysis
-Project [spark_catalog.default.foo2_1c(a#x, b#x) AS spark_catalog.default.foo2_1c(5, 6)#x]
-+- Project [cast(5 as int) AS a#x, cast(6 as int) AS b#x]
+Project [spark_catalog.default.foo6c( a#x, a#x, a b#x) AS spark_catalog.default.foo6c(1, 2, 3)#x]
++- Project [cast(1 as int) AS a#x, cast(2 as int) AS a#x, cast(3 as int) AS a b#x]
+- OneRowRelation
-- !query
-CREATE FUNCTION foo2_1d(a INT, b INT) RETURNS INT RETURN ABS(a) - LENGTH(CAST(b AS VARCHAR(10)))
+CREATE FUNCTION foo6d() RETURNS TABLE(` a` INT, a INT, `a b` INT) RETURN SELECT 1, 2, 3
-- !query analysis
-org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException
-{
- "errorClass" : "ROUTINE_ALREADY_EXISTS",
- "sqlState" : "42723",
- "messageParameters" : {
- "existingRoutineType" : "routine",
- "newRoutineType" : "routine",
- "routineName" : "`default`.`foo2_1d`"
- }
-}
+CreateSQLFunctionCommand spark_catalog.default.foo6d, ` a` INT, a INT, `a b` INT, SELECT 1, 2, 3, true, false, false, false
-- !query
-SELECT foo2_1d(-5, 6)
+SELECT * FROM foo6d()
-- !query analysis
-Project [spark_catalog.default.foo2_1d(a#x, b#x) AS spark_catalog.default.foo2_1d(-5, 6)#x]
-+- Project [cast(-5 as int) AS a#x, cast(6 as int) AS b#x]
- +- OneRowRelation
+Project [ a#x, a#x, a b#x]
++- SQLFunctionNode spark_catalog.default.foo6d
+ +- SubqueryAlias foo6d
+ +- Project [cast(1#x as int) AS a#x, cast(2#x as int) AS a#x, cast(3#x as int) AS a b#x]
+ +- Project [1 AS 1#x, 2 AS 2#x, 3 AS 3#x]
+ +- OneRowRelation
-- !query
-CREATE FUNCTION foo2_2a(a INT) RETURNS INT RETURN SELECT a
+CREATE FUNCTION foo7a(a STRING, b STRING, c STRING) RETURNS STRING RETURN
+SELECT 'Foo.a: ' || a || ' Foo.a: ' || foo7a.a
+ || ' T.b: ' || b || ' Foo.b: ' || foo7a.b
+ || ' T.c: ' || c || ' T.c: ' || t.c FROM VALUES('t.b', 't.c') AS T(b, c)
-- !query analysis
-org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException
-{
- "errorClass" : "ROUTINE_ALREADY_EXISTS",
- "sqlState" : "42723",
- "messageParameters" : {
- "existingRoutineType" : "routine",
- "newRoutineType" : "routine",
- "routineName" : "`default`.`foo2_2a`"
- }
-}
+CreateSQLFunctionCommand spark_catalog.default.foo7a, a STRING, b STRING, c STRING, STRING, SELECT 'Foo.a: ' || a || ' Foo.a: ' || foo7a.a
+ || ' T.b: ' || b || ' Foo.b: ' || foo7a.b
+ || ' T.c: ' || c || ' T.c: ' || t.c FROM VALUES('t.b', 't.c') AS T(b, c), false, false, false, false
-- !query
-SELECT foo2_2a(5)
+SELECT foo7a('Foo.a', 'Foo.b', 'Foo.c')
-- !query analysis
-Project [spark_catalog.default.foo2_2a(a#x) AS spark_catalog.default.foo2_2a(5)#x]
-+- Project [cast(5 as int) AS a#x]
+Project [spark_catalog.default.foo7a(a#x, b#x, c#x) AS spark_catalog.default.foo7a(Foo.a, Foo.b, Foo.c)#x]
+: +- Project [concat(concat(concat(concat(concat(concat(concat(concat(concat(concat(concat(Foo.a: , outer(a#x)), Foo.a: ), outer(a#x)), T.b: ), b#x), Foo.b: ), outer(b#x)), T.c: ), c#x), T.c: ), c#x) AS concat(concat(concat(concat(concat(concat(concat(concat(concat(concat(concat(Foo.a: , outer(foo7a.a)), Foo.a: ), outer(foo7a.a)), T.b: ), b), Foo.b: ), outer(foo7a.b)), T.c: ), c), T.c: ), c)#x]
+: +- SubqueryAlias T
+: +- LocalRelation [b#x, c#x]
++- Project [cast(Foo.a as string) AS a#x, cast(Foo.b as string) AS b#x, cast(Foo.c as string) AS c#x]
+- OneRowRelation
-- !query
-CREATE FUNCTION foo2_2b(a INT) RETURNS INT RETURN 1 + (SELECT a)
+CREATE FUNCTION foo7at(a STRING, b STRING, c STRING) RETURNS TABLE (a STRING, b STRING, c STRING, d STRING, e STRING) RETURN
+SELECT CONCAT('Foo.a: ', a), CONCAT('Foo.b: ', foo7at.b), CONCAT('T.b: ', b),
+ CONCAT('Foo.c: ', foo7at.c), CONCAT('T.c: ', c)
+FROM VALUES ('t.b', 't.c') AS T(b, c)
-- !query analysis
-org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException
-{
- "errorClass" : "ROUTINE_ALREADY_EXISTS",
- "sqlState" : "42723",
- "messageParameters" : {
- "existingRoutineType" : "routine",
- "newRoutineType" : "routine",
- "routineName" : "`default`.`foo2_2b`"
- }
-}
+CreateSQLFunctionCommand spark_catalog.default.foo7at, a STRING, b STRING, c STRING, a STRING, b STRING, c STRING, d STRING, e STRING, SELECT CONCAT('Foo.a: ', a), CONCAT('Foo.b: ', foo7at.b), CONCAT('T.b: ', b),
+ CONCAT('Foo.c: ', foo7at.c), CONCAT('T.c: ', c)
+FROM VALUES ('t.b', 't.c') AS T(b, c), true, false, false, false
-- !query
-SELECT foo2_2b(5)
+SELECT * FROM foo7at('Foo.a', 'Foo.b', 'Foo.c')
-- !query analysis
-Project [spark_catalog.default.foo2_2b(a#x) AS spark_catalog.default.foo2_2b(5)#x]
-: +- Project [outer(a#x)]
-: +- OneRowRelation
-+- Project [cast(5 as int) AS a#x]
- +- OneRowRelation
+Project [a#x, b#x, c#x, d#x, e#x]
++- SQLFunctionNode spark_catalog.default.foo7at
+ +- SubqueryAlias foo7at
+ +- Project [cast(concat(Foo.a: , outer(foo7at.a))#x as string) AS a#x, cast(concat(Foo.b: , outer(foo7at.b))#x as string) AS b#x, cast(concat(T.b: , b)#x as string) AS c#x, cast(concat(Foo.c: , outer(foo7at.c))#x as string) AS d#x, cast(concat(T.c: , c)#x as string) AS e#x]
+ +- Project [concat(Foo.a: , cast(Foo.a as string)) AS concat(Foo.a: , outer(foo7at.a))#x, concat(Foo.b: , cast(Foo.b as string)) AS concat(Foo.b: , outer(foo7at.b))#x, concat(T.b: , b#x) AS concat(T.b: , b)#x, concat(Foo.c: , cast(Foo.c as string)) AS concat(Foo.c: , outer(foo7at.c))#x, concat(T.c: , c#x) AS concat(T.c: , c)#x]
+ +- SubqueryAlias T
+ +- LocalRelation [b#x, c#x]
-- !query
-CREATE FUNCTION foo2_2c(a INT) RETURNS INT RETURN 1 + (SELECT (SELECT a))
+CREATE FUNCTION foo9a(a BOOLEAN) RETURNS BOOLEAN RETURN NOT a
-- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION",
- "sqlState" : "42703",
- "messageParameters" : {
- "objectName" : "`a`"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 21,
- "stopIndex" : 21,
- "fragment" : "a"
- } ]
-}
+CreateSQLFunctionCommand spark_catalog.default.foo9a, a BOOLEAN, BOOLEAN, NOT a, false, false, false, false
-- !query
-CREATE FUNCTION foo2_2d(a INT) RETURNS INT RETURN 1 + (SELECT (SELECT (SELECT (SELECT a))))
+SELECT foo9a(true)
-- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION",
- "sqlState" : "42703",
- "messageParameters" : {
- "objectName" : "`a`"
- },
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 37,
- "stopIndex" : 37,
- "fragment" : "a"
- } ]
-}
+Project [spark_catalog.default.foo9a(a#x) AS spark_catalog.default.foo9a(true)#x]
++- Project [cast(true as boolean) AS a#x]
+ +- OneRowRelation
-- !query
-CREATE FUNCTION foo2_2e(a INT) RETURNS INT RETURN
-SELECT a FROM (VALUES 1) AS V(c1) WHERE c1 = 2
-UNION ALL
-SELECT a + 1 FROM (VALUES 1) AS V(c1)
+SELECT foo9a(5)
-- !query analysis
-org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException
-{
- "errorClass" : "ROUTINE_ALREADY_EXISTS",
- "sqlState" : "42723",
- "messageParameters" : {
- "existingRoutineType" : "routine",
- "newRoutineType" : "routine",
- "routineName" : "`default`.`foo2_2e`"
- }
-}
+Project [spark_catalog.default.foo9a(a#x) AS spark_catalog.default.foo9a(5)#x]
++- Project [cast(5 as boolean) AS a#x]
+ +- OneRowRelation
-- !query
-CREATE FUNCTION foo2_2f(a INT) RETURNS INT RETURN
-SELECT a FROM (VALUES 1) AS V(c1)
-EXCEPT
-SELECT a + 1 FROM (VALUES 1) AS V(a)
+SELECT foo9a('Nonsense')
-- !query analysis
-org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException
-{
- "errorClass" : "ROUTINE_ALREADY_EXISTS",
- "sqlState" : "42723",
- "messageParameters" : {
- "existingRoutineType" : "routine",
- "newRoutineType" : "routine",
- "routineName" : "`default`.`foo2_2f`"
- }
-}
+Project [spark_catalog.default.foo9a(a#x) AS spark_catalog.default.foo9a(Nonsense)#x]
++- Project [cast(Nonsense as boolean) AS a#x]
+ +- OneRowRelation
-- !query
-CREATE FUNCTION foo2_2g(a INT) RETURNS INT RETURN
-SELECT a FROM (VALUES 1) AS V(c1)
-INTERSECT
-SELECT a FROM (VALUES 1) AS V(a)
+CREATE FUNCTION foo9b(a BYTE) RETURNS BYTE RETURN CAST(a AS SHORT) + 1
-- !query analysis
-org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException
-{
- "errorClass" : "ROUTINE_ALREADY_EXISTS",
- "sqlState" : "42723",
- "messageParameters" : {
- "existingRoutineType" : "routine",
- "newRoutineType" : "routine",
- "routineName" : "`default`.`foo2_2g`"
- }
-}
+CreateSQLFunctionCommand spark_catalog.default.foo9b, a BYTE, BYTE, CAST(a AS SHORT) + 1, false, false, false, false
-- !query
-DROP TABLE IF EXISTS t1
+SELECT foo9b(126)
-- !query analysis
-DropTable true, false
-+- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t1
+Project [spark_catalog.default.foo9b(a#x) AS spark_catalog.default.foo9b(126)#x]
++- Project [cast(126 as tinyint) AS a#x]
+ +- OneRowRelation
-- !query
-DROP TABLE IF EXISTS t2
+SELECT foo9b(127)
-- !query analysis
-DropTable true, false
-+- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t2
+Project [spark_catalog.default.foo9b(a#x) AS spark_catalog.default.foo9b(127)#x]
++- Project [cast(127 as tinyint) AS a#x]
+ +- OneRowRelation
-- !query
-DROP TABLE IF EXISTS ts
+SELECT foo9b(128)
-- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "WRONG_COMMAND_FOR_OBJECT_TYPE",
- "sqlState" : "42809",
- "messageParameters" : {
- "alternative" : "DROP VIEW",
- "foundType" : "VIEW",
- "objectName" : "spark_catalog.default.ts",
- "operation" : "DROP TABLE",
- "requiredType" : "EXTERNAL or MANAGED"
- }
-}
+Project [spark_catalog.default.foo9b(a#x) AS spark_catalog.default.foo9b(128)#x]
++- Project [cast(128 as tinyint) AS a#x]
+ +- OneRowRelation
-- !query
-DROP TABLE IF EXISTS tm
+CREATE FUNCTION foo9c(a SHORT) RETURNS SHORT RETURN CAST(a AS INTEGER) + 1
-- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "WRONG_COMMAND_FOR_OBJECT_TYPE",
- "sqlState" : "42809",
- "messageParameters" : {
- "alternative" : "DROP VIEW",
- "foundType" : "VIEW",
- "objectName" : "spark_catalog.default.tm",
- "operation" : "DROP TABLE",
- "requiredType" : "EXTERNAL or MANAGED"
- }
-}
+CreateSQLFunctionCommand spark_catalog.default.foo9c, a SHORT, SHORT, CAST(a AS INTEGER) + 1, false, false, false, false
-- !query
-DROP TABLE IF EXISTS ta
+SELECT foo9c(32766)
-- !query analysis
-org.apache.spark.sql.AnalysisException
-{
- "errorClass" : "WRONG_COMMAND_FOR_OBJECT_TYPE",
- "sqlState" : "42809",
- "messageParameters" : {
- "alternative" : "DROP VIEW",
- "foundType" : "VIEW",
- "objectName" : "spark_catalog.default.ta",
- "operation" : "DROP TABLE",
- "requiredType" : "EXTERNAL or MANAGED"
- }
-}
+Project [spark_catalog.default.foo9c(a#x) AS spark_catalog.default.foo9c(32766)#x]
++- Project [cast(32766 as smallint) AS a#x]
+ +- OneRowRelation
-- !query
-DROP TABLE IF EXISTS V1
+SELECT foo9c(32767)
-- !query analysis
-DropTable true, false
-+- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.V1
+Project [spark_catalog.default.foo9c(a#x) AS spark_catalog.default.foo9c(32767)#x]
++- Project [cast(32767 as smallint) AS a#x]
+ +- OneRowRelation
-- !query
-DROP TABLE IF EXISTS V2
+SELECT foo9c(32768)
-- !query analysis
-DropTable true, false
-+- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.V2
+Project [spark_catalog.default.foo9c(a#x) AS spark_catalog.default.foo9c(32768)#x]
++- Project [cast(32768 as smallint) AS a#x]
+ +- OneRowRelation
-- !query
-DROP VIEW IF EXISTS t1
+CREATE FUNCTION foo9d(a INTEGER) RETURNS INTEGER RETURN CAST(a AS BIGINT) + 1
-- !query analysis
-DropTableCommand `spark_catalog`.`default`.`t1`, true, true, false
+CreateSQLFunctionCommand spark_catalog.default.foo9d, a INTEGER, INTEGER, CAST(a AS BIGINT) + 1, false, false, false, false
-- !query
-DROP VIEW IF EXISTS t2
+SELECT foo9d(2147483647 - 1)
-- !query analysis
-DropTableCommand `spark_catalog`.`default`.`t2`, true, true, false
+Project [spark_catalog.default.foo9d(a#x) AS spark_catalog.default.foo9d((2147483647 - 1))#x]
++- Project [cast((2147483647 - 1) as int) AS a#x]
+ +- OneRowRelation
-- !query
-DROP VIEW IF EXISTS ts
+SELECT foo9d(2147483647)
-- !query analysis
-DropTableCommand `spark_catalog`.`default`.`ts`, true, true, false
+Project [spark_catalog.default.foo9d(a#x) AS spark_catalog.default.foo9d(2147483647)#x]
++- Project [cast(2147483647 as int) AS a#x]
+ +- OneRowRelation
-- !query
-DROP VIEW IF EXISTS tm
+SELECT foo9d(2147483647 + 1)
-- !query analysis
-DropTableCommand `spark_catalog`.`default`.`tm`, true, true, false
+Project [spark_catalog.default.foo9d(a#x) AS spark_catalog.default.foo9d((2147483647 + 1))#x]
++- Project [cast((2147483647 + 1) as int) AS a#x]
+ +- OneRowRelation
-- !query
-DROP VIEW IF EXISTS ta
+CREATE FUNCTION foo9e(a BIGINT) RETURNS BIGINT RETURN CAST(a AS DECIMAL(20, 0)) + 1
-- !query analysis
-DropTableCommand `spark_catalog`.`default`.`ta`, true, true, false
+CreateSQLFunctionCommand spark_catalog.default.foo9e, a BIGINT, BIGINT, CAST(a AS DECIMAL(20, 0)) + 1, false, false, false, false
-- !query
-DROP VIEW IF EXISTS V1
+SELECT foo9e(9223372036854775807 - 1)
-- !query analysis
-DropTableCommand `spark_catalog`.`default`.`V1`, true, true, false
+Project [spark_catalog.default.foo9e(a#xL) AS spark_catalog.default.foo9e((9223372036854775807 - 1))#xL]
++- Project [cast((9223372036854775807 - cast(1 as bigint)) as bigint) AS a#xL]
+ +- OneRowRelation
-- !query
-DROP VIEW IF EXISTS V2
+SELECT foo9e(9223372036854775807)
-- !query analysis
-DropTableCommand `spark_catalog`.`default`.`V2`, true, true, false
+Project [spark_catalog.default.foo9e(a#xL) AS spark_catalog.default.foo9e(9223372036854775807)#xL]
++- Project [cast(9223372036854775807 as bigint) AS a#xL]
+ +- OneRowRelation
-- !query
-CREATE FUNCTION foo2_3(a INT, b INT) RETURNS INT RETURN a + b
+SELECT foo9e(9223372036854775807.0 + 1)
-- !query analysis
-org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException
-{
- "errorClass" : "ROUTINE_ALREADY_EXISTS",
- "sqlState" : "42723",
- "messageParameters" : {
- "existingRoutineType" : "routine",
- "newRoutineType" : "routine",
- "routineName" : "`default`.`foo2_3`"
- }
-}
+Project [spark_catalog.default.foo9e(a#xL) AS spark_catalog.default.foo9e((9223372036854775807.0 + 1))#xL]
++- Project [cast((9223372036854775807.0 + cast(1 as decimal(1,0))) as bigint) AS a#xL]
+ +- OneRowRelation
-- !query
-CREATE VIEW V1(c1, c2) AS VALUES (1, 2), (3, 4), (5, 6)
+CREATE FUNCTION foo9f(a DECIMAL( 5, 2 )) RETURNS DECIMAL (5, 2) RETURN CAST(a AS DECIMAL(6, 2)) + 1
-- !query analysis
-CreateViewCommand `spark_catalog`.`default`.`V1`, [(c1,None), (c2,None)], VALUES (1, 2), (3, 4), (5, 6), false, false, PersistedView, COMPENSATION, true
- +- LocalRelation [col1#x, col2#x]
+CreateSQLFunctionCommand spark_catalog.default.foo9f, a DECIMAL( 5, 2 ), DECIMAL (5, 2), CAST(a AS DECIMAL(6, 2)) + 1, false, false, false, false
-- !query
-CREATE VIEW V2(c1, c2) AS VALUES (-1, -2), (-3, -4), (-5, -6)
+SELECT foo9f(999 - 1)
-- !query analysis
-CreateViewCommand `spark_catalog`.`default`.`V2`, [(c1,None), (c2,None)], VALUES (-1, -2), (-3, -4), (-5, -6), false, false, PersistedView, COMPENSATION, true
- +- LocalRelation [col1#x, col2#x]
+Project [spark_catalog.default.foo9f(a#x) AS spark_catalog.default.foo9f((999 - 1))#x]
++- Project [cast((999 - 1) as decimal(5,2)) AS a#x]
+ +- OneRowRelation
-- !query
-SELECT foo2_3(c1, c2), foo2_3(c2, 1), foo2_3(c1, c2) - foo2_3(c2, c1 - 1) FROM V1 ORDER BY 1, 2, 3
+SELECT foo9f(999)
-- !query analysis
-Sort [spark_catalog.default.foo2_3(c1, c2)#x ASC NULLS FIRST, spark_catalog.default.foo2_3(c2, 1)#x ASC NULLS FIRST, (spark_catalog.default.foo2_3(c1, c2) - spark_catalog.default.foo2_3(c2, (c1 - 1)))#x ASC NULLS FIRST], true
-+- Project [spark_catalog.default.foo2_3(a#x, b#x) AS spark_catalog.default.foo2_3(c1, c2)#x, spark_catalog.default.foo2_3(a#x, b#x) AS spark_catalog.default.foo2_3(c2, 1)#x, (spark_catalog.default.foo2_3(a#x, b#x) - spark_catalog.default.foo2_3(a#x, b#x)) AS (spark_catalog.default.foo2_3(c1, c2) - spark_catalog.default.foo2_3(c2, (c1 - 1)))#x]
- +- Project [c1#x, c2#x, cast(c1#x as int) AS a#x, cast(c2#x as int) AS b#x, cast(c2#x as int) AS a#x, cast(1 as int) AS b#x, cast(c1#x as int) AS a#x, cast(c2#x as int) AS b#x, cast(c2#x as int) AS a#x, cast((c1#x - 1) as int) AS b#x]
- +- SubqueryAlias spark_catalog.default.v1
- +- View (`spark_catalog`.`default`.`v1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+Project [spark_catalog.default.foo9f(a#x) AS spark_catalog.default.foo9f(999)#x]
++- Project [cast(999 as decimal(5,2)) AS a#x]
+ +- OneRowRelation
-- !query
-SELECT * FROM V1 WHERE foo2_3(c1, 0) = c1 AND foo2_3(c1, c2) < 8
+SELECT foo9f(999 + 1)
-- !query analysis
-Project [c1#x, c2#x]
-+- Project [c1#x, c2#x]
- +- Filter ((spark_catalog.default.foo2_3(a#x, b#x) = c1#x) AND (spark_catalog.default.foo2_3(a#x, b#x) < 8))
- +- Project [c1#x, c2#x, cast(c1#x as int) AS a#x, cast(0 as int) AS b#x, cast(c1#x as int) AS a#x, cast(c2#x as int) AS b#x]
- +- SubqueryAlias spark_catalog.default.v1
- +- View (`spark_catalog`.`default`.`v1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+Project [spark_catalog.default.foo9f(a#x) AS spark_catalog.default.foo9f((999 + 1))#x]
++- Project [cast((999 + 1) as decimal(5,2)) AS a#x]
+ +- OneRowRelation
-- !query
-SELECT foo2_3(SUM(c1), SUM(c2)), SUM(c1) + SUM(c2), SUM(foo2_3(c1, c2) + foo2_3(c2, c1) - foo2_3(c2, c1))
-FROM V1
+CREATE FUNCTION foo9g(a FLOAT, b String) RETURNS FLOAT RETURN b || CAST(a AS String)
-- !query analysis
-Project [spark_catalog.default.foo2_3(a#x, b#x) AS spark_catalog.default.foo2_3(sum(c1), sum(c2))#x, (sum(c1) + sum(c2))#xL, sum(((spark_catalog.default.foo2_3(c1, c2) + spark_catalog.default.foo2_3(c2, c1)) - spark_catalog.default.foo2_3(c2, c1)))#xL]
-+- Project [sum(c1)#xL, sum(c2)#xL, (sum(c1) + sum(c2))#xL, sum(((spark_catalog.default.foo2_3(c1, c2) + spark_catalog.default.foo2_3(c2, c1)) - spark_catalog.default.foo2_3(c2, c1)))#xL, cast(sum(c1)#xL as int) AS a#x, cast(sum(c2)#xL as int) AS b#x]
- +- Aggregate [sum(c1#x) AS sum(c1)#xL, sum(c2#x) AS sum(c2)#xL, (sum(c1#x) + sum(c2#x)) AS (sum(c1) + sum(c2))#xL, sum(((spark_catalog.default.foo2_3(a#x, b#x) + spark_catalog.default.foo2_3(a#x, b#x)) - spark_catalog.default.foo2_3(a#x, b#x))) AS sum(((spark_catalog.default.foo2_3(c1, c2) + spark_catalog.default.foo2_3(c2, c1)) - spark_catalog.default.foo2_3(c2, c1)))#xL]
- +- Project [c1#x, c2#x, cast(c1#x as int) AS a#x, cast(c2#x as int) AS b#x, cast(c2#x as int) AS a#x, cast(c1#x as int) AS b#x, cast(c2#x as int) AS a#x, cast(c1#x as int) AS b#x]
- +- SubqueryAlias spark_catalog.default.v1
- +- View (`spark_catalog`.`default`.`v1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+CreateSQLFunctionCommand spark_catalog.default.foo9g, a FLOAT, b String, FLOAT, b || CAST(a AS String), false, false, false, false
-- !query
-CREATE FUNCTION foo2_4a(a ARRAY) RETURNS STRING RETURN
-SELECT array_sort(a, (i, j) -> rank[i] - rank[j])[0] FROM (SELECT MAP('a', 1, 'b', 2) rank)
+SELECT foo9g(123.23, '7')
-- !query analysis
-org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException
-{
- "errorClass" : "ROUTINE_ALREADY_EXISTS",
- "sqlState" : "42723",
- "messageParameters" : {
- "existingRoutineType" : "routine",
- "newRoutineType" : "routine",
- "routineName" : "`default`.`foo2_4a`"
- }
-}
+Project [spark_catalog.default.foo9g(a#x, b#x) AS spark_catalog.default.foo9g(123.23, 7)#x]
++- Project [cast(123.23 as float) AS a#x, cast(7 as string) AS b#x]
+ +- OneRowRelation
-- !query
-SELECT foo2_4a(ARRAY('a', 'b'))
+SELECT foo9g('hello', '7')
-- !query analysis
-Project [spark_catalog.default.foo2_4a(a#x) AS spark_catalog.default.foo2_4a(array(a, b))#x]
-: +- Project [array_sort(outer(a#x), lambdafunction((rank#x[lambda i#x] - rank#x[lambda j#x]), lambda i#x, lambda j#x, false), false)[0] AS array_sort(outer(foo2_4a.a), lambdafunction((rank[namedlambdavariable()] - rank[namedlambdavariable()]), namedlambdavariable(), namedlambdavariable()))[0]#x]
-: +- SubqueryAlias __auto_generated_subquery_name
-: +- Project [map(a, 1, b, 2) AS rank#x]
-: +- OneRowRelation
-+- Project [cast(array(a, b) as array) AS a#x]
+Project [spark_catalog.default.foo9g(a#x, b#x) AS spark_catalog.default.foo9g(hello, 7)#x]
++- Project [cast(hello as float) AS a#x, cast(7 as string) AS b#x]
+- OneRowRelation
-- !query
-CREATE FUNCTION foo2_4b(m MAP, k STRING) RETURNS STRING RETURN
-SELECT v || ' ' || v FROM (SELECT upper(m[k]) AS v)
+SELECT foo9g(123.23, 'q')
-- !query analysis
-org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException
-{
- "errorClass" : "ROUTINE_ALREADY_EXISTS",
- "sqlState" : "42723",
- "messageParameters" : {
- "existingRoutineType" : "routine",
- "newRoutineType" : "routine",
- "routineName" : "`default`.`foo2_4b`"
- }
-}
+Project [spark_catalog.default.foo9g(a#x, b#x) AS spark_catalog.default.foo9g(123.23, q)#x]
++- Project [cast(123.23 as float) AS a#x, cast(q as string) AS b#x]
+ +- OneRowRelation
-- !query
-SELECT foo2_4b(map('a', 'hello', 'b', 'world'), 'a')
+CREATE FUNCTION foo9h(a DOUBLE, b String) RETURNS DOUBLE RETURN b || CAST(a AS String)
-- !query analysis
-Project [spark_catalog.default.foo2_4b(m#x, k#x) AS spark_catalog.default.foo2_4b(map(a, hello, b, world), a)#x]
-: +- Project [concat(concat(v#x, ), v#x) AS concat(concat(v, ), v)#x]
-: +- SubqueryAlias __auto_generated_subquery_name
-: +- Project [upper(outer(m#x)[outer(k#x)]) AS v#x]
-: +- OneRowRelation
-+- Project [cast(map(a, hello, b, world) as map) AS m#x, cast(a as string) AS k#x]
+CreateSQLFunctionCommand spark_catalog.default.foo9h, a DOUBLE, b String, DOUBLE, b || CAST(a AS String), false, false, false, false
+
+
+-- !query
+SELECT foo9h(123.23, '7')
+-- !query analysis
+Project [spark_catalog.default.foo9h(a#x, b#x) AS spark_catalog.default.foo9h(123.23, 7)#x]
++- Project [cast(123.23 as double) AS a#x, cast(7 as string) AS b#x]
+- OneRowRelation
-- !query
-DROP VIEW V2
+SELECT foo9h('hello', '7')
-- !query analysis
-DropTableCommand `spark_catalog`.`default`.`V2`, false, true, false
+Project [spark_catalog.default.foo9h(a#x, b#x) AS spark_catalog.default.foo9h(hello, 7)#x]
++- Project [cast(hello as double) AS a#x, cast(7 as string) AS b#x]
+ +- OneRowRelation
-- !query
-DROP VIEW V1
+SELECT foo9h(123.23, 'q')
-- !query analysis
-DropTableCommand `spark_catalog`.`default`.`V1`, false, true, false
+Project [spark_catalog.default.foo9h(a#x, b#x) AS spark_catalog.default.foo9h(123.23, q)#x]
++- Project [cast(123.23 as double) AS a#x, cast(q as string) AS b#x]
+ +- OneRowRelation
-- !query
-CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (0, 2), (1, 2)
+CREATE FUNCTION foo9i(a VARCHAR(10), b VARCHAR(10)) RETURNS VARCHAR(12) RETURN a || b
-- !query analysis
-CreateViewCommand `spark_catalog`.`default`.`t1`, [(c1,None), (c2,None)], VALUES (0, 1), (0, 2), (1, 2), false, false, PersistedView, COMPENSATION, true
- +- LocalRelation [col1#x, col2#x]
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING",
+ "sqlState" : "0A000"
+}
-- !query
-CREATE VIEW t2(c1, c2) AS VALUES (0, 2), (0, 3)
+CREATE FUNCTION foo9j(a STRING, b STRING) RETURNS STRING RETURN a || b
-- !query analysis
-CreateViewCommand `spark_catalog`.`default`.`t2`, [(c1,None), (c2,None)], VALUES (0, 2), (0, 3), false, false, PersistedView, COMPENSATION, true
- +- LocalRelation [col1#x, col2#x]
+CreateSQLFunctionCommand spark_catalog.default.foo9j, a STRING, b STRING, STRING, a || b, false, false, false, false
-- !query
-CREATE VIEW ts(x) AS VALUES NAMED_STRUCT('a', 1, 'b', 2)
+SELECT foo9j('1234567890', '12')
-- !query analysis
-CreateViewCommand `spark_catalog`.`default`.`ts`, [(x,None)], VALUES NAMED_STRUCT('a', 1, 'b', 2), false, false, PersistedView, COMPENSATION, true
- +- LocalRelation [col1#x]
+Project [spark_catalog.default.foo9j(a#x, b#x) AS spark_catalog.default.foo9j(1234567890, 12)#x]
++- Project [cast(1234567890 as string) AS a#x, cast(12 as string) AS b#x]
+ +- OneRowRelation
-- !query
-CREATE VIEW tm(x) AS VALUES MAP('a', 1, 'b', 2)
+SELECT foo9j(12345678901, '12')
-- !query analysis
-CreateViewCommand `spark_catalog`.`default`.`tm`, [(x,None)], VALUES MAP('a', 1, 'b', 2), false, false, PersistedView, COMPENSATION, true
- +- LocalRelation [col1#x]
+Project [spark_catalog.default.foo9j(a#x, b#x) AS spark_catalog.default.foo9j(12345678901, 12)#x]
++- Project [cast(12345678901 as string) AS a#x, cast(12 as string) AS b#x]
+ +- OneRowRelation
-- !query
-CREATE VIEW ta(x) AS VALUES ARRAY(1, 2, 3)
+CREATE FUNCTION foo9l(a DATE, b INTERVAL) RETURNS DATE RETURN a + b
-- !query analysis
-CreateViewCommand `spark_catalog`.`default`.`ta`, [(x,None)], VALUES ARRAY(1, 2, 3), false, false, PersistedView, COMPENSATION, true
- +- LocalRelation [col1#x]
+CreateSQLFunctionCommand spark_catalog.default.foo9l, a DATE, b INTERVAL, DATE, a + b, false, false, false, false
-- !query
-CREATE FUNCTION foo3_1a(a DOUBLE, b DOUBLE) RETURNS DOUBLE RETURN a * b
+SELECT foo9l(DATE '2020-02-02', INTERVAL '1' YEAR)
-- !query analysis
-org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
{
- "errorClass" : "ROUTINE_ALREADY_EXISTS",
- "sqlState" : "42723",
+ "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION",
+ "sqlState" : "42K09",
"messageParameters" : {
- "existingRoutineType" : "routine",
- "newRoutineType" : "routine",
- "routineName" : "`default`.`foo3_1a`"
- }
+ "sqlExpr" : "\"INTERVAL '1' YEAR\"",
+ "srcType" : "\"INTERVAL YEAR\"",
+ "targetType" : "\"INTERVAL\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 50,
+ "fragment" : "foo9l(DATE '2020-02-02', INTERVAL '1' YEAR)"
+ } ]
}
-- !query
-CREATE FUNCTION foo3_1b(x INT) RETURNS INT RETURN x
+SELECT foo9l('2020-02-02', INTERVAL '1' YEAR)
-- !query analysis
-org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
{
- "errorClass" : "ROUTINE_ALREADY_EXISTS",
- "sqlState" : "42723",
+ "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION",
+ "sqlState" : "42K09",
"messageParameters" : {
- "existingRoutineType" : "routine",
- "newRoutineType" : "routine",
- "routineName" : "`default`.`foo3_1b`"
- }
+ "sqlExpr" : "\"INTERVAL '1' YEAR\"",
+ "srcType" : "\"INTERVAL YEAR\"",
+ "targetType" : "\"INTERVAL\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 45,
+ "fragment" : "foo9l('2020-02-02', INTERVAL '1' YEAR)"
+ } ]
}
-- !query
-CREATE FUNCTION foo3_1c(x INT) RETURNS INT RETURN SELECT x
+SELECT foo9l(DATE '-7', INTERVAL '1' YEAR)
-- !query analysis
-org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException
+org.apache.spark.sql.catalyst.parser.ParseException
{
- "errorClass" : "ROUTINE_ALREADY_EXISTS",
- "sqlState" : "42723",
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
"messageParameters" : {
- "existingRoutineType" : "routine",
- "newRoutineType" : "routine",
- "routineName" : "`default`.`foo3_1c`"
- }
+ "value" : "'-7'",
+ "valueType" : "\"DATE\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 14,
+ "stopIndex" : 22,
+ "fragment" : "DATE '-7'"
+ } ]
}
-- !query
-CREATE FUNCTION foo3_1d(x INT) RETURNS INT RETURN (SELECT SUM(c2) FROM t2 WHERE c1 = x)
+SELECT foo9l(DATE '2020-02-02', INTERVAL '9999999' YEAR)
-- !query analysis
-org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
{
- "errorClass" : "ROUTINE_ALREADY_EXISTS",
- "sqlState" : "42723",
+ "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION",
+ "sqlState" : "42K09",
"messageParameters" : {
- "existingRoutineType" : "routine",
- "newRoutineType" : "routine",
- "routineName" : "`default`.`foo3_1d`"
- }
+ "sqlExpr" : "\"INTERVAL '9999999' YEAR\"",
+ "srcType" : "\"INTERVAL YEAR\"",
+ "targetType" : "\"INTERVAL\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 56,
+ "fragment" : "foo9l(DATE '2020-02-02', INTERVAL '9999999' YEAR)"
+ } ]
}
-- !query
-CREATE FUNCTION foo3_1e() RETURNS INT RETURN foo3_1d(0)
+CREATE FUNCTION foo9m(a TIMESTAMP, b INTERVAL) RETURNS TIMESTAMP RETURN a + b
+-- !query analysis
+CreateSQLFunctionCommand spark_catalog.default.foo9m, a TIMESTAMP, b INTERVAL, TIMESTAMP, a + b, false, false, false, false
+
+
+-- !query
+SELECT foo9m(TIMESTAMP'2020-02-02 12:15:16.123', INTERVAL '1' YEAR)
-- !query analysis
-org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
{
- "errorClass" : "ROUTINE_ALREADY_EXISTS",
- "sqlState" : "42723",
+ "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION",
+ "sqlState" : "42K09",
"messageParameters" : {
- "existingRoutineType" : "routine",
- "newRoutineType" : "routine",
- "routineName" : "`default`.`foo3_1e`"
- }
+ "sqlExpr" : "\"INTERVAL '1' YEAR\"",
+ "srcType" : "\"INTERVAL YEAR\"",
+ "targetType" : "\"INTERVAL\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 67,
+ "fragment" : "foo9m(TIMESTAMP'2020-02-02 12:15:16.123', INTERVAL '1' YEAR)"
+ } ]
}
-- !query
-CREATE FUNCTION foo3_1f() RETURNS INT RETURN SELECT SUM(c2) FROM t2 WHERE c1 = 0
+SELECT foo9m('2020-02-02 12:15:16.123', INTERVAL '1' YEAR)
-- !query analysis
-org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
{
- "errorClass" : "ROUTINE_ALREADY_EXISTS",
- "sqlState" : "42723",
+ "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION",
+ "sqlState" : "42K09",
"messageParameters" : {
- "existingRoutineType" : "routine",
- "newRoutineType" : "routine",
- "routineName" : "`default`.`foo3_1f`"
- }
+ "sqlExpr" : "\"INTERVAL '1' YEAR\"",
+ "srcType" : "\"INTERVAL YEAR\"",
+ "targetType" : "\"INTERVAL\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 58,
+ "fragment" : "foo9m('2020-02-02 12:15:16.123', INTERVAL '1' YEAR)"
+ } ]
}
-- !query
-CREATE FUNCTION foo3_1g(x INT) RETURNS INT RETURN SELECT (SELECT x)
+SELECT foo9m(TIMESTAMP'2020-02-02 12:15:16.123', INTERVAL '999999' YEAR)
-- !query analysis
-org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
{
- "errorClass" : "ROUTINE_ALREADY_EXISTS",
- "sqlState" : "42723",
+ "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION",
+ "sqlState" : "42K09",
"messageParameters" : {
- "existingRoutineType" : "routine",
- "newRoutineType" : "routine",
- "routineName" : "`default`.`foo3_1g`"
- }
+ "sqlExpr" : "\"INTERVAL '999999' YEAR\"",
+ "srcType" : "\"INTERVAL YEAR\"",
+ "targetType" : "\"INTERVAL\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 72,
+ "fragment" : "foo9m(TIMESTAMP'2020-02-02 12:15:16.123', INTERVAL '999999' YEAR)"
+ } ]
}
-- !query
-SELECT a, b, foo3_1a(a + 1, b + 1) FROM t1 AS t(a, b)
+CREATE FUNCTION foo9n(a ARRAY) RETURNS ARRAY RETURN a
-- !query analysis
-Project [a#x, b#x, spark_catalog.default.foo3_1a(a#x, b#x) AS spark_catalog.default.foo3_1a((a + 1), (b + 1))#x]
-+- Project [a#x, b#x, cast((a#x + 1) as double) AS a#x, cast((b#x + 1) as double) AS b#x]
- +- SubqueryAlias t
- +- Project [c1#x AS a#x, c2#x AS b#x]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+CreateSQLFunctionCommand spark_catalog.default.foo9n, a ARRAY, ARRAY, a, false, false, false, false
-- !query
-SELECT x, foo3_1c(x) FROM t1 AS t(x, y)
+SELECT foo9n(ARRAY(1, 2, 3))
-- !query analysis
-Project [x#x, spark_catalog.default.foo3_1c(x#x) AS spark_catalog.default.foo3_1c(x)#x]
-+- Project [x#x, y#x, cast(x#x as int) AS x#x]
- +- SubqueryAlias t
- +- Project [c1#x AS x#x, c2#x AS y#x]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+Project [spark_catalog.default.foo9n(a#x) AS spark_catalog.default.foo9n(array(1, 2, 3))#x]
++- Project [cast(array(1, 2, 3) as array) AS a#x]
+ +- OneRowRelation
-- !query
-SELECT c1, foo3_1d(c1) FROM t1
+SELECT foo9n(from_json('[1, 2, 3]', 'array'))
-- !query analysis
-Project [c1#x, spark_catalog.default.foo3_1d(x#x) AS spark_catalog.default.foo3_1d(c1)#x]
-: +- Aggregate [sum(c2#x) AS sum(c2)#xL]
-: +- Filter (c1#x = outer(x#x))
-: +- SubqueryAlias spark_catalog.default.t2
-: +- View (`spark_catalog`.`default`.`t2`, [c1#x, c2#x])
-: +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
-: +- LocalRelation [col1#x, col2#x]
-+- Project [c1#x, c2#x, cast(c1#x as int) AS x#x]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+Project [spark_catalog.default.foo9n(a#x) AS spark_catalog.default.foo9n(from_json([1, 2, 3]))#x]
++- Project [cast(from_json(ArrayType(IntegerType,true), [1, 2, 3], Some(America/Los_Angeles), false) as array) AS a#x]
+ +- OneRowRelation
-- !query
-SELECT c1, foo3_1a(foo3_1b(c1), foo3_1b(c1)) FROM t1
+CREATE FUNCTION foo9o(a MAP) RETURNS MAP RETURN a
-- !query analysis
-Project [c1#x, spark_catalog.default.foo3_1a(a#x, b#x) AS spark_catalog.default.foo3_1a(spark_catalog.default.foo3_1b(c1), spark_catalog.default.foo3_1b(c1))#x]
-+- Project [c1#x, c2#x, x#x, x#x, cast(spark_catalog.default.foo3_1b(x#x) as double) AS a#x, cast(spark_catalog.default.foo3_1b(x#x) as double) AS b#x]
- +- Project [c1#x, c2#x, cast(c1#x as int) AS x#x, cast(c1#x as int) AS x#x]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+CreateSQLFunctionCommand spark_catalog.default.foo9o, a MAP, MAP, a, false, false, false, false
-- !query
-SELECT c1, foo3_1d(foo3_1c(foo3_1b(c1))) FROM t1
+SELECT foo9o(MAP('hello', 1, 'world', 2))
-- !query analysis
-Project [c1#x, spark_catalog.default.foo3_1d(x#x) AS spark_catalog.default.foo3_1d(spark_catalog.default.foo3_1c(spark_catalog.default.foo3_1b(c1)))#x]
-: +- Aggregate [sum(c2#x) AS sum(c2)#xL]
-: +- Filter (c1#x = outer(x#x))
-: +- SubqueryAlias spark_catalog.default.t2
-: +- View (`spark_catalog`.`default`.`t2`, [c1#x, c2#x])
-: +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
-: +- LocalRelation [col1#x, col2#x]
-+- Project [c1#x, c2#x, x#x, x#x, cast(spark_catalog.default.foo3_1c(x#x) as int) AS x#x]
- +- Project [c1#x, c2#x, x#x, cast(spark_catalog.default.foo3_1b(x#x) as int) AS x#x]
- +- Project [c1#x, c2#x, cast(c1#x as int) AS x#x]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+Project [spark_catalog.default.foo9o(a#x) AS spark_catalog.default.foo9o(map(hello, 1, world, 2))#x]
++- Project [cast(map(hello, 1, world, 2) as map) AS a#x]
+ +- OneRowRelation
-- !query
-SELECT c1, foo3_1a(foo3_1c(foo3_1b(c1)), foo3_1d(foo3_1b(c1))) FROM t1
+SELECT foo9o(from_json('{"hello":1, "world":2}', 'map'))
-- !query analysis
-Project [c1#x, spark_catalog.default.foo3_1a(a#x, b#x) AS spark_catalog.default.foo3_1a(spark_catalog.default.foo3_1c(spark_catalog.default.foo3_1b(c1)), spark_catalog.default.foo3_1d(spark_catalog.default.foo3_1b(c1)))#x]
-+- Project [c1#x, c2#x, x#x, x#x, x#x, x#x, cast(spark_catalog.default.foo3_1c(x#x) as double) AS a#x, cast(spark_catalog.default.foo3_1d(x#x) as double) AS b#x]
- : +- Aggregate [sum(c2#x) AS sum(c2)#xL]
- : +- Filter (c1#x = outer(x#x))
- : +- SubqueryAlias spark_catalog.default.t2
- : +- View (`spark_catalog`.`default`.`t2`, [c1#x, c2#x])
- : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- : +- LocalRelation [col1#x, col2#x]
- +- Project [c1#x, c2#x, x#x, x#x, cast(spark_catalog.default.foo3_1b(x#x) as int) AS x#x, cast(spark_catalog.default.foo3_1b(x#x) as int) AS x#x]
- +- Project [c1#x, c2#x, cast(c1#x as int) AS x#x, cast(c1#x as int) AS x#x]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+Project [spark_catalog.default.foo9o(a#x) AS spark_catalog.default.foo9o(entries)#x]
++- Project [cast(from_json(MapType(StringType,IntegerType,true), {"hello":1, "world":2}, Some(America/Los_Angeles), false) as map) AS a#x]
+ +- OneRowRelation
-- !query
-SELECT foo3_1c(foo3_1e()) FROM t1
+CREATE FUNCTION foo9p(a STRUCT) RETURNS STRUCT RETURN a
-- !query analysis
-Project [spark_catalog.default.foo3_1c(x#x) AS spark_catalog.default.foo3_1c(spark_catalog.default.foo3_1e())#x]
-+- Project [c1#x, c2#x, x#x, cast(spark_catalog.default.foo3_1e() as int) AS x#x]
- : +- Aggregate [sum(c2#x) AS sum(c2)#xL]
- : +- Filter (c1#x = outer(x#x))
- : +- SubqueryAlias spark_catalog.default.t2
- : +- View (`spark_catalog`.`default`.`t2`, [c1#x, c2#x])
- : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- : +- LocalRelation [col1#x, col2#x]
- +- Project [c1#x, c2#x, cast(0 as int) AS x#x]
- +- Project [c1#x, c2#x]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+CreateSQLFunctionCommand spark_catalog.default.foo9p, a STRUCT, STRUCT, a, false, false, false, false
-- !query
-SELECT foo3_1a(MAX(c1), MAX(c2)) FROM t1
+SELECT foo9p(STRUCT(1, 'hello'))
-- !query analysis
-Project [spark_catalog.default.foo3_1a(a#x, b#x) AS spark_catalog.default.foo3_1a(max(c1), max(c2))#x]
-+- Project [max(c1)#x, max(c2)#x, cast(max(c1)#x as double) AS a#x, cast(max(c2)#x as double) AS b#x]
- +- Aggregate [max(c1#x) AS max(c1)#x, max(c2#x) AS max(c2)#x]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+Project [spark_catalog.default.foo9p(a#x) AS spark_catalog.default.foo9p(struct(1, hello))#x]
++- Project [cast(struct(col1, 1, col2, hello) as struct) AS a#x]
+ +- OneRowRelation
-- !query
-SELECT foo3_1a(MAX(c1), c2) FROM t1 GROUP BY c2
+SELECT foo9p(from_json('{1:"hello"}', 'struct'))
-- !query analysis
-Project [spark_catalog.default.foo3_1a(a#x, b#x) AS spark_catalog.default.foo3_1a(max(c1), c2)#x]
-+- Project [max(c1)#x, c2#x, cast(max(c1)#x as double) AS a#x, cast(c2#x as double) AS b#x]
- +- Aggregate [c2#x], [max(c1#x) AS max(c1)#x, c2#x]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+Project [spark_catalog.default.foo9p(a#x) AS spark_catalog.default.foo9p(from_json({1:"hello"}))#x]
++- Project [cast(from_json(StructField(a1,IntegerType,true), StructField(a2,StringType,true), {1:"hello"}, Some(America/Los_Angeles), false) as struct) AS a#x]
+ +- OneRowRelation
-- !query
-SELECT foo3_1a(c1, c2) FROM t1 GROUP BY c1, c2
+CREATE FUNCTION foo9q(a ARRAY>) RETURNS ARRAY> RETURN a
-- !query analysis
-Project [spark_catalog.default.foo3_1a(a#x, b#x) AS spark_catalog.default.foo3_1a(c1, c2)#x]
-+- Project [c1#x, c2#x, cast(c1#x as double) AS a#x, cast(c2#x as double) AS b#x]
- +- Aggregate [c1#x, c2#x], [c1#x, c2#x]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+CreateSQLFunctionCommand spark_catalog.default.foo9q, a ARRAY>, ARRAY>, a, false, false, false, false
-- !query
-SELECT MAX(foo3_1a(c1, c2)) FROM t1 GROUP BY c1, c2
+SELECT foo9q(ARRAY(STRUCT(1, 'hello'), STRUCT(2, 'world')))
-- !query analysis
-Project [max(spark_catalog.default.foo3_1a(c1, c2))#x]
-+- Aggregate [c1#x, c2#x], [max(spark_catalog.default.foo3_1a(a#x, b#x)) AS max(spark_catalog.default.foo3_1a(c1, c2))#x]
- +- Project [c1#x, c2#x, cast(c1#x as double) AS a#x, cast(c2#x as double) AS b#x]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+Project [spark_catalog.default.foo9q(a#x) AS spark_catalog.default.foo9q(array(struct(1, hello), struct(2, world)))#x]
++- Project [cast(array(struct(col1, 1, col2, hello), struct(col1, 2, col2, world)) as array>) AS a#x]
+ +- OneRowRelation
-- !query
-SELECT MAX(c1) + foo3_1b(MAX(c1)) FROM t1 GROUP BY c2
+SELECT foo9q(ARRAY(NAMED_STRUCT('x', 1, 'y', 'hello'), NAMED_STRUCT('x', 2, 'y', 'world')))
-- !query analysis
-Project [(max(c1)#x + spark_catalog.default.foo3_1b(x#x)) AS (max(c1) + spark_catalog.default.foo3_1b(max(c1)))#x]
-+- Project [max(c1)#x, max(c1)#x, cast(max(c1)#x as int) AS x#x]
- +- Aggregate [c2#x], [max(c1#x) AS max(c1)#x, max(c1#x) AS max(c1)#x]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+Project [spark_catalog.default.foo9q(a#x) AS spark_catalog.default.foo9q(array(named_struct(x, 1, y, hello), named_struct(x, 2, y, world)))#x]
++- Project [cast(array(named_struct(x, 1, y, hello), named_struct(x, 2, y, world)) as array>) AS a#x]
+ +- OneRowRelation
-- !query
-SELECT c1, SUM(foo3_1c(c2)) FROM t1 GROUP BY c1
+SELECT foo9q(from_json('[{1:"hello"}, {2:"world"}]', 'array>'))
-- !query analysis
-Project [c1#x, sum(spark_catalog.default.foo3_1c(c2))#xL]
-+- Aggregate [c1#x], [c1#x, sum(spark_catalog.default.foo3_1c(x#x)) AS sum(spark_catalog.default.foo3_1c(c2))#xL]
- +- Project [c1#x, c2#x, cast(c2#x as int) AS x#x]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+Project [spark_catalog.default.foo9q(a#x) AS spark_catalog.default.foo9q(from_json([{1:"hello"}, {2:"world"}]))#x]
++- Project [cast(from_json(ArrayType(StructType(StructField(a1,IntegerType,true),StructField(a2,StringType,true)),true), [{1:"hello"}, {2:"world"}], Some(America/Los_Angeles), false) as array>) AS a#x]
+ +- OneRowRelation
-- !query
-SELECT c1, SUM(foo3_1d(c2)) FROM t1 GROUP BY c1
+CREATE FUNCTION foo9r(a ARRAY>) RETURNS ARRAY> RETURN a
-- !query analysis
-Project [c1#x, sum(spark_catalog.default.foo3_1d(c2))#xL]
-+- Aggregate [c1#x], [c1#x, sum(spark_catalog.default.foo3_1d(x#x)) AS sum(spark_catalog.default.foo3_1d(c2))#xL]
- : +- Aggregate [sum(c2#x) AS sum(c2)#xL]
- : +- Filter (c1#x = outer(x#x))
- : +- SubqueryAlias spark_catalog.default.t2
- : +- View (`spark_catalog`.`default`.`t2`, [c1#x, c2#x])
- : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- : +- LocalRelation [col1#x, col2#x]
- +- Project [c1#x, c2#x, cast(c2#x as int) AS x#x]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+CreateSQLFunctionCommand spark_catalog.default.foo9r, a ARRAY>, ARRAY>, a, false, false, false, false
-- !query
-SELECT foo3_1c(c1), foo3_1d(c1) FROM t1 GROUP BY c1
+SELECT foo9r(ARRAY(MAP('hello', 1), MAP('world', 2)))
-- !query analysis
-Project [spark_catalog.default.foo3_1c(x#x) AS spark_catalog.default.foo3_1c(c1)#x, spark_catalog.default.foo3_1d(x#x) AS spark_catalog.default.foo3_1d(c1)#x]
-: +- Aggregate [sum(c2#x) AS sum(c2)#xL]
-: +- Filter (c1#x = outer(x#x))
-: +- SubqueryAlias spark_catalog.default.t2
-: +- View (`spark_catalog`.`default`.`t2`, [c1#x, c2#x])
-: +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
-: +- LocalRelation [col1#x, col2#x]
-+- Project [c1#x, c1#x, cast(c1#x as int) AS x#x, cast(c1#x as int) AS x#x]
- +- Aggregate [c1#x], [c1#x, c1#x]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+Project [spark_catalog.default.foo9r(a#x) AS spark_catalog.default.foo9r(array(map(hello, 1), map(world, 2)))#x]
++- Project [cast(array(map(hello, 1), map(world, 2)) as array>) AS a#x]
+ +- OneRowRelation
-- !query
-SELECT foo3_1a(SUM(c1), rand(0) * 0) FROM t1
+SELECT foo9r(from_json('[{"hello":1}, {"world":2}]', 'array>'))
-- !query analysis
-[Analyzer test output redacted due to nondeterminism]
+Project [spark_catalog.default.foo9r(a#x) AS spark_catalog.default.foo9r(from_json([{"hello":1}, {"world":2}]))#x]
++- Project [cast(from_json(ArrayType(MapType(StringType,IntegerType,true),true), [{"hello":1}, {"world":2}], Some(America/Los_Angeles), false) as array>) AS a#x]
+ +- OneRowRelation
-- !query
-SELECT foo3_1a(SUM(c1) + rand(0) * 0, SUM(c2)) FROM t1
+CREATE OR REPLACE FUNCTION foo1_10(a INT) RETURNS INT RETURN a + 2
-- !query analysis
-[Analyzer test output redacted due to nondeterminism]
+CreateSQLFunctionCommand spark_catalog.default.foo1_10, a INT, INT, a + 2, false, false, false, true
-- !query
-SELECT foo3_1b(SUM(c1) + rand(0) * 0) FROM t1
+CREATE OR REPLACE FUNCTION bar1_10(b INT) RETURNS STRING RETURN foo1_10(TRY_CAST(b AS STRING))
-- !query analysis
-[Analyzer test output redacted due to nondeterminism]
+CreateSQLFunctionCommand spark_catalog.default.bar1_10, b INT, STRING, foo1_10(TRY_CAST(b AS STRING)), false, false, false, true
-- !query
-SELECT foo3_1b(SUM(1) + rand(0) * 0) FROM t1 GROUP BY c2
+SELECT bar1_10(3)
-- !query analysis
-[Analyzer test output redacted due to nondeterminism]
+Project [spark_catalog.default.bar1_10(b#x) AS spark_catalog.default.bar1_10(3)#x]
++- Project [b#x, cast(try_cast(b#x as string) as int) AS a#x]
+ +- Project [cast(3 as int) AS b#x]
+ +- OneRowRelation
-- !query
-SELECT foo3_1c(SUM(c2) + rand(0) * 0) FROM t1 GROUP by c1
+CREATE OR REPLACE FUNCTION foo1_11a() RETURN 42
-- !query analysis
-[Analyzer test output redacted due to nondeterminism]
+CreateSQLFunctionCommand spark_catalog.default.foo1_11a, , 42, false, false, false, true
-- !query
-SELECT foo3_1b(foo3_1b(MAX(c2))) FROM t1
+SELECT foo1_11a()
-- !query analysis
-Project [spark_catalog.default.foo3_1b(x#x) AS spark_catalog.default.foo3_1b(spark_catalog.default.foo3_1b(max(c2)))#x]
-+- Project [max(c2)#x, x#x, cast(spark_catalog.default.foo3_1b(x#x) as int) AS x#x]
- +- Project [max(c2)#x, cast(max(c2)#x as int) AS x#x]
- +- Aggregate [max(c2#x) AS max(c2)#x]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+Project [spark_catalog.default.foo1_11a() AS spark_catalog.default.foo1_11a()#x]
++- Project
+ +- OneRowRelation
-- !query
-SELECT foo3_1b(MAX(foo3_1b(c2))) FROM t1
+CREATE OR REPLACE FUNCTION foo1_11b() RETURN 'hello world'
-- !query analysis
-Project [spark_catalog.default.foo3_1b(x#x) AS spark_catalog.default.foo3_1b(max(spark_catalog.default.foo3_1b(c2)))#x]
-+- Project [max(spark_catalog.default.foo3_1b(c2))#x, cast(max(spark_catalog.default.foo3_1b(c2))#x as int) AS x#x]
- +- Aggregate [max(spark_catalog.default.foo3_1b(x#x)) AS max(spark_catalog.default.foo3_1b(c2))#x]
- +- Project [c1#x, c2#x, cast(c2#x as int) AS x#x]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+CreateSQLFunctionCommand spark_catalog.default.foo1_11b, , 'hello world', false, false, false, true
-- !query
-SELECT foo3_1a(foo3_1b(c1), MAX(c2)) FROM t1 GROUP BY c1
+SELECT foo1_11b()
-- !query analysis
-Project [spark_catalog.default.foo3_1a(a#x, b#x) AS spark_catalog.default.foo3_1a(spark_catalog.default.foo3_1b(c1), max(c2))#x]
-+- Project [c1#x, max(c2)#x, x#x, cast(spark_catalog.default.foo3_1b(x#x) as double) AS a#x, cast(max(c2)#x as double) AS b#x]
- +- Project [c1#x, max(c2)#x, cast(c1#x as int) AS x#x]
- +- Aggregate [c1#x], [c1#x, max(c2#x) AS max(c2)#x]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+Project [spark_catalog.default.foo1_11b() AS spark_catalog.default.foo1_11b()#x]
++- Project
+ +- OneRowRelation
-- !query
-SELECT c1, foo3_1b(c1) FROM t1 GROUP BY c1
+CREATE OR REPLACE FUNCTION foo1_11c(a INT, b INT) RETURN a + b
-- !query analysis
-Project [c1#x, spark_catalog.default.foo3_1b(x#x) AS spark_catalog.default.foo3_1b(c1)#x]
-+- Project [c1#x, c1#x, cast(c1#x as int) AS x#x]
- +- Aggregate [c1#x], [c1#x, c1#x]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+CreateSQLFunctionCommand spark_catalog.default.foo1_11c, a INT, b INT, , a + b, false, false, false, true
-- !query
-SELECT c1, foo3_1b(c1 + 1) FROM t1 GROUP BY c1
+SELECT foo1_11c(3, 5)
-- !query analysis
-Project [c1#x, spark_catalog.default.foo3_1b(x#x) AS spark_catalog.default.foo3_1b((c1 + 1))#x]
-+- Project [c1#x, (c1 + 1)#x, cast((c1 + 1)#x as int) AS x#x]
- +- Aggregate [c1#x], [c1#x, (c1#x + 1) AS (c1 + 1)#x]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+Project [spark_catalog.default.foo1_11c(a#x, b#x) AS spark_catalog.default.foo1_11c(3, 5)#x]
++- Project [cast(3 as int) AS a#x, cast(5 as int) AS b#x]
+ +- OneRowRelation
-- !query
-SELECT c1, foo3_1b(c1 + rand(0) * 0) FROM t1 GROUP BY c1
+CREATE OR REPLACE FUNCTION foo1_11d(a DOUBLE, b INT) RETURN a * b + 1.5
-- !query analysis
-[Analyzer test output redacted due to nondeterminism]
+CreateSQLFunctionCommand spark_catalog.default.foo1_11d, a DOUBLE, b INT, , a * b + 1.5, false, false, false, true
-- !query
-SELECT c1, foo3_1a(c1, MIN(c2)) FROM t1 GROUP BY c1
+SELECT foo1_11d(3.0, 5)
-- !query analysis
-Project [c1#x, spark_catalog.default.foo3_1a(a#x, b#x) AS spark_catalog.default.foo3_1a(c1, min(c2))#x]
-+- Project [c1#x, c1#x, min(c2)#x, cast(c1#x as double) AS a#x, cast(min(c2)#x as double) AS b#x]
- +- Aggregate [c1#x], [c1#x, c1#x, min(c2#x) AS min(c2)#x]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+Project [spark_catalog.default.foo1_11d(a#x, b#x) AS spark_catalog.default.foo1_11d(3.0, 5)#x]
++- Project [cast(3.0 as double) AS a#x, cast(5 as int) AS b#x]
+ +- OneRowRelation
-- !query
-SELECT c1, foo3_1a(c1 + 1, MIN(c2 + 1)) FROM t1 GROUP BY c1
+CREATE OR REPLACE FUNCTION foo1_11e(a INT) RETURN a > 10
-- !query analysis
-Project [c1#x, spark_catalog.default.foo3_1a(a#x, b#x) AS spark_catalog.default.foo3_1a((c1 + 1), min((c2 + 1)))#x]
-+- Project [c1#x, (c1 + 1)#x, min((c2 + 1))#x, cast((c1 + 1)#x as double) AS a#x, cast(min((c2 + 1))#x as double) AS b#x]
- +- Aggregate [c1#x], [c1#x, (c1#x + 1) AS (c1 + 1)#x, min((c2#x + 1)) AS min((c2 + 1))#x]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+CreateSQLFunctionCommand spark_catalog.default.foo1_11e, a INT, , a > 10, false, false, false, true
-- !query
-SELECT c1, c2, foo3_1a(c1, c2) FROM t1 GROUP BY c1, c2
+SELECT foo1_11e(15), foo1_11e(5)
-- !query analysis
-Project [c1#x, c2#x, spark_catalog.default.foo3_1a(a#x, b#x) AS spark_catalog.default.foo3_1a(c1, c2)#x]
-+- Project [c1#x, c2#x, c1#x, c2#x, cast(c1#x as double) AS a#x, cast(c2#x as double) AS b#x]
- +- Aggregate [c1#x, c2#x], [c1#x, c2#x, c1#x, c2#x]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+Project [spark_catalog.default.foo1_11e(a#x) AS spark_catalog.default.foo1_11e(15)#x, spark_catalog.default.foo1_11e(a#x) AS spark_catalog.default.foo1_11e(5)#x]
++- Project [cast(15 as int) AS a#x, cast(5 as int) AS a#x]
+ +- OneRowRelation
-- !query
-SELECT c1, c2, foo3_1a(1, 2) FROM t1 GROUP BY c1, c2
+CREATE OR REPLACE FUNCTION foo1_11f(d DATE) RETURN d + INTERVAL '1' DAY
-- !query analysis
-Project [c1#x, c2#x, spark_catalog.default.foo3_1a(a#x, b#x) AS spark_catalog.default.foo3_1a(1, 2)#x]
-+- Project [c1#x, c2#x, 1#x, 2#x, cast(1#x as double) AS a#x, cast(2#x as double) AS b#x]
- +- Aggregate [c1#x, c2#x], [c1#x, c2#x, 1 AS 1#x, 2 AS 2#x]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+CreateSQLFunctionCommand spark_catalog.default.foo1_11f, d DATE, , d + INTERVAL '1' DAY, false, false, false, true
-- !query
-SELECT c1 + c2, foo3_1b(c1 + c2 + 1) FROM t1 GROUP BY c1 + c2
+SELECT foo1_11f(DATE '2024-01-01')
-- !query analysis
-Project [(c1 + c2)#x, spark_catalog.default.foo3_1b(x#x) AS spark_catalog.default.foo3_1b(((c1 + c2) + 1))#x]
-+- Project [(c1 + c2)#x, ((c1 + c2) + 1)#x, cast(((c1 + c2) + 1)#x as int) AS x#x]
- +- Aggregate [(c1#x + c2#x)], [(c1#x + c2#x) AS (c1 + c2)#x, ((c1#x + c2#x) + 1) AS ((c1 + c2) + 1)#x]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+[Analyzer test output redacted due to nondeterminism]
-- !query
-SELECT COUNT(*) + foo3_1b(c1) + foo3_1b(SUM(c2)) + SUM(foo3_1b(c2)) FROM t1 GROUP BY c1
+CREATE OR REPLACE FUNCTION foo1_11g(n INT) RETURN ARRAY(1, 2, n)
-- !query analysis
-Project [(((count(1)#xL + cast(spark_catalog.default.foo3_1b(x#x) as bigint)) + cast(spark_catalog.default.foo3_1b(x#x) as bigint)) + sum(spark_catalog.default.foo3_1b(c2))#xL) AS (((count(1) + spark_catalog.default.foo3_1b(c1)) + spark_catalog.default.foo3_1b(sum(c2))) + sum(spark_catalog.default.foo3_1b(c2)))#xL]
-+- Project [count(1)#xL, c1#x, sum(c2)#xL, sum(spark_catalog.default.foo3_1b(c2))#xL, cast(c1#x as int) AS x#x, cast(sum(c2)#xL as int) AS x#x]
- +- Aggregate [c1#x], [count(1) AS count(1)#xL, c1#x, sum(c2#x) AS sum(c2)#xL, sum(spark_catalog.default.foo3_1b(x#x)) AS sum(spark_catalog.default.foo3_1b(c2))#xL]
- +- Project [c1#x, c2#x, cast(c2#x as int) AS x#x]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+CreateSQLFunctionCommand spark_catalog.default.foo1_11g, n INT, , ARRAY(1, 2, n), false, false, false, true
-- !query
-SELECT c1, COUNT(*), foo3_1b(SUM(c2)) FROM t1 GROUP BY c1 HAVING COUNT(*) > 0
+SELECT foo1_11g(5)
-- !query analysis
-Filter (count(1)#xL > cast(0 as bigint))
-+- Project [c1#x, count(1)#xL, spark_catalog.default.foo3_1b(x#x) AS spark_catalog.default.foo3_1b(sum(c2))#x]
- +- Project [c1#x, count(1)#xL, sum(c2)#xL, cast(sum(c2)#xL as int) AS x#x]
- +- Aggregate [c1#x], [c1#x, count(1) AS count(1)#xL, sum(c2#x) AS sum(c2)#xL]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+Project [spark_catalog.default.foo1_11g(n#x) AS spark_catalog.default.foo1_11g(5)#x]
++- Project [cast(5 as int) AS n#x]
+ +- OneRowRelation
-- !query
-SELECT c1, COUNT(*), foo3_1b(SUM(c2)) FROM t1 GROUP BY c1 HAVING foo3_1b(SUM(c2)) > 0
+CREATE OR REPLACE FUNCTION foo1_11h(a INT, b STRING) RETURN STRUCT(a, b)
-- !query analysis
-Filter (spark_catalog.default.foo3_1b(sum(c2))#x > 0)
-+- Project [c1#x, count(1)#xL, spark_catalog.default.foo3_1b(x#x) AS spark_catalog.default.foo3_1b(sum(c2))#x]
- +- Project [c1#x, count(1)#xL, sum(c2)#xL, cast(sum(c2)#xL as int) AS x#x]
- +- Aggregate [c1#x], [c1#x, count(1) AS count(1)#xL, sum(c2#x) AS sum(c2)#xL]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+CreateSQLFunctionCommand spark_catalog.default.foo1_11h, a INT, b STRING, , STRUCT(a, b), false, false, false, true
-- !query
-SELECT c1, COUNT(*), foo3_1b(SUM(c2)) FROM t1 GROUP BY c1 HAVING SUM(foo3_1b(c2)) > 0
+SELECT foo1_11h(1, 'test')
-- !query analysis
-Project [c1#x, count(1)#xL, spark_catalog.default.foo3_1b(sum(c2))#x]
-+- Filter (sum(spark_catalog.default.foo3_1b(c2))#xL > cast(0 as bigint))
- +- Project [c1#x, count(1)#xL, spark_catalog.default.foo3_1b(x#x) AS spark_catalog.default.foo3_1b(sum(c2))#x, sum(spark_catalog.default.foo3_1b(c2))#xL]
- +- Project [c1#x, count(1)#xL, sum(c2)#xL, sum(spark_catalog.default.foo3_1b(c2))#xL, cast(sum(c2)#xL as int) AS x#x]
- +- Aggregate [c1#x], [c1#x, count(1) AS count(1)#xL, sum(c2#x) AS sum(c2)#xL, sum(spark_catalog.default.foo3_1b(x#x)) AS sum(spark_catalog.default.foo3_1b(c2))#xL]
- +- Project [c1#x, c2#x, cast(c2#x as int) AS x#x]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+Project [spark_catalog.default.foo1_11h(a#x, b#x) AS spark_catalog.default.foo1_11h(1, test)#x]
++- Project [cast(1 as int) AS a#x, cast(test as string) AS b#x]
+ +- OneRowRelation
-- !query
-SELECT foo3_1b(c1), MIN(c2) FROM t1 GROUP BY 1
+CREATE OR REPLACE FUNCTION foo1_11i(x INT) RETURN (SELECT x * 2)
-- !query analysis
-Project [spark_catalog.default.foo3_1b(c1)#x, min(c2)#x]
-+- Aggregate [spark_catalog.default.foo3_1b#x], [spark_catalog.default.foo3_1b#x AS spark_catalog.default.foo3_1b(c1)#x, min(c2#x) AS min(c2)#x]
- +- Project [c1#x, c2#x, spark_catalog.default.foo3_1b(x#x) AS spark_catalog.default.foo3_1b#x]
- +- Project [c1#x, c2#x, cast(c1#x as int) AS x#x]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+CreateSQLFunctionCommand spark_catalog.default.foo1_11i, x INT, , (SELECT x * 2), false, false, false, true
-- !query
-SELECT foo3_1a(c1 + rand(0) * 0, c2) FROM t1 GROUP BY 1
+SELECT foo1_11i(5)
-- !query analysis
-[Analyzer test output redacted due to nondeterminism]
+Project [spark_catalog.default.foo1_11i(x#x) AS spark_catalog.default.foo1_11i(5)#x]
++- Project [cast(5 as int) AS x#x]
+ +- OneRowRelation
-- !query
-SELECT c1, c2, foo3_1a(c1, c2) FROM t1 GROUP BY c1, c2, 3
+CREATE OR REPLACE FUNCTION foo1_11j(s STRING) RETURN UPPER(s)
-- !query analysis
-Project [c1#x, c2#x, spark_catalog.default.foo3_1a(c1, c2)#x]
-+- Aggregate [c1#x, c2#x, spark_catalog.default.foo3_1a#x], [c1#x, c2#x, spark_catalog.default.foo3_1a#x AS spark_catalog.default.foo3_1a(c1, c2)#x]
- +- Project [c1#x, c2#x, spark_catalog.default.foo3_1a(a#x, b#x) AS spark_catalog.default.foo3_1a#x]
- +- Project [c1#x, c2#x, cast(c1#x as double) AS a#x, cast(c2#x as double) AS b#x]
- +- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+CreateSQLFunctionCommand spark_catalog.default.foo1_11j, s STRING, , UPPER(s), false, false, false, true
-- !query
-SELECT c1, (SELECT c1), (SELECT foo3_1b(c1)), SUM(c2) FROM t1 GROUP BY 1, 2, 3
+SELECT foo1_11j('hello')
-- !query analysis
-Aggregate [c1#x, scalar-subquery#x [c1#x], scalar-subquery#x [c1#x]], [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#x, sum(c2#x) AS sum(c2)#xL]
-: :- Project [outer(c1#x)]
-: : +- OneRowRelation
-: :- Project [spark_catalog.default.foo3_1b(x#x) AS spark_catalog.default.foo3_1b(outer(spark_catalog.default.t1.c1))#x]
-: : +- Project [cast(outer(c1#x) as int) AS x#x]
-: : +- OneRowRelation
-: :- Project [outer(c1#x)]
-: : +- OneRowRelation
-: +- Project [spark_catalog.default.foo3_1b(x#x) AS spark_catalog.default.foo3_1b(outer(spark_catalog.default.t1.c1))#x]
-: +- Project [cast(outer(c1#x) as int) AS x#x]
-: +- OneRowRelation
-+- SubqueryAlias spark_catalog.default.t1
- +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
+Project [spark_catalog.default.foo1_11j(s#x) AS spark_catalog.default.foo1_11j(hello)#x]
++- Project [cast(hello as string) AS s#x]
+ +- OneRowRelation
-- !query
-SELECT c1, SUM(c2) + foo3_1a(MIN(c2), MAX(c2)) + (SELECT SUM(c2)) FROM t1 GROUP BY c1
+CREATE OR REPLACE FUNCTION foo1_11k(a INT, b STRING) RETURN CONCAT(CAST(a AS STRING), '_', b)
-- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
-{
- "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE",
- "sqlState" : "0A000",
- "messageParameters" : {
- "sqlExprs" : "\"sum(c2) AS `sum(outer(spark_catalog.default.t1.c2))`\""
+CreateSQLFunctionCommand spark_catalog.default.foo1_11k, a INT, b STRING, , CONCAT(CAST(a AS STRING), '_', b), false, false, false, true
+
+
+-- !query
+SELECT foo1_11k(123, 'test')
+-- !query analysis
+Project [spark_catalog.default.foo1_11k(a#x, b#x) AS spark_catalog.default.foo1_11k(123, test)#x]
++- Project [cast(123 as int) AS a#x, cast(test as string) AS b#x]
+ +- OneRowRelation
+
+
+-- !query
+CREATE OR REPLACE FUNCTION foo1_11l() RETURNS TABLE RETURN SELECT 1 as id, 'hello' as name
+-- !query analysis
+CreateSQLFunctionCommand spark_catalog.default.foo1_11l, TABLE, SELECT 1 as id, 'hello' as name, true, false, false, true
+
+
+-- !query
+SELECT * FROM foo1_11l()
+-- !query analysis
+Project [id#x, name#x]
++- SQLFunctionNode spark_catalog.default.foo1_11l
+ +- SubqueryAlias foo1_11l
+ +- Project [cast(id#x as int) AS id#x, cast(name#x as string) AS name#x]
+ +- Project [1 AS id#x, hello AS name#x]
+ +- OneRowRelation
+
+
+-- !query
+CREATE OR REPLACE FUNCTION foo1_11m(a INT, b STRING) RETURNS TABLE RETURN SELECT a * 2 as doubled, UPPER(b) as upper_name
+-- !query analysis
+CreateSQLFunctionCommand spark_catalog.default.foo1_11m, a INT, b STRING, TABLE, SELECT a * 2 as doubled, UPPER(b) as upper_name, true, false, false, true
+
+
+-- !query
+SELECT * FROM foo1_11m(5, 'world')
+-- !query analysis
+Project [doubled#x, upper_name#x]
++- SQLFunctionNode spark_catalog.default.foo1_11m
+ +- SubqueryAlias foo1_11m
+ +- Project [cast(doubled#x as int) AS doubled#x, cast(upper_name#x as string) AS upper_name#x]
+ +- Project [(cast(5 as int) * 2) AS doubled#x, upper(cast(world as string)) AS upper_name#x]
+ +- OneRowRelation
+
+
+-- !query
+CREATE OR REPLACE FUNCTION foo1_11n(arr ARRAY) RETURNS TABLE RETURN SELECT size(arr) as array_size, arr[0] as first_element
+-- !query analysis
+CreateSQLFunctionCommand spark_catalog.default.foo1_11n, arr ARRAY, TABLE, SELECT size(arr) as array_size, arr[0] as first_element, true, false, false, true
+
+
+-- !query
+SELECT * FROM foo1_11n(ARRAY(1, 2, 3))
+-- !query analysis
+Project [array_size#x, first_element#x]
++- SQLFunctionNode spark_catalog.default.foo1_11n
+ +- SubqueryAlias foo1_11n
+ +- Project [cast(array_size#x as int) AS array_size#x, cast(first_element#x as int) AS first_element#x]
+ +- Project [size(cast(array(1, 2, 3) as array), false) AS array_size#x, cast(array(1, 2, 3) as array)[0] AS first_element#x]
+ +- OneRowRelation
+
+
+-- !query
+CREATE OR REPLACE FUNCTION foo1_11o(id INT, name STRING) RETURNS TABLE RETURN SELECT STRUCT(id, name) as person_info, id + 100 as modified_id
+-- !query analysis
+CreateSQLFunctionCommand spark_catalog.default.foo1_11o, id INT, name STRING, TABLE, SELECT STRUCT(id, name) as person_info, id + 100 as modified_id, true, false, false, true
+
+
+-- !query
+SELECT * FROM foo1_11o(1, 'Alice')
+-- !query analysis
+Project [person_info#x, modified_id#x]
++- SQLFunctionNode spark_catalog.default.foo1_11o
+ +- SubqueryAlias foo1_11o
+ +- Project [cast(person_info#x as struct) AS person_info#x, cast(modified_id#x as int) AS modified_id#x]
+ +- Project [struct(id, cast(1 as int), name, cast(Alice as string)) AS person_info#x, (cast(1 as int) + 100) AS modified_id#x]
+ +- OneRowRelation
+
+
+-- !query
+CREATE FUNCTION foo2_1a(a INT) RETURNS INT RETURN a
+-- !query analysis
+CreateSQLFunctionCommand spark_catalog.default.foo2_1a, a INT, INT, a, false, false, false, false
+
+
+-- !query
+SELECT foo2_1a(5)
+-- !query analysis
+Project [spark_catalog.default.foo2_1a(a#x) AS spark_catalog.default.foo2_1a(5)#x]
++- Project [cast(5 as int) AS a#x]
+ +- OneRowRelation
+
+
+-- !query
+CREATE FUNCTION foo2_1b(a INT, b INT) RETURNS INT RETURN a + b
+-- !query analysis
+CreateSQLFunctionCommand spark_catalog.default.foo2_1b, a INT, b INT, INT, a + b, false, false, false, false
+
+
+-- !query
+SELECT foo2_1b(5, 6)
+-- !query analysis
+Project [spark_catalog.default.foo2_1b(a#x, b#x) AS spark_catalog.default.foo2_1b(5, 6)#x]
++- Project [cast(5 as int) AS a#x, cast(6 as int) AS b#x]
+ +- OneRowRelation
+
+
+-- !query
+CREATE FUNCTION foo2_1c(a INT, b INT) RETURNS INT RETURN 10 * (a + b) + 100 * (a -b)
+-- !query analysis
+CreateSQLFunctionCommand spark_catalog.default.foo2_1c, a INT, b INT, INT, 10 * (a + b) + 100 * (a -b), false, false, false, false
+
+
+-- !query
+SELECT foo2_1c(5, 6)
+-- !query analysis
+Project [spark_catalog.default.foo2_1c(a#x, b#x) AS spark_catalog.default.foo2_1c(5, 6)#x]
++- Project [cast(5 as int) AS a#x, cast(6 as int) AS b#x]
+ +- OneRowRelation
+
+
+-- !query
+CREATE FUNCTION foo2_1d(a INT, b INT) RETURNS INT RETURN ABS(a) - LENGTH(CAST(b AS VARCHAR(10)))
+-- !query analysis
+CreateSQLFunctionCommand spark_catalog.default.foo2_1d, a INT, b INT, INT, ABS(a) - LENGTH(CAST(b AS VARCHAR(10))), false, false, false, false
+
+
+-- !query
+SELECT foo2_1d(-5, 6)
+-- !query analysis
+Project [spark_catalog.default.foo2_1d(a#x, b#x) AS spark_catalog.default.foo2_1d(-5, 6)#x]
++- Project [cast(-5 as int) AS a#x, cast(6 as int) AS b#x]
+ +- OneRowRelation
+
+
+-- !query
+CREATE FUNCTION foo2_2a(a INT) RETURNS INT RETURN SELECT a
+-- !query analysis
+CreateSQLFunctionCommand spark_catalog.default.foo2_2a, a INT, INT, SELECT a, false, false, false, false
+
+
+-- !query
+SELECT foo2_2a(5)
+-- !query analysis
+Project [spark_catalog.default.foo2_2a(a#x) AS spark_catalog.default.foo2_2a(5)#x]
++- Project [cast(5 as int) AS a#x]
+ +- OneRowRelation
+
+
+-- !query
+CREATE FUNCTION foo2_2b(a INT) RETURNS INT RETURN 1 + (SELECT a)
+-- !query analysis
+CreateSQLFunctionCommand spark_catalog.default.foo2_2b, a INT, INT, 1 + (SELECT a), false, false, false, false
+
+
+-- !query
+SELECT foo2_2b(5)
+-- !query analysis
+Project [spark_catalog.default.foo2_2b(a#x) AS spark_catalog.default.foo2_2b(5)#x]
+: +- Project [outer(a#x)]
+: +- OneRowRelation
++- Project [cast(5 as int) AS a#x]
+ +- OneRowRelation
+
+
+-- !query
+CREATE FUNCTION foo2_2c(a INT) RETURNS INT RETURN 1 + (SELECT (SELECT a))
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION",
+ "sqlState" : "42703",
+ "messageParameters" : {
+ "objectName" : "`a`"
},
"queryContext" : [ {
"objectType" : "",
"objectName" : "",
- "startIndex" : 51,
- "stopIndex" : 64,
- "fragment" : "SELECT SUM(c2)"
+ "startIndex" : 21,
+ "stopIndex" : 21,
+ "fragment" : "a"
} ]
}
-- !query
-SELECT foo3_1b(SUM(c1)) + (SELECT foo3_1b(SUM(c1))) FROM t1
+CREATE FUNCTION foo2_2d(a INT) RETURNS INT RETURN 1 + (SELECT (SELECT (SELECT (SELECT a))))
-- !query analysis
org.apache.spark.sql.catalyst.ExtendedAnalysisException
{
- "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE",
- "sqlState" : "0A000",
+ "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION",
+ "sqlState" : "42703",
"messageParameters" : {
- "sqlExprs" : "\"sum(c1) AS `sum(outer(spark_catalog.default.t1.c1))`\""
+ "objectName" : "`a`"
},
"queryContext" : [ {
"objectType" : "",
"objectName" : "",
- "startIndex" : 27,
- "stopIndex" : 51,
- "fragment" : "(SELECT foo3_1b(SUM(c1)))"
+ "startIndex" : 37,
+ "stopIndex" : 37,
+ "fragment" : "a"
} ]
}
-- !query
-SELECT SUM(foo3_1b(SUM(c1))) FROM t1
+CREATE FUNCTION foo2_2e(a INT) RETURNS INT RETURN
+SELECT a FROM (VALUES 1) AS V(c1) WHERE c1 = 2
+UNION ALL
+SELECT a + 1 FROM (VALUES 1) AS V(c1)
+-- !query analysis
+CreateSQLFunctionCommand spark_catalog.default.foo2_2e, a INT, INT, SELECT a FROM (VALUES 1) AS V(c1) WHERE c1 = 2
+UNION ALL
+SELECT a + 1 FROM (VALUES 1) AS V(c1), false, false, false, false
+
+
+-- !query
+CREATE FUNCTION foo2_2f(a INT) RETURNS INT RETURN
+SELECT a FROM (VALUES 1) AS V(c1)
+EXCEPT
+SELECT a + 1 FROM (VALUES 1) AS V(a)
+-- !query analysis
+CreateSQLFunctionCommand spark_catalog.default.foo2_2f, a INT, INT, SELECT a FROM (VALUES 1) AS V(c1)
+EXCEPT
+SELECT a + 1 FROM (VALUES 1) AS V(a), false, false, false, false
+
+
+-- !query
+CREATE FUNCTION foo2_2g(a INT) RETURNS INT RETURN
+SELECT a FROM (VALUES 1) AS V(c1)
+INTERSECT
+SELECT a FROM (VALUES 1) AS V(a)
+-- !query analysis
+CreateSQLFunctionCommand spark_catalog.default.foo2_2g, a INT, INT, SELECT a FROM (VALUES 1) AS V(c1)
+INTERSECT
+SELECT a FROM (VALUES 1) AS V(a), false, false, false, false
+
+
+-- !query
+DROP TABLE IF EXISTS t1
+-- !query analysis
+DropTable true, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t1
+
+
+-- !query
+DROP TABLE IF EXISTS t2
+-- !query analysis
+DropTable true, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t2
+
+
+-- !query
+DROP TABLE IF EXISTS ts
-- !query analysis
org.apache.spark.sql.AnalysisException
{
- "errorClass" : "NESTED_AGGREGATE_FUNCTION",
- "sqlState" : "42607",
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 12,
- "stopIndex" : 27,
- "fragment" : "foo3_1b(SUM(c1))"
- } ]
+ "errorClass" : "WRONG_COMMAND_FOR_OBJECT_TYPE",
+ "sqlState" : "42809",
+ "messageParameters" : {
+ "alternative" : "DROP VIEW",
+ "foundType" : "VIEW",
+ "objectName" : "spark_catalog.default.ts",
+ "operation" : "DROP TABLE",
+ "requiredType" : "EXTERNAL or MANAGED"
+ }
}
-- !query
-SELECT foo3_1b(SUM(c1)) + (SELECT SUM(SUM(c1))) FROM t1
+DROP TABLE IF EXISTS tm
-- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
+org.apache.spark.sql.AnalysisException
{
- "errorClass" : "NESTED_AGGREGATE_FUNCTION",
- "sqlState" : "42607",
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 39,
- "stopIndex" : 45,
- "fragment" : "SUM(c1)"
- } ]
+ "errorClass" : "WRONG_COMMAND_FOR_OBJECT_TYPE",
+ "sqlState" : "42809",
+ "messageParameters" : {
+ "alternative" : "DROP VIEW",
+ "foundType" : "VIEW",
+ "objectName" : "spark_catalog.default.tm",
+ "operation" : "DROP TABLE",
+ "requiredType" : "EXTERNAL or MANAGED"
+ }
}
-- !query
-SELECT foo3_1b(SUM(c1) + SUM(SUM(c1))) FROM t1
+DROP TABLE IF EXISTS ta
-- !query analysis
-org.apache.spark.sql.catalyst.ExtendedAnalysisException
+org.apache.spark.sql.AnalysisException
{
- "errorClass" : "NESTED_AGGREGATE_FUNCTION",
- "sqlState" : "42607",
- "queryContext" : [ {
- "objectType" : "",
- "objectName" : "",
- "startIndex" : 30,
- "stopIndex" : 36,
- "fragment" : "SUM(c1)"
- } ]
-}
+ "errorClass" : "WRONG_COMMAND_FOR_OBJECT_TYPE",
+ "sqlState" : "42809",
+ "messageParameters" : {
+ "alternative" : "DROP VIEW",
+ "foundType" : "VIEW",
+ "objectName" : "spark_catalog.default.ta",
+ "operation" : "DROP TABLE",
+ "requiredType" : "EXTERNAL or MANAGED"
+ }
+}
+
+
+-- !query
+DROP TABLE IF EXISTS V1
+-- !query analysis
+DropTable true, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.V1
+
+
+-- !query
+DROP TABLE IF EXISTS V2
+-- !query analysis
+DropTable true, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.V2
+
+
+-- !query
+DROP VIEW IF EXISTS t1
+-- !query analysis
+DropTableCommand `spark_catalog`.`default`.`t1`, true, true, false
+
+
+-- !query
+DROP VIEW IF EXISTS t2
+-- !query analysis
+DropTableCommand `spark_catalog`.`default`.`t2`, true, true, false
+
+
+-- !query
+DROP VIEW IF EXISTS ts
+-- !query analysis
+DropTableCommand `spark_catalog`.`default`.`ts`, true, true, false
+
+
+-- !query
+DROP VIEW IF EXISTS tm
+-- !query analysis
+DropTableCommand `spark_catalog`.`default`.`tm`, true, true, false
+
+
+-- !query
+DROP VIEW IF EXISTS ta
+-- !query analysis
+DropTableCommand `spark_catalog`.`default`.`ta`, true, true, false
+
+
+-- !query
+DROP VIEW IF EXISTS V1
+-- !query analysis
+DropTableCommand `spark_catalog`.`default`.`V1`, true, true, false
+
+
+-- !query
+DROP VIEW IF EXISTS V2
+-- !query analysis
+DropTableCommand `spark_catalog`.`default`.`V2`, true, true, false
+
+
+-- !query
+CREATE FUNCTION foo2_3(a INT, b INT) RETURNS INT RETURN a + b
+-- !query analysis
+CreateSQLFunctionCommand spark_catalog.default.foo2_3, a INT, b INT, INT, a + b, false, false, false, false
+
+
+-- !query
+CREATE VIEW V1(c1, c2) AS VALUES (1, 2), (3, 4), (5, 6)
+-- !query analysis
+CreateViewCommand `spark_catalog`.`default`.`V1`, [(c1,None), (c2,None)], VALUES (1, 2), (3, 4), (5, 6), false, false, PersistedView, COMPENSATION, true
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+CREATE VIEW V2(c1, c2) AS VALUES (-1, -2), (-3, -4), (-5, -6)
+-- !query analysis
+CreateViewCommand `spark_catalog`.`default`.`V2`, [(c1,None), (c2,None)], VALUES (-1, -2), (-3, -4), (-5, -6), false, false, PersistedView, COMPENSATION, true
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT foo2_3(c1, c2), foo2_3(c2, 1), foo2_3(c1, c2) - foo2_3(c2, c1 - 1) FROM V1 ORDER BY 1, 2, 3
+-- !query analysis
+Sort [spark_catalog.default.foo2_3(c1, c2)#x ASC NULLS FIRST, spark_catalog.default.foo2_3(c2, 1)#x ASC NULLS FIRST, (spark_catalog.default.foo2_3(c1, c2) - spark_catalog.default.foo2_3(c2, (c1 - 1)))#x ASC NULLS FIRST], true
++- Project [spark_catalog.default.foo2_3(a#x, b#x) AS spark_catalog.default.foo2_3(c1, c2)#x, spark_catalog.default.foo2_3(a#x, b#x) AS spark_catalog.default.foo2_3(c2, 1)#x, (spark_catalog.default.foo2_3(a#x, b#x) - spark_catalog.default.foo2_3(a#x, b#x)) AS (spark_catalog.default.foo2_3(c1, c2) - spark_catalog.default.foo2_3(c2, (c1 - 1)))#x]
+ +- Project [c1#x, c2#x, cast(c1#x as int) AS a#x, cast(c2#x as int) AS b#x, cast(c2#x as int) AS a#x, cast(1 as int) AS b#x, cast(c1#x as int) AS a#x, cast(c2#x as int) AS b#x, cast(c2#x as int) AS a#x, cast((c1#x - 1) as int) AS b#x]
+ +- SubqueryAlias spark_catalog.default.v1
+ +- View (`spark_catalog`.`default`.`v1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT * FROM V1 WHERE foo2_3(c1, 0) = c1 AND foo2_3(c1, c2) < 8
+-- !query analysis
+Project [c1#x, c2#x]
++- Project [c1#x, c2#x]
+ +- Filter ((spark_catalog.default.foo2_3(a#x, b#x) = c1#x) AND (spark_catalog.default.foo2_3(a#x, b#x) < 8))
+ +- Project [c1#x, c2#x, cast(c1#x as int) AS a#x, cast(0 as int) AS b#x, cast(c1#x as int) AS a#x, cast(c2#x as int) AS b#x]
+ +- SubqueryAlias spark_catalog.default.v1
+ +- View (`spark_catalog`.`default`.`v1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT foo2_3(SUM(c1), SUM(c2)), SUM(c1) + SUM(c2), SUM(foo2_3(c1, c2) + foo2_3(c2, c1) - foo2_3(c2, c1))
+FROM V1
+-- !query analysis
+Project [spark_catalog.default.foo2_3(a#x, b#x) AS spark_catalog.default.foo2_3(sum(c1), sum(c2))#x, (sum(c1) + sum(c2))#xL, sum(((spark_catalog.default.foo2_3(c1, c2) + spark_catalog.default.foo2_3(c2, c1)) - spark_catalog.default.foo2_3(c2, c1)))#xL]
++- Project [sum(c1)#xL, sum(c2)#xL, (sum(c1) + sum(c2))#xL, sum(((spark_catalog.default.foo2_3(c1, c2) + spark_catalog.default.foo2_3(c2, c1)) - spark_catalog.default.foo2_3(c2, c1)))#xL, cast(sum(c1)#xL as int) AS a#x, cast(sum(c2)#xL as int) AS b#x]
+ +- Aggregate [sum(c1#x) AS sum(c1)#xL, sum(c2#x) AS sum(c2)#xL, (sum(c1#x) + sum(c2#x)) AS (sum(c1) + sum(c2))#xL, sum(((spark_catalog.default.foo2_3(a#x, b#x) + spark_catalog.default.foo2_3(a#x, b#x)) - spark_catalog.default.foo2_3(a#x, b#x))) AS sum(((spark_catalog.default.foo2_3(c1, c2) + spark_catalog.default.foo2_3(c2, c1)) - spark_catalog.default.foo2_3(c2, c1)))#xL]
+ +- Project [c1#x, c2#x, cast(c1#x as int) AS a#x, cast(c2#x as int) AS b#x, cast(c2#x as int) AS a#x, cast(c1#x as int) AS b#x, cast(c2#x as int) AS a#x, cast(c1#x as int) AS b#x]
+ +- SubqueryAlias spark_catalog.default.v1
+ +- View (`spark_catalog`.`default`.`v1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+CREATE FUNCTION foo2_4a(a ARRAY) RETURNS STRING RETURN
+SELECT array_sort(a, (i, j) -> rank[i] - rank[j])[0] FROM (SELECT MAP('a', 1, 'b', 2) rank)
+-- !query analysis
+CreateSQLFunctionCommand spark_catalog.default.foo2_4a, a ARRAY, STRING, SELECT array_sort(a, (i, j) -> rank[i] - rank[j])[0] FROM (SELECT MAP('a', 1, 'b', 2) rank), false, false, false, false
+
+
+-- !query
+SELECT foo2_4a(ARRAY('a', 'b'))
+-- !query analysis
+Project [spark_catalog.default.foo2_4a(a#x) AS spark_catalog.default.foo2_4a(array(a, b))#x]
+: +- Project [array_sort(outer(a#x), lambdafunction((rank#x[lambda i#x] - rank#x[lambda j#x]), lambda i#x, lambda j#x, false), false)[0] AS array_sort(outer(foo2_4a.a), lambdafunction((rank[namedlambdavariable()] - rank[namedlambdavariable()]), namedlambdavariable(), namedlambdavariable()))[0]#x]
+: +- SubqueryAlias __auto_generated_subquery_name
+: +- Project [map(a, 1, b, 2) AS rank#x]
+: +- OneRowRelation
++- Project [cast(array(a, b) as array) AS a#x]
+ +- OneRowRelation
+
+
+-- !query
+CREATE FUNCTION foo2_4b(m MAP, k STRING) RETURNS STRING RETURN
+SELECT v || ' ' || v FROM (SELECT upper(m[k]) AS v)
+-- !query analysis
+CreateSQLFunctionCommand spark_catalog.default.foo2_4b, m MAP, k STRING, STRING, SELECT v || ' ' || v FROM (SELECT upper(m[k]) AS v), false, false, false, false
+
+
+-- !query
+SELECT foo2_4b(map('a', 'hello', 'b', 'world'), 'a')
+-- !query analysis
+Project [spark_catalog.default.foo2_4b(m#x, k#x) AS spark_catalog.default.foo2_4b(map(a, hello, b, world), a)#x]
+: +- Project [concat(concat(v#x, ), v#x) AS concat(concat(v, ), v)#x]
+: +- SubqueryAlias __auto_generated_subquery_name
+: +- Project [upper(outer(m#x)[outer(k#x)]) AS v#x]
+: +- OneRowRelation
++- Project [cast(map(a, hello, b, world) as map) AS m#x, cast(a as string) AS k#x]
+ +- OneRowRelation
+
+
+-- !query
+DROP VIEW V2
+-- !query analysis
+DropTableCommand `spark_catalog`.`default`.`V2`, false, true, false
+
+
+-- !query
+DROP VIEW V1
+-- !query analysis
+DropTableCommand `spark_catalog`.`default`.`V1`, false, true, false
+
+
+-- !query
+CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (0, 2), (1, 2)
+-- !query analysis
+CreateViewCommand `spark_catalog`.`default`.`t1`, [(c1,None), (c2,None)], VALUES (0, 1), (0, 2), (1, 2), false, false, PersistedView, COMPENSATION, true
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+CREATE VIEW t2(c1, c2) AS VALUES (0, 2), (0, 3)
+-- !query analysis
+CreateViewCommand `spark_catalog`.`default`.`t2`, [(c1,None), (c2,None)], VALUES (0, 2), (0, 3), false, false, PersistedView, COMPENSATION, true
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+CREATE VIEW ts(x) AS VALUES NAMED_STRUCT('a', 1, 'b', 2)
+-- !query analysis
+CreateViewCommand `spark_catalog`.`default`.`ts`, [(x,None)], VALUES NAMED_STRUCT('a', 1, 'b', 2), false, false, PersistedView, COMPENSATION, true
+ +- LocalRelation [col1#x]
+
+
+-- !query
+CREATE VIEW tm(x) AS VALUES MAP('a', 1, 'b', 2)
+-- !query analysis
+CreateViewCommand `spark_catalog`.`default`.`tm`, [(x,None)], VALUES MAP('a', 1, 'b', 2), false, false, PersistedView, COMPENSATION, true
+ +- LocalRelation [col1#x]
+
+
+-- !query
+CREATE VIEW ta(x) AS VALUES ARRAY(1, 2, 3)
+-- !query analysis
+CreateViewCommand `spark_catalog`.`default`.`ta`, [(x,None)], VALUES ARRAY(1, 2, 3), false, false, PersistedView, COMPENSATION, true
+ +- LocalRelation [col1#x]
+
+
+-- !query
+CREATE FUNCTION foo3_1a(a DOUBLE, b DOUBLE) RETURNS DOUBLE RETURN a * b
+-- !query analysis
+CreateSQLFunctionCommand spark_catalog.default.foo3_1a, a DOUBLE, b DOUBLE, DOUBLE, a * b, false, false, false, false
+
+
+-- !query
+CREATE FUNCTION foo3_1b(x INT) RETURNS INT RETURN x
+-- !query analysis
+CreateSQLFunctionCommand spark_catalog.default.foo3_1b, x INT, INT, x, false, false, false, false
+
+
+-- !query
+CREATE FUNCTION foo3_1c(x INT) RETURNS INT RETURN SELECT x
+-- !query analysis
+CreateSQLFunctionCommand spark_catalog.default.foo3_1c, x INT, INT, SELECT x, false, false, false, false
+
+
+-- !query
+CREATE FUNCTION foo3_1d(x INT) RETURNS INT RETURN (SELECT SUM(c2) FROM t2 WHERE c1 = x)
+-- !query analysis
+CreateSQLFunctionCommand spark_catalog.default.foo3_1d, x INT, INT, (SELECT SUM(c2) FROM t2 WHERE c1 = x), false, false, false, false
+
+
+-- !query
+CREATE FUNCTION foo3_1e() RETURNS INT RETURN foo3_1d(0)
+-- !query analysis
+CreateSQLFunctionCommand spark_catalog.default.foo3_1e, INT, foo3_1d(0), false, false, false, false
+
+
+-- !query
+CREATE FUNCTION foo3_1f() RETURNS INT RETURN SELECT SUM(c2) FROM t2 WHERE c1 = 0
+-- !query analysis
+CreateSQLFunctionCommand spark_catalog.default.foo3_1f, INT, SELECT SUM(c2) FROM t2 WHERE c1 = 0, false, false, false, false
+
+
+-- !query
+CREATE FUNCTION foo3_1g(x INT) RETURNS INT RETURN SELECT (SELECT x)
+-- !query analysis
+CreateSQLFunctionCommand spark_catalog.default.foo3_1g, x INT, INT, SELECT (SELECT x), false, false, false, false
+
+
+-- !query
+SELECT a, b, foo3_1a(a + 1, b + 1) FROM t1 AS t(a, b)
+-- !query analysis
+Project [a#x, b#x, spark_catalog.default.foo3_1a(a#x, b#x) AS spark_catalog.default.foo3_1a((a + 1), (b + 1))#x]
++- Project [a#x, b#x, cast((a#x + 1) as double) AS a#x, cast((b#x + 1) as double) AS b#x]
+ +- SubqueryAlias t
+ +- Project [c1#x AS a#x, c2#x AS b#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT x, foo3_1c(x) FROM t1 AS t(x, y)
+-- !query analysis
+Project [x#x, spark_catalog.default.foo3_1c(x#x) AS spark_catalog.default.foo3_1c(x)#x]
++- Project [x#x, y#x, cast(x#x as int) AS x#x]
+ +- SubqueryAlias t
+ +- Project [c1#x AS x#x, c2#x AS y#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT c1, foo3_1d(c1) FROM t1
+-- !query analysis
+Project [c1#x, spark_catalog.default.foo3_1d(x#x) AS spark_catalog.default.foo3_1d(c1)#x]
+: +- Aggregate [sum(c2#x) AS sum(c2)#xL]
+: +- Filter (c1#x = outer(x#x))
+: +- SubqueryAlias spark_catalog.default.t2
+: +- View (`spark_catalog`.`default`.`t2`, [c1#x, c2#x])
+: +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+: +- LocalRelation [col1#x, col2#x]
++- Project [c1#x, c2#x, cast(c1#x as int) AS x#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT c1, foo3_1a(foo3_1b(c1), foo3_1b(c1)) FROM t1
+-- !query analysis
+Project [c1#x, spark_catalog.default.foo3_1a(a#x, b#x) AS spark_catalog.default.foo3_1a(spark_catalog.default.foo3_1b(c1), spark_catalog.default.foo3_1b(c1))#x]
++- Project [c1#x, c2#x, x#x, x#x, cast(spark_catalog.default.foo3_1b(x#x) as double) AS a#x, cast(spark_catalog.default.foo3_1b(x#x) as double) AS b#x]
+ +- Project [c1#x, c2#x, cast(c1#x as int) AS x#x, cast(c1#x as int) AS x#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT c1, foo3_1d(foo3_1c(foo3_1b(c1))) FROM t1
+-- !query analysis
+Project [c1#x, spark_catalog.default.foo3_1d(x#x) AS spark_catalog.default.foo3_1d(spark_catalog.default.foo3_1c(spark_catalog.default.foo3_1b(c1)))#x]
+: +- Aggregate [sum(c2#x) AS sum(c2)#xL]
+: +- Filter (c1#x = outer(x#x))
+: +- SubqueryAlias spark_catalog.default.t2
+: +- View (`spark_catalog`.`default`.`t2`, [c1#x, c2#x])
+: +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+: +- LocalRelation [col1#x, col2#x]
++- Project [c1#x, c2#x, x#x, x#x, cast(spark_catalog.default.foo3_1c(x#x) as int) AS x#x]
+ +- Project [c1#x, c2#x, x#x, cast(spark_catalog.default.foo3_1b(x#x) as int) AS x#x]
+ +- Project [c1#x, c2#x, cast(c1#x as int) AS x#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT c1, foo3_1a(foo3_1c(foo3_1b(c1)), foo3_1d(foo3_1b(c1))) FROM t1
+-- !query analysis
+Project [c1#x, spark_catalog.default.foo3_1a(a#x, b#x) AS spark_catalog.default.foo3_1a(spark_catalog.default.foo3_1c(spark_catalog.default.foo3_1b(c1)), spark_catalog.default.foo3_1d(spark_catalog.default.foo3_1b(c1)))#x]
++- Project [c1#x, c2#x, x#x, x#x, x#x, x#x, cast(spark_catalog.default.foo3_1c(x#x) as double) AS a#x, cast(spark_catalog.default.foo3_1d(x#x) as double) AS b#x]
+ : +- Aggregate [sum(c2#x) AS sum(c2)#xL]
+ : +- Filter (c1#x = outer(x#x))
+ : +- SubqueryAlias spark_catalog.default.t2
+ : +- View (`spark_catalog`.`default`.`t2`, [c1#x, c2#x])
+ : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ : +- LocalRelation [col1#x, col2#x]
+ +- Project [c1#x, c2#x, x#x, x#x, cast(spark_catalog.default.foo3_1b(x#x) as int) AS x#x, cast(spark_catalog.default.foo3_1b(x#x) as int) AS x#x]
+ +- Project [c1#x, c2#x, cast(c1#x as int) AS x#x, cast(c1#x as int) AS x#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT foo3_1c(foo3_1e()) FROM t1
+-- !query analysis
+Project [spark_catalog.default.foo3_1c(x#x) AS spark_catalog.default.foo3_1c(spark_catalog.default.foo3_1e())#x]
++- Project [c1#x, c2#x, x#x, cast(spark_catalog.default.foo3_1e() as int) AS x#x]
+ : +- Aggregate [sum(c2#x) AS sum(c2)#xL]
+ : +- Filter (c1#x = outer(x#x))
+ : +- SubqueryAlias spark_catalog.default.t2
+ : +- View (`spark_catalog`.`default`.`t2`, [c1#x, c2#x])
+ : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ : +- LocalRelation [col1#x, col2#x]
+ +- Project [c1#x, c2#x, cast(0 as int) AS x#x]
+ +- Project [c1#x, c2#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT foo3_1a(MAX(c1), MAX(c2)) FROM t1
+-- !query analysis
+Project [spark_catalog.default.foo3_1a(a#x, b#x) AS spark_catalog.default.foo3_1a(max(c1), max(c2))#x]
++- Project [max(c1)#x, max(c2)#x, cast(max(c1)#x as double) AS a#x, cast(max(c2)#x as double) AS b#x]
+ +- Aggregate [max(c1#x) AS max(c1)#x, max(c2#x) AS max(c2)#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT foo3_1a(MAX(c1), c2) FROM t1 GROUP BY c2
+-- !query analysis
+Project [spark_catalog.default.foo3_1a(a#x, b#x) AS spark_catalog.default.foo3_1a(max(c1), c2)#x]
++- Project [max(c1)#x, c2#x, cast(max(c1)#x as double) AS a#x, cast(c2#x as double) AS b#x]
+ +- Aggregate [c2#x], [max(c1#x) AS max(c1)#x, c2#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT foo3_1a(c1, c2) FROM t1 GROUP BY c1, c2
+-- !query analysis
+Project [spark_catalog.default.foo3_1a(a#x, b#x) AS spark_catalog.default.foo3_1a(c1, c2)#x]
++- Project [c1#x, c2#x, cast(c1#x as double) AS a#x, cast(c2#x as double) AS b#x]
+ +- Aggregate [c1#x, c2#x], [c1#x, c2#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT MAX(foo3_1a(c1, c2)) FROM t1 GROUP BY c1, c2
+-- !query analysis
+Project [max(spark_catalog.default.foo3_1a(c1, c2))#x]
++- Aggregate [c1#x, c2#x], [max(spark_catalog.default.foo3_1a(a#x, b#x)) AS max(spark_catalog.default.foo3_1a(c1, c2))#x]
+ +- Project [c1#x, c2#x, cast(c1#x as double) AS a#x, cast(c2#x as double) AS b#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT MAX(c1) + foo3_1b(MAX(c1)) FROM t1 GROUP BY c2
+-- !query analysis
+Project [(max(c1)#x + spark_catalog.default.foo3_1b(x#x)) AS (max(c1) + spark_catalog.default.foo3_1b(max(c1)))#x]
++- Project [max(c1)#x, max(c1)#x, cast(max(c1)#x as int) AS x#x]
+ +- Aggregate [c2#x], [max(c1#x) AS max(c1)#x, max(c1#x) AS max(c1)#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT c1, SUM(foo3_1c(c2)) FROM t1 GROUP BY c1
+-- !query analysis
+Project [c1#x, sum(spark_catalog.default.foo3_1c(c2))#xL]
++- Aggregate [c1#x], [c1#x, sum(spark_catalog.default.foo3_1c(x#x)) AS sum(spark_catalog.default.foo3_1c(c2))#xL]
+ +- Project [c1#x, c2#x, cast(c2#x as int) AS x#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT c1, SUM(foo3_1d(c2)) FROM t1 GROUP BY c1
+-- !query analysis
+Project [c1#x, sum(spark_catalog.default.foo3_1d(c2))#xL]
++- Aggregate [c1#x], [c1#x, sum(spark_catalog.default.foo3_1d(x#x)) AS sum(spark_catalog.default.foo3_1d(c2))#xL]
+ : +- Aggregate [sum(c2#x) AS sum(c2)#xL]
+ : +- Filter (c1#x = outer(x#x))
+ : +- SubqueryAlias spark_catalog.default.t2
+ : +- View (`spark_catalog`.`default`.`t2`, [c1#x, c2#x])
+ : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ : +- LocalRelation [col1#x, col2#x]
+ +- Project [c1#x, c2#x, cast(c2#x as int) AS x#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT foo3_1c(c1), foo3_1d(c1) FROM t1 GROUP BY c1
+-- !query analysis
+Project [spark_catalog.default.foo3_1c(x#x) AS spark_catalog.default.foo3_1c(c1)#x, spark_catalog.default.foo3_1d(x#x) AS spark_catalog.default.foo3_1d(c1)#x]
+: +- Aggregate [sum(c2#x) AS sum(c2)#xL]
+: +- Filter (c1#x = outer(x#x))
+: +- SubqueryAlias spark_catalog.default.t2
+: +- View (`spark_catalog`.`default`.`t2`, [c1#x, c2#x])
+: +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+: +- LocalRelation [col1#x, col2#x]
++- Project [c1#x, c1#x, cast(c1#x as int) AS x#x, cast(c1#x as int) AS x#x]
+ +- Aggregate [c1#x], [c1#x, c1#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT foo3_1a(SUM(c1), rand(0) * 0) FROM t1
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT foo3_1a(SUM(c1) + rand(0) * 0, SUM(c2)) FROM t1
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT foo3_1b(SUM(c1) + rand(0) * 0) FROM t1
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT foo3_1b(SUM(1) + rand(0) * 0) FROM t1 GROUP BY c2
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT foo3_1c(SUM(c2) + rand(0) * 0) FROM t1 GROUP by c1
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT foo3_1b(foo3_1b(MAX(c2))) FROM t1
+-- !query analysis
+Project [spark_catalog.default.foo3_1b(x#x) AS spark_catalog.default.foo3_1b(spark_catalog.default.foo3_1b(max(c2)))#x]
++- Project [max(c2)#x, x#x, cast(spark_catalog.default.foo3_1b(x#x) as int) AS x#x]
+ +- Project [max(c2)#x, cast(max(c2)#x as int) AS x#x]
+ +- Aggregate [max(c2#x) AS max(c2)#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT foo3_1b(MAX(foo3_1b(c2))) FROM t1
+-- !query analysis
+Project [spark_catalog.default.foo3_1b(x#x) AS spark_catalog.default.foo3_1b(max(spark_catalog.default.foo3_1b(c2)))#x]
++- Project [max(spark_catalog.default.foo3_1b(c2))#x, cast(max(spark_catalog.default.foo3_1b(c2))#x as int) AS x#x]
+ +- Aggregate [max(spark_catalog.default.foo3_1b(x#x)) AS max(spark_catalog.default.foo3_1b(c2))#x]
+ +- Project [c1#x, c2#x, cast(c2#x as int) AS x#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT foo3_1a(foo3_1b(c1), MAX(c2)) FROM t1 GROUP BY c1
+-- !query analysis
+Project [spark_catalog.default.foo3_1a(a#x, b#x) AS spark_catalog.default.foo3_1a(spark_catalog.default.foo3_1b(c1), max(c2))#x]
++- Project [c1#x, max(c2)#x, x#x, cast(spark_catalog.default.foo3_1b(x#x) as double) AS a#x, cast(max(c2)#x as double) AS b#x]
+ +- Project [c1#x, max(c2)#x, cast(c1#x as int) AS x#x]
+ +- Aggregate [c1#x], [c1#x, max(c2#x) AS max(c2)#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT c1, foo3_1b(c1) FROM t1 GROUP BY c1
+-- !query analysis
+Project [c1#x, spark_catalog.default.foo3_1b(x#x) AS spark_catalog.default.foo3_1b(c1)#x]
++- Project [c1#x, c1#x, cast(c1#x as int) AS x#x]
+ +- Aggregate [c1#x], [c1#x, c1#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT c1, foo3_1b(c1 + 1) FROM t1 GROUP BY c1
+-- !query analysis
+Project [c1#x, spark_catalog.default.foo3_1b(x#x) AS spark_catalog.default.foo3_1b((c1 + 1))#x]
++- Project [c1#x, (c1 + 1)#x, cast((c1 + 1)#x as int) AS x#x]
+ +- Aggregate [c1#x], [c1#x, (c1#x + 1) AS (c1 + 1)#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT c1, foo3_1b(c1 + rand(0) * 0) FROM t1 GROUP BY c1
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT c1, foo3_1a(c1, MIN(c2)) FROM t1 GROUP BY c1
+-- !query analysis
+Project [c1#x, spark_catalog.default.foo3_1a(a#x, b#x) AS spark_catalog.default.foo3_1a(c1, min(c2))#x]
++- Project [c1#x, c1#x, min(c2)#x, cast(c1#x as double) AS a#x, cast(min(c2)#x as double) AS b#x]
+ +- Aggregate [c1#x], [c1#x, c1#x, min(c2#x) AS min(c2)#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT c1, foo3_1a(c1 + 1, MIN(c2 + 1)) FROM t1 GROUP BY c1
+-- !query analysis
+Project [c1#x, spark_catalog.default.foo3_1a(a#x, b#x) AS spark_catalog.default.foo3_1a((c1 + 1), min((c2 + 1)))#x]
++- Project [c1#x, (c1 + 1)#x, min((c2 + 1))#x, cast((c1 + 1)#x as double) AS a#x, cast(min((c2 + 1))#x as double) AS b#x]
+ +- Aggregate [c1#x], [c1#x, (c1#x + 1) AS (c1 + 1)#x, min((c2#x + 1)) AS min((c2 + 1))#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT c1, c2, foo3_1a(c1, c2) FROM t1 GROUP BY c1, c2
+-- !query analysis
+Project [c1#x, c2#x, spark_catalog.default.foo3_1a(a#x, b#x) AS spark_catalog.default.foo3_1a(c1, c2)#x]
++- Project [c1#x, c2#x, c1#x, c2#x, cast(c1#x as double) AS a#x, cast(c2#x as double) AS b#x]
+ +- Aggregate [c1#x, c2#x], [c1#x, c2#x, c1#x, c2#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT c1, c2, foo3_1a(1, 2) FROM t1 GROUP BY c1, c2
+-- !query analysis
+Project [c1#x, c2#x, spark_catalog.default.foo3_1a(a#x, b#x) AS spark_catalog.default.foo3_1a(1, 2)#x]
++- Project [c1#x, c2#x, 1#x, 2#x, cast(1#x as double) AS a#x, cast(2#x as double) AS b#x]
+ +- Aggregate [c1#x, c2#x], [c1#x, c2#x, 1 AS 1#x, 2 AS 2#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT c1 + c2, foo3_1b(c1 + c2 + 1) FROM t1 GROUP BY c1 + c2
+-- !query analysis
+Project [(c1 + c2)#x, spark_catalog.default.foo3_1b(x#x) AS spark_catalog.default.foo3_1b(((c1 + c2) + 1))#x]
++- Project [(c1 + c2)#x, ((c1 + c2) + 1)#x, cast(((c1 + c2) + 1)#x as int) AS x#x]
+ +- Aggregate [(c1#x + c2#x)], [(c1#x + c2#x) AS (c1 + c2)#x, ((c1#x + c2#x) + 1) AS ((c1 + c2) + 1)#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT COUNT(*) + foo3_1b(c1) + foo3_1b(SUM(c2)) + SUM(foo3_1b(c2)) FROM t1 GROUP BY c1
+-- !query analysis
+Project [(((count(1)#xL + cast(spark_catalog.default.foo3_1b(x#x) as bigint)) + cast(spark_catalog.default.foo3_1b(x#x) as bigint)) + sum(spark_catalog.default.foo3_1b(c2))#xL) AS (((count(1) + spark_catalog.default.foo3_1b(c1)) + spark_catalog.default.foo3_1b(sum(c2))) + sum(spark_catalog.default.foo3_1b(c2)))#xL]
++- Project [count(1)#xL, c1#x, sum(c2)#xL, sum(spark_catalog.default.foo3_1b(c2))#xL, cast(c1#x as int) AS x#x, cast(sum(c2)#xL as int) AS x#x]
+ +- Aggregate [c1#x], [count(1) AS count(1)#xL, c1#x, sum(c2#x) AS sum(c2)#xL, sum(spark_catalog.default.foo3_1b(x#x)) AS sum(spark_catalog.default.foo3_1b(c2))#xL]
+ +- Project [c1#x, c2#x, cast(c2#x as int) AS x#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT c1, COUNT(*), foo3_1b(SUM(c2)) FROM t1 GROUP BY c1 HAVING COUNT(*) > 0
+-- !query analysis
+Filter (count(1)#xL > cast(0 as bigint))
++- Project [c1#x, count(1)#xL, spark_catalog.default.foo3_1b(x#x) AS spark_catalog.default.foo3_1b(sum(c2))#x]
+ +- Project [c1#x, count(1)#xL, sum(c2)#xL, cast(sum(c2)#xL as int) AS x#x]
+ +- Aggregate [c1#x], [c1#x, count(1) AS count(1)#xL, sum(c2#x) AS sum(c2)#xL]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT c1, COUNT(*), foo3_1b(SUM(c2)) FROM t1 GROUP BY c1 HAVING foo3_1b(SUM(c2)) > 0
+-- !query analysis
+Filter (spark_catalog.default.foo3_1b(sum(c2))#x > 0)
++- Project [c1#x, count(1)#xL, spark_catalog.default.foo3_1b(x#x) AS spark_catalog.default.foo3_1b(sum(c2))#x]
+ +- Project [c1#x, count(1)#xL, sum(c2)#xL, cast(sum(c2)#xL as int) AS x#x]
+ +- Aggregate [c1#x], [c1#x, count(1) AS count(1)#xL, sum(c2#x) AS sum(c2)#xL]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT c1, COUNT(*), foo3_1b(SUM(c2)) FROM t1 GROUP BY c1 HAVING SUM(foo3_1b(c2)) > 0
+-- !query analysis
+Project [c1#x, count(1)#xL, spark_catalog.default.foo3_1b(sum(c2))#x]
++- Filter (sum(spark_catalog.default.foo3_1b(c2))#xL > cast(0 as bigint))
+ +- Project [c1#x, count(1)#xL, spark_catalog.default.foo3_1b(x#x) AS spark_catalog.default.foo3_1b(sum(c2))#x, sum(spark_catalog.default.foo3_1b(c2))#xL]
+ +- Project [c1#x, count(1)#xL, sum(c2)#xL, sum(spark_catalog.default.foo3_1b(c2))#xL, cast(sum(c2)#xL as int) AS x#x]
+ +- Aggregate [c1#x], [c1#x, count(1) AS count(1)#xL, sum(c2#x) AS sum(c2)#xL, sum(spark_catalog.default.foo3_1b(x#x)) AS sum(spark_catalog.default.foo3_1b(c2))#xL]
+ +- Project [c1#x, c2#x, cast(c2#x as int) AS x#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT foo3_1b(c1), MIN(c2) FROM t1 GROUP BY 1
+-- !query analysis
+Project [spark_catalog.default.foo3_1b(c1)#x, min(c2)#x]
++- Aggregate [spark_catalog.default.foo3_1b#x], [spark_catalog.default.foo3_1b#x AS spark_catalog.default.foo3_1b(c1)#x, min(c2#x) AS min(c2)#x]
+ +- Project [c1#x, c2#x, spark_catalog.default.foo3_1b(x#x) AS spark_catalog.default.foo3_1b#x]
+ +- Project [c1#x, c2#x, cast(c1#x as int) AS x#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT foo3_1a(c1 + rand(0) * 0, c2) FROM t1 GROUP BY 1
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT c1, c2, foo3_1a(c1, c2) FROM t1 GROUP BY c1, c2, 3
+-- !query analysis
+Project [c1#x, c2#x, spark_catalog.default.foo3_1a(c1, c2)#x]
++- Aggregate [c1#x, c2#x, spark_catalog.default.foo3_1a#x], [c1#x, c2#x, spark_catalog.default.foo3_1a#x AS spark_catalog.default.foo3_1a(c1, c2)#x]
+ +- Project [c1#x, c2#x, spark_catalog.default.foo3_1a(a#x, b#x) AS spark_catalog.default.foo3_1a#x]
+ +- Project [c1#x, c2#x, cast(c1#x as double) AS a#x, cast(c2#x as double) AS b#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT c1, (SELECT c1), (SELECT foo3_1b(c1)), SUM(c2) FROM t1 GROUP BY 1, 2, 3
+-- !query analysis
+Aggregate [c1#x, scalar-subquery#x [c1#x], scalar-subquery#x [c1#x]], [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#x, sum(c2#x) AS sum(c2)#xL]
+: :- Project [outer(c1#x)]
+: : +- OneRowRelation
+: :- Project [spark_catalog.default.foo3_1b(x#x) AS spark_catalog.default.foo3_1b(outer(spark_catalog.default.t1.c1))#x]
+: : +- Project [cast(outer(c1#x) as int) AS x#x]
+: : +- OneRowRelation
+: :- Project [outer(c1#x)]
+: : +- OneRowRelation
+: +- Project [spark_catalog.default.foo3_1b(x#x) AS spark_catalog.default.foo3_1b(outer(spark_catalog.default.t1.c1))#x]
+: +- Project [cast(outer(c1#x) as int) AS x#x]
+: +- OneRowRelation
++- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT c1, SUM(c2) + foo3_1a(MIN(c2), MAX(c2)) + (SELECT SUM(c2)) FROM t1 GROUP BY c1
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE",
+ "sqlState" : "0A000",
+ "messageParameters" : {
+ "sqlExprs" : "\"sum(c2) AS `sum(outer(spark_catalog.default.t1.c2))`\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 51,
+ "stopIndex" : 64,
+ "fragment" : "SELECT SUM(c2)"
+ } ]
+}
+
+
+-- !query
+SELECT foo3_1b(SUM(c1)) + (SELECT foo3_1b(SUM(c1))) FROM t1
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE",
+ "sqlState" : "0A000",
+ "messageParameters" : {
+ "sqlExprs" : "\"sum(c1) AS `sum(outer(spark_catalog.default.t1.c1))`\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 27,
+ "stopIndex" : 51,
+ "fragment" : "(SELECT foo3_1b(SUM(c1)))"
+ } ]
+}
+
+
+-- !query
+SELECT SUM(foo3_1b(SUM(c1))) FROM t1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "NESTED_AGGREGATE_FUNCTION",
+ "sqlState" : "42607",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 12,
+ "stopIndex" : 27,
+ "fragment" : "foo3_1b(SUM(c1))"
+ } ]
+}
+
+
+-- !query
+SELECT foo3_1b(SUM(c1)) + (SELECT SUM(SUM(c1))) FROM t1
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "NESTED_AGGREGATE_FUNCTION",
+ "sqlState" : "42607",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 39,
+ "stopIndex" : 45,
+ "fragment" : "SUM(c1)"
+ } ]
+}
+
+
+-- !query
+SELECT foo3_1b(SUM(c1) + SUM(SUM(c1))) FROM t1
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "NESTED_AGGREGATE_FUNCTION",
+ "sqlState" : "42607",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 30,
+ "stopIndex" : 36,
+ "fragment" : "SUM(c1)"
+ } ]
+}
+
+
+-- !query
+SELECT foo3_1b(SUM(c1 + rand(0) * 0)) FROM t1
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "AGGREGATE_FUNCTION_WITH_NONDETERMINISTIC_EXPRESSION",
+ "sqlState" : "42845",
+ "messageParameters" : {
+ "sqlExpr" : "\"sum((c1 + (rand(0) * 0)))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 20,
+ "stopIndex" : 35,
+ "fragment" : "c1 + rand(0) * 0"
+ } ]
+}
+
+
+-- !query
+SELECT SUM(foo3_1b(c1) + rand(0) * 0) FROM t1
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "AGGREGATE_FUNCTION_WITH_NONDETERMINISTIC_EXPRESSION",
+ "sqlState" : "42845",
+ "messageParameters" : {
+ "sqlExpr" : "\"sum((spark_catalog.default.foo3_1b(foo3_1b.x) + (rand(0) * 0)))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 12,
+ "stopIndex" : 36,
+ "fragment" : "foo3_1b(c1) + rand(0) * 0"
+ } ]
+}
+
+
+-- !query
+SELECT SUM(foo3_1b(c1 + rand(0) * 0)) FROM t1
+-- !query analysis
+[Analyzer test output redacted due to nondeterminism]
+
+
+-- !query
+SELECT foo3_1b(SUM(c1) + foo3_1b(SUM(c1))) FROM t1
+-- !query analysis
+Project [spark_catalog.default.foo3_1b(x#x) AS spark_catalog.default.foo3_1b((sum(c1) + spark_catalog.default.foo3_1b(sum(c1))))#x]
++- Project [sum(c1)#xL, sum(c1)#xL, x#x, cast((sum(c1)#xL + cast(spark_catalog.default.foo3_1b(x#x) as bigint)) as int) AS x#x]
+ +- Project [sum(c1)#xL, sum(c1)#xL, cast(sum(c1)#xL as int) AS x#x]
+ +- Aggregate [sum(c1#x) AS sum(c1)#xL, sum(c1#x) AS sum(c1)#xL]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT foo3_1b(SUM(c2) + foo3_1b(SUM(c1))) AS foo FROM t1 HAVING foo > 0
+-- !query analysis
+Filter (foo#x > 0)
++- Project [spark_catalog.default.foo3_1b(x#x) AS foo#x]
+ +- Project [sum(c2)#xL, sum(c1)#xL, x#x, cast((sum(c2)#xL + cast(spark_catalog.default.foo3_1b(x#x) as bigint)) as int) AS x#x]
+ +- Project [sum(c2)#xL, sum(c1)#xL, cast(sum(c1)#xL as int) AS x#x]
+ +- Aggregate [sum(c2#x) AS sum(c2)#xL, sum(c1#x) AS sum(c1)#xL]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT c1, COUNT(*), foo3_1b(SUM(c2) + foo3_1b(SUM(c2))) FROM t1 GROUP BY c1 HAVING COUNT(*) > 0
+-- !query analysis
+Filter (count(1)#xL > cast(0 as bigint))
++- Project [c1#x, count(1)#xL, spark_catalog.default.foo3_1b(x#x) AS spark_catalog.default.foo3_1b((sum(c2) + spark_catalog.default.foo3_1b(sum(c2))))#x]
+ +- Project [c1#x, count(1)#xL, sum(c2)#xL, sum(c2)#xL, x#x, cast((sum(c2)#xL + cast(spark_catalog.default.foo3_1b(x#x) as bigint)) as int) AS x#x]
+ +- Project [c1#x, count(1)#xL, sum(c2)#xL, sum(c2)#xL, cast(sum(c2)#xL as int) AS x#x]
+ +- Aggregate [c1#x], [c1#x, count(1) AS count(1)#xL, sum(c2#x) AS sum(c2)#xL, sum(c2#x) AS sum(c2)#xL]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT foo3_1a(c1, MAX(c2)) FROM t1 GROUP BY c1, 1
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "GROUP_BY_POS_AGGREGATE",
+ "sqlState" : "42903",
+ "messageParameters" : {
+ "aggExpr" : "spark_catalog.default.foo3_1a(spark_catalog.default.t1.c1, max(spark_catalog.default.t1.c2)) AS `spark_catalog.default.foo3_1a(c1, max(c2))`",
+ "index" : "1"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 50,
+ "stopIndex" : 50,
+ "fragment" : "1"
+ } ]
+}
+
+
+-- !query
+WITH cte AS (SELECT foo3_1a(c1, c2) FROM t1)
+SELECT * FROM cte
+-- !query analysis
+WithCTE
+:- CTERelationDef xxxx, false
+: +- SubqueryAlias cte
+: +- Project [spark_catalog.default.foo3_1a(a#x, b#x) AS spark_catalog.default.foo3_1a(c1, c2)#x]
+: +- Project [c1#x, c2#x, cast(c1#x as double) AS a#x, cast(c2#x as double) AS b#x]
+: +- SubqueryAlias spark_catalog.default.t1
+: +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+: +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+: +- LocalRelation [col1#x, col2#x]
++- Project [spark_catalog.default.foo3_1a(c1, c2)#x]
+ +- SubqueryAlias cte
+ +- CTERelationRef xxxx, true, [spark_catalog.default.foo3_1a(c1, c2)#x], false, false
+
+
+-- !query
+SELECT SUM(c2) FROM t1 GROUP BY foo3_1b(c1)
+-- !query analysis
+Project [sum(c2)#xL]
++- Aggregate [spark_catalog.default.foo3_1b#x], [sum(c2#x) AS sum(c2)#xL]
+ +- Project [c1#x, c2#x, spark_catalog.default.foo3_1b(x#x) AS spark_catalog.default.foo3_1b#x]
+ +- Project [c1#x, c2#x, cast(c1#x as int) AS x#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT foo3_1b(c1), SUM(c2) FROM t1 GROUP BY 1
+-- !query analysis
+Project [spark_catalog.default.foo3_1b(c1)#x, sum(c2)#xL]
++- Aggregate [spark_catalog.default.foo3_1b#x], [spark_catalog.default.foo3_1b#x AS spark_catalog.default.foo3_1b(c1)#x, sum(c2#x) AS sum(c2)#xL]
+ +- Project [c1#x, c2#x, spark_catalog.default.foo3_1b(x#x) AS spark_catalog.default.foo3_1b#x]
+ +- Project [c1#x, c2#x, cast(c1#x as int) AS x#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT foo3_1b(c1), c2, GROUPING(foo3_1b(c1)), SUM(c1) FROM t1 GROUP BY ROLLUP(foo3_1b(c1), c2)
+-- !query analysis
+Aggregate [spark_catalog.default.foo3_1b(c1)#x, c2#x, spark_grouping_id#xL], [spark_catalog.default.foo3_1b(c1)#x AS spark_catalog.default.foo3_1b(c1)#x, c2#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(spark_catalog.default.foo3_1b(c1))#x, sum(c1#x) AS sum(c1)#xL]
++- Expand [[c1#x, c2#x, spark_catalog.default.foo3_1b(c1)#x, c2#x, 0], [c1#x, c2#x, spark_catalog.default.foo3_1b(c1)#x, null, 1], [c1#x, c2#x, null, null, 3]], [c1#x, c2#x, spark_catalog.default.foo3_1b(c1)#x, c2#x, spark_grouping_id#xL]
+ +- Project [c1#x, c2#x, spark_catalog.default.foo3_1b(x#x) AS spark_catalog.default.foo3_1b(c1)#x, c2#x AS c2#x]
+ +- Project [c1#x, c2#x, cast(c1#x as int) AS x#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT c1, SUM(c2) FROM t1 GROUP BY c1 HAVING foo3_1b(SUM(c2)) > 1
+-- !query analysis
+Project [c1#x, sum(c2)#xL]
++- Filter (spark_catalog.default.foo3_1b(x#x) > 1)
+ +- Project [c1#x, sum(c2)#xL, cast(sum(c2)#xL as int) AS x#x]
+ +- Aggregate [c1#x], [c1#x, sum(c2#x) AS sum(c2)#xL]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT c1, SUM(c2) FROM t1 GROUP BY CUBE(c1) HAVING foo3_1b(GROUPING(c1)) = 0
+-- !query analysis
+Project [c1#x, sum(c2)#xL]
++- Project [c1#x, sum(c2)#xL, spark_grouping_id#xL]
+ +- Filter (spark_catalog.default.foo3_1b(x#x) = 0)
+ +- Project [c1#x, sum(c2)#xL, spark_grouping_id#xL, cast(cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) as int) AS x#x]
+ +- Aggregate [c1#x, spark_grouping_id#xL], [c1#x, sum(c2#x) AS sum(c2)#xL, spark_grouping_id#xL]
+ +- Expand [[c1#x, c2#x, c1#x, 0], [c1#x, c2#x, null, 1]], [c1#x, c2#x, c1#x, spark_grouping_id#xL]
+ +- Project [c1#x, c2#x, c1#x AS c1#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT * FROM t1 JOIN t2 ON foo3_1a(t1.c1, t2.c2) >= 2
+-- !query analysis
+Project [c1#x, c2#x, c1#x, c2#x]
++- Join Inner, (spark_catalog.default.foo3_1a(a#x, b#x) >= cast(2 as double))
+ :- SubqueryAlias spark_catalog.default.t1
+ : +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ : +- LocalRelation [col1#x, col2#x]
+ +- SubqueryAlias spark_catalog.default.t2
+ +- View (`spark_catalog`.`default`.`t2`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT * FROM t1 JOIN t2 ON foo3_1b(t1.c2) = foo3_1b(t2.c2)
+-- !query analysis
+Project [c1#x, c2#x, c1#x, c2#x]
++- Join Inner, (spark_catalog.default.foo3_1b(x#x) = spark_catalog.default.foo3_1b(x#x))
+ :- SubqueryAlias spark_catalog.default.t1
+ : +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ : +- LocalRelation [col1#x, col2#x]
+ +- SubqueryAlias spark_catalog.default.t2
+ +- View (`spark_catalog`.`default`.`t2`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT * FROM t1 JOIN t2 ON foo3_1b(t1.c1 + t2.c1 + 2) > 2
+-- !query analysis
+Project [c1#x, c2#x, c1#x, c2#x]
++- Join Inner, (spark_catalog.default.foo3_1b(x#x) > 2)
+ :- SubqueryAlias spark_catalog.default.t1
+ : +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ : +- LocalRelation [col1#x, col2#x]
+ +- SubqueryAlias spark_catalog.default.t2
+ +- View (`spark_catalog`.`default`.`t2`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT * FROM t1 JOIN t2 ON foo3_1a(foo3_1b(t1.c1), t2.c2) >= 2
+-- !query analysis
+Project [c1#x, c2#x, c1#x, c2#x]
++- Join Inner, (spark_catalog.default.foo3_1a(a#x, b#x) >= cast(2 as double))
+ :- SubqueryAlias spark_catalog.default.t1
+ : +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ : +- LocalRelation [col1#x, col2#x]
+ +- SubqueryAlias spark_catalog.default.t2
+ +- View (`spark_catalog`.`default`.`t2`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT * FROM t1 JOIN t2 ON foo3_1f() > 0
+-- !query analysis
+Project [c1#x, c2#x, c1#x, c2#x]
++- Join Inner, (spark_catalog.default.foo3_1f() > 0)
+ : +- Aggregate [sum(c2#x) AS sum(c2)#xL]
+ : +- Filter (c1#x = 0)
+ : +- SubqueryAlias spark_catalog.default.t2
+ : +- View (`spark_catalog`.`default`.`t2`, [c1#x, c2#x])
+ : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ : +- LocalRelation [col1#x, col2#x]
+ :- SubqueryAlias spark_catalog.default.t1
+ : +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ : +- LocalRelation [col1#x, col2#x]
+ +- SubqueryAlias spark_catalog.default.t2
+ +- View (`spark_catalog`.`default`.`t2`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT * FROM t1 JOIN t2 ON foo3_1b(t1.c1 + rand(0) * 0) > 1
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "INVALID_NON_DETERMINISTIC_EXPRESSIONS",
+ "sqlState" : "42K0E",
+ "messageParameters" : {
+ "sqlExprs" : "\"(spark_catalog.default.foo3_1b(foo3_1b.x) > 1)\""
+ }
+}
+
+
+-- !query
+SELECT * FROM t1 JOIN t2 ON foo3_1c(t1.c1) = 2
+-- !query analysis
+Project [c1#x, c2#x, c1#x, c2#x]
++- Join Inner, (spark_catalog.default.foo3_1c(x#x) = 2)
+ :- SubqueryAlias spark_catalog.default.t1
+ : +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ : +- LocalRelation [col1#x, col2#x]
+ +- SubqueryAlias spark_catalog.default.t2
+ +- View (`spark_catalog`.`default`.`t2`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT * FROM t1 JOIN t2 ON foo3_1g(t1.c1) = 2
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.UNSUPPORTED_CORRELATED_SCALAR_SUBQUERY",
+ "sqlState" : "0A000",
+ "messageParameters" : {
+ "treeNode" : "Join Inner, (spark_catalog.default.foo3_1g(x#x) = 2)\n: +- Project [outer(x#x)]\n: +- OneRowRelation\n:- SubqueryAlias spark_catalog.default.t1\n: +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])\n: +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]\n: +- LocalRelation [col1#x, col2#x]\n+- SubqueryAlias spark_catalog.default.t2\n +- View (`spark_catalog`.`default`.`t2`, [c1#x, c2#x])\n +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]\n +- LocalRelation [col1#x, col2#x]\n"
+ }
+}
+
+
+-- !query
+SELECT * FROM t1 ORDER BY foo3_1b(c1)
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNSUPPORTED_SQL_UDF_USAGE",
+ "sqlState" : "0A000",
+ "messageParameters" : {
+ "functionName" : "`spark_catalog`.`default`.`foo3_1b`",
+ "nodeName" : "Sort"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 27,
+ "stopIndex" : 37,
+ "fragment" : "foo3_1b(c1)"
+ } ]
+}
+
+
+-- !query
+SELECT * FROM t1 LIMIT foo3_1b(1)
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNSUPPORTED_SQL_UDF_USAGE",
+ "sqlState" : "0A000",
+ "messageParameters" : {
+ "functionName" : "`spark_catalog`.`default`.`foo3_1b`",
+ "nodeName" : "GlobalLimit"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 24,
+ "stopIndex" : 33,
+ "fragment" : "foo3_1b(1)"
+ } ]
+}
+
+
+-- !query
+SELECT * FROM ta LATERAL VIEW EXPLODE(ARRAY(foo3_1b(x[0]), foo3_1b(x[1]))) AS t
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNSUPPORTED_SQL_UDF_USAGE",
+ "sqlState" : "0A000",
+ "messageParameters" : {
+ "functionName" : "`spark_catalog`.`default`.`foo3_1b`",
+ "nodeName" : "Generate"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 45,
+ "stopIndex" : 57,
+ "fragment" : "foo3_1b(x[0])"
+ } ]
+}
+
+
+-- !query
+SELECT CASE WHEN foo3_1b(rand(0) * 0 < 1 THEN 1 ELSE -1 END
+-- !query analysis
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'foo3_1b'",
+ "hint" : ""
+ }
+}
+
+
+-- !query
+SELECT (SELECT SUM(c2) FROM t2 WHERE c1 = foo3_1b(t1.c1)) FROM t1
+-- !query analysis
+Project [scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL]
+: +- Aggregate [sum(c2#x) AS sum(c2)#xL]
+: +- Project [c1#x, c2#x]
+: +- Filter (c1#x = spark_catalog.default.foo3_1b(x#x))
+: +- Project [c1#x, c2#x, cast(outer(c1#x) as int) AS x#x]
+: +- SubqueryAlias spark_catalog.default.t2
+: +- View (`spark_catalog`.`default`.`t2`, [c1#x, c2#x])
+: +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+: +- LocalRelation [col1#x, col2#x]
++- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT foo3_1b((SELECT SUM(c1) FROM t1))
+-- !query analysis
+Project [spark_catalog.default.foo3_1b(x#x) AS spark_catalog.default.foo3_1b(scalarsubquery())#x]
++- Project [cast(scalar-subquery#x [] as int) AS x#x]
+ : +- Aggregate [sum(c1#x) AS sum(c1)#xL]
+ : +- SubqueryAlias spark_catalog.default.t1
+ : +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ : +- LocalRelation [col1#x, col2#x]
+ +- OneRowRelation
+
+
+-- !query
+SELECT foo3_1a(c1, (SELECT MIN(c1) FROM t1)) FROM t1
+-- !query analysis
+Project [spark_catalog.default.foo3_1a(a#x, b#x) AS spark_catalog.default.foo3_1a(c1, scalarsubquery())#x]
++- Project [c1#x, c2#x, cast(c1#x as double) AS a#x, cast(scalar-subquery#x [] as double) AS b#x]
+ : +- Aggregate [min(c1#x) AS min(c1)#x]
+ : +- SubqueryAlias spark_catalog.default.t1
+ : +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ : +- LocalRelation [col1#x, col2#x]
+ +- SubqueryAlias spark_catalog.default.t1
+ +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT foo3_1b((SELECT SUM(c1))) FROM t1
+-- !query analysis
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE",
+ "sqlState" : "0A000",
+ "messageParameters" : {
+ "sqlExprs" : "\"sum(c1) AS `sum(outer(spark_catalog.default.t1.c1))`\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 30,
+ "fragment" : "SELECT SUM(c1)"
+ } ]
+}
+
+
+-- !query
+SELECT foo3_1b((SELECT SUM(c1) FROM t1 WHERE c2 = t2.c2)) FROM t2
+-- !query analysis
+Project [spark_catalog.default.foo3_1b(x#x) AS spark_catalog.default.foo3_1b(scalarsubquery(c2))#x]
++- Project [c1#x, c2#x, cast(scalar-subquery#x [c2#x] as int) AS x#x]
+ : +- Aggregate [sum(c1#x) AS sum(c1)#xL]
+ : +- Filter (c2#x = outer(c2#x))
+ : +- SubqueryAlias spark_catalog.default.t1
+ : +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ : +- LocalRelation [col1#x, col2#x]
+ +- SubqueryAlias spark_catalog.default.t2
+ +- View (`spark_catalog`.`default`.`t2`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+SELECT c2, AVG(foo3_1b((SELECT COUNT(*) FROM t1 WHERE c2 = t2.c2))) OVER (PARTITION BY c1) AS r FROM t2
+-- !query analysis
+Project [c2#x, r#x]
++- Project [c2#x, _w0#x, c1#x, r#x, r#x]
+ +- Window [avg(_w0#x) windowspecdefinition(c1#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS r#x], [c1#x]
+ +- Project [c2#x, spark_catalog.default.foo3_1b(x#x) AS _w0#x, c1#x]
+ +- Project [c1#x, c2#x, cast(scalar-subquery#x [c2#x] as int) AS x#x]
+ : +- Aggregate [count(1) AS count(1)#xL]
+ : +- Filter (c2#x = outer(c2#x))
+ : +- SubqueryAlias spark_catalog.default.t1
+ : +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])
+ : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ : +- LocalRelation [col1#x, col2#x]
+ +- SubqueryAlias spark_catalog.default.t2
+ +- View (`spark_catalog`.`default`.`t2`, [c1#x, c2#x])
+ +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+CREATE FUNCTION foo3_1x(x STRUCT) RETURNS INT RETURN x.a + x.b
+-- !query analysis
+CreateSQLFunctionCommand spark_catalog.default.foo3_1x, x STRUCT, INT, x.a + x.b, false, false, false, false
+
+
+-- !query
+CREATE FUNCTION foo3_1y(x ARRAY) RETURNS INT RETURN aggregate(x, BIGINT(0), (x, y) -> x + y)
+-- !query analysis
+CreateSQLFunctionCommand spark_catalog.default.foo3_1y, x ARRAY, INT, aggregate(x, BIGINT(0), (x, y) -> x + y), false, false, false, false
+
+
+-- !query
+SELECT foo3_1a(x.a, x.b) FROM ts
+-- !query analysis
+Project [spark_catalog.default.foo3_1a(a#x, b#x) AS spark_catalog.default.foo3_1a(x.a, x.b)#x]
++- Project [x#x, cast(x#x.a as double) AS a#x, cast(x#x.b as double) AS b#x]
+ +- SubqueryAlias spark_catalog.default.ts
+ +- View (`spark_catalog`.`default`.`ts`, [x#x])
+ +- Project [cast(col1#x as struct) AS x#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT foo3_1x(x) FROM ts
+-- !query analysis
+Project [spark_catalog.default.foo3_1x(x#x) AS spark_catalog.default.foo3_1x(x)#x]
++- Project [x#x, cast(x#x as struct) AS x#x]
+ +- SubqueryAlias spark_catalog.default.ts
+ +- View (`spark_catalog`.`default`.`ts`, [x#x])
+ +- Project [cast(col1#x as struct) AS x#x]
+ +- LocalRelation [col1#x]
+
+
+-- !query
+SELECT foo3_1a(x['a'], x['b']) FROM tm
+-- !query analysis
+Project [spark_catalog.default.foo3_1a(a#x, b#x) AS spark_catalog.default.foo3_1a(x[a], x[b])#x]
++- Project [x#x, cast(x#x[a] as double) AS a#x, cast(x#x[b] as double) AS b#x]
+ +- SubqueryAlias spark_catalog.default.tm
+ +- View (`spark_catalog`.`default`.`tm`, [x#x])
+ +- Project [cast(col1#x as map