diff --git a/datagen/src/main/spark400/scala/org/apache/spark/sql/tests/datagen/DataGenExprShims.scala b/datagen/src/main/spark400/scala/org/apache/spark/sql/tests/datagen/DataGenExprShims.scala index 54b7b1a28cd..05921832f76 100644 --- a/datagen/src/main/spark400/scala/org/apache/spark/sql/tests/datagen/DataGenExprShims.scala +++ b/datagen/src/main/spark400/scala/org/apache/spark/sql/tests/datagen/DataGenExprShims.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/dist/scripts/binary-dedupe.sh b/dist/scripts/binary-dedupe.sh index 6813818bcd1..fcaa9583077 100755 --- a/dist/scripts/binary-dedupe.sh +++ b/dist/scripts/binary-dedupe.sh @@ -169,8 +169,15 @@ function verify_same_sha_for_unshimmed() { # but it is compatible with previous versions because it merely adds a new method. # we might need to replace this strict check with MiMa # https://github.com/apache/spark/blob/7011706a0a8dbec6adb5b5b121921b29b314335f/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala#L75-L95 + # ProxyRapidsShuffleInternalManagerBase is not bitwise-identical when + # DB 17.3 is included because ShuffleManager.getReader signature differs + # (8-param with prismMapStatusEnabled vs 7-param). This is safe because + # the class provides concrete implementations for ALL getReader variants, + # so the JVM resolves the correct one at runtime regardless of which + # ShuffleManager version the class was compiled against. if [[ ! "$class_file_quoted" =~ com/nvidia/spark/rapids/spark[34].*/.*ShuffleManager.class && \ - "$class_file_quoted" != "com/nvidia/spark/ParquetCachedBatchSerializer.class" ]]; then + "$class_file_quoted" != "com/nvidia/spark/ParquetCachedBatchSerializer.class" && \ + ! "$class_file_quoted" =~ org/apache/spark/sql/rapids/ProxyRapidsShuffleInternalManagerBase ]]; then if ! grep -q "/spark.\+/$class_file_quoted" "$SPARK_SHARED_TXT"; then echo >&2 "$class_file is not bitwise-identical across shims" exit 255 diff --git a/integration_tests/src/main/python/array_test.py b/integration_tests/src/main/python/array_test.py index 709ccd6d96b..b91e3c70c89 100644 --- a/integration_tests/src/main/python/array_test.py +++ b/integration_tests/src/main/python/array_test.py @@ -152,8 +152,7 @@ def test_array_item_with_strict_index(strict_index_enabled, index): @pytest.mark.parametrize('index', [-2, 100, array_neg_index_gen, array_out_index_gen], ids=idfn) @disable_ansi_mode def test_array_item_ansi_fail_invalid_index(index): - message = "SparkArrayIndexOutOfBoundsException" if (is_databricks104_or_later() or is_spark_330_or_later() and is_before_spark_400()) else \ - "ArrayIndexOutOfBoundsException" + message = "ArrayIndexOutOfBoundsException" if isinstance(index, int): test_func = lambda spark: unary_op_df(spark, ArrayGen(int_gen)).select(col('a')[index]).collect() else: diff --git a/integration_tests/src/test/spark400/scala/org/apache/spark/sql/rapids/shims/TrampolineUtilShim.scala b/integration_tests/src/test/spark400/scala/org/apache/spark/sql/rapids/shims/TrampolineUtilShim.scala index d643a7978cb..8ddf35a4bf4 100644 --- a/integration_tests/src/test/spark400/scala/org/apache/spark/sql/rapids/shims/TrampolineUtilShim.scala +++ b/integration_tests/src/test/spark400/scala/org/apache/spark/sql/rapids/shims/TrampolineUtilShim.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/jenkins/Jenkinsfile-blossom.premerge-databricks b/jenkins/Jenkinsfile-blossom.premerge-databricks index 55464be368b..8cf82b80301 100644 --- a/jenkins/Jenkinsfile-blossom.premerge-databricks +++ b/jenkins/Jenkinsfile-blossom.premerge-databricks @@ -93,7 +93,7 @@ pipeline { // 'name' and 'value' only supprt literal string in the declarative Jenkins // Refer to Jenkins issue https://issues.jenkins.io/browse/JENKINS-62127 name 'DB_RUNTIME' - values '12.2', '13.3', '14.3' + values '12.2', '13.3', '14.3', '17.3' } } stages { @@ -184,13 +184,16 @@ void databricksBuild() { } } - stage("Test agaist $SPARK_MAJOR DB") { - dbStep = 'TEST' - withCredentials([file(credentialsId: 'SPARK_DATABRICKS_PRIVKEY', variable: 'DATABRICKS_PRIVKEY')]) { - def TEST_PARAMS = DbUtils.getParameters(this, dbStep, "-c $CLUSTER_ID") - // Get built tar from the path "$testArgs" on Databricks cluster - // Refer to https://github.com/NVIDIA/spark-rapids/pull/11788/files#diff-db28879431d57d0e454a2c7ee89fdda9abdec463c61771333d6a6565bf96c062R52-R55 - sh "python3 ./jenkins/databricks/run-tests.py $TEST_PARAMS $testArgs" + // TODO: Temporarily skip tests on Databricks 17.3 until the test failures are fixed + if (env.DB_RUNTIME != '17.3') { + stage("Test agaist $SPARK_MAJOR DB") { + dbStep = 'TEST' + withCredentials([file(credentialsId: 'SPARK_DATABRICKS_PRIVKEY', variable: 'DATABRICKS_PRIVKEY')]) { + def TEST_PARAMS = DbUtils.getParameters(this, dbStep, "-c $CLUSTER_ID") + // Get built tar from the path "$testArgs" on Databricks cluster + // Refer to https://github.com/NVIDIA/spark-rapids/pull/11788/files#diff-db28879431d57d0e454a2c7ee89fdda9abdec463c61771333d6a6565bf96c062R52-R55 + sh "python3 ./jenkins/databricks/run-tests.py $TEST_PARAMS $testArgs" + } } } } finally { diff --git a/jenkins/databricks/build.sh b/jenkins/databricks/build.sh index dcb2621b57f..4f6b1bcd5d8 100755 --- a/jenkins/databricks/build.sh +++ b/jenkins/databricks/build.sh @@ -119,11 +119,19 @@ initialize() # the version of Spark used when we install the Databricks jars in .m2 BASE_SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS=${BASE_SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS:-$BASE_SPARK_VERSION} SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS=${BASE_SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS}-databricks + + # Determine Scala version based on Spark version + # Spark 4.0+ uses Scala 2.13, earlier versions use 2.12 + if [[ "$BASE_SPARK_VERSION" == 4.* ]]; then + export SCALA_BINARY_VER=2.13 + fi + DBR_VER=$(cat /databricks/DBR_VERSION) - if [ $DBR_VER == '14.3' ]; then + if [ $DBR_VER == '14.3' ] || [ $DBR_VER == '17.3' ]; then DBR_VER=$(echo $DBR_VER | sed 's/\.//g') # We are appending 143 in addition to the base spark version because Databricks 14.3 - # and Databricks 15.4 are both based on spark version 3.5.0 + # and Databricks 15.4 are both based on spark version 3.5.0. Similarly, we are appending 173 + # for Databricks 17.3 based on Spark 4.0.0. BUILDVER="$BUILDVER$DBR_VER" SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS="$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS-$DBR_VER" fi @@ -145,10 +153,18 @@ initialize() export WORKSPACE=$PWD # set the retry count for mvn commands MVN_CMD="mvn -Dmaven.wagon.http.retryHandler.count=3" + + # Determine which pom to use based on Scala version + if [[ "$SCALA_BINARY_VER" == "2.13" ]]; then + POM_FILE="scala2.13/pom.xml" + else + POM_FILE="pom.xml" + fi + # getting the versions of CUDA, SCALA and SPARK_PLUGIN - SPARK_PLUGIN_JAR_VERSION=$($MVN_CMD help:evaluate -q -pl dist -Dexpression=project.version -DforceStdout) - SCALA_VERSION=$($MVN_CMD help:evaluate -q -pl dist -Dexpression=scala.binary.version -DforceStdout) - CUDA_VERSION=$($MVN_CMD help:evaluate -q -pl dist -Dexpression=cuda.version -DforceStdout) + SPARK_PLUGIN_JAR_VERSION=$($MVN_CMD help:evaluate -q -f $POM_FILE -pl dist -Dexpression=project.version -DforceStdout) + SCALA_VERSION=$($MVN_CMD help:evaluate -q -f $POM_FILE -pl dist -Dexpression=scala.binary.version -DforceStdout) + CUDA_VERSION=$($MVN_CMD help:evaluate -q -f $POM_FILE -pl dist -Dexpression=cuda.version -DforceStdout) RAPIDS_BUILT_JAR=rapids-4-spark_$SCALA_VERSION-$SPARK_PLUGIN_JAR_VERSION.jar # If set to 1, skips installing dependencies into mvn repo. SKIP_DEP_INSTALL=${SKIP_DEP_INSTALL:-'0'} @@ -228,18 +244,24 @@ else fi # Build the RAPIDS plugin by running package command for databricks -$MVN_CMD -B -Ddatabricks -Dbuildver=$BUILDVER $MVN_PHASES -DskipTests $MVN_OPT +$MVN_CMD -B -f $POM_FILE -Ddatabricks -Dbuildver=$BUILDVER $MVN_PHASES -DskipTests $MVN_OPT if [[ "$WITH_DEFAULT_UPSTREAM_SHIM" != "0" ]]; then echo "Building the default Spark shim and creating a two-shim dist jar" - UPSTREAM_BUILDVER=$($MVN_CMD help:evaluate -q -pl dist -Dexpression=buildver -DforceStdout) - $MVN_CMD -B package -pl dist -am -DskipTests -Dmaven.scaladoc.skip $MVN_OPT \ + UPSTREAM_BUILDVER=$($MVN_CMD help:evaluate -q -f $POM_FILE -pl dist -Dexpression=buildver -DforceStdout) + $MVN_CMD -B -f $POM_FILE -Dbuildver=$UPSTREAM_BUILDVER package -pl dist -am -DskipTests -Dmaven.scaladoc.skip $MVN_OPT \ -Dincluded_buildvers=$UPSTREAM_BUILDVER,$BUILDVER fi # "Delete the unused object files to reduce the size of the Spark Rapids built tar." -rm -rf dist/target/jni-deps/ -find dist/target/parallel-world/ -mindepth 1 -maxdepth 1 ! -name META-INF -exec rm -rf {} + +# Determine the correct dist target directory based on which POM was used +if [[ "$SCALA_BINARY_VER" == "2.13" ]]; then + DIST_TARGET="scala2.13/dist/target" +else + DIST_TARGET="dist/target" +fi +rm -rf $DIST_TARGET/jni-deps/ +find $DIST_TARGET/parallel-world/ -mindepth 1 -maxdepth 1 ! -name META-INF -exec rm -rf {} + cd /home/ubuntu tar -zcf spark-rapids-built.tgz spark-rapids diff --git a/jenkins/databricks/common_vars.sh b/jenkins/databricks/common_vars.sh index 92ca7e94efe..8965d4a727c 100644 --- a/jenkins/databricks/common_vars.sh +++ b/jenkins/databricks/common_vars.sh @@ -67,7 +67,7 @@ EOF fi # TODO make this standard going forward -if [[ "$SPARK_VER" == '3.5.0' ]]; then +if [[ "$SPARK_VER" == '3.5.0' || "$SPARK_VER" == '4.0.0' ]]; then DB_VER_SUFFIX="${PYSP_TEST_spark_databricks_clusterUsageTags_sparkVersion//./}" else DB_VER_SUFFIX="" diff --git a/jenkins/databricks/deploy.sh b/jenkins/databricks/deploy.sh index 1079ee7dc6a..2544c773f36 100755 --- a/jenkins/databricks/deploy.sh +++ b/jenkins/databricks/deploy.sh @@ -1,6 +1,6 @@ #!/bin/bash # -# Copyright (c) 2020-2023, NVIDIA CORPORATION. All rights reserved. +# Copyright (c) 2020-2026, NVIDIA CORPORATION. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -24,14 +24,26 @@ cd spark-rapids echo "Maven mirror is $MVN_URM_MIRROR" SERVER_ID='snapshots' SERVER_URL="$URM_URL-local" -SCALA_VERSION=`mvn help:evaluate -q -pl dist -Dexpression=scala.binary.version -DforceStdout` +# Determine Scala version and POM file from Spark version +if [[ "$BASE_SPARK_VERSION" == 4.* ]]; then + SCALA_VERSION="2.13" + POM_FILE="scala2.13/pom.xml" +else + SCALA_VERSION="2.12" + POM_FILE="pom.xml" +fi + # remove the periods so change something like 3.2.1 to 321 VERSION_NUM=${BASE_SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS//.} SPARK_VERSION_STR=spark$VERSION_NUM -SPARK_PLUGIN_JAR_VERSION=`mvn help:evaluate -q -pl dist -Dexpression=project.version -DforceStdout` -# Append 143 into the db shim version because Databricks 14.3.x and 15.4.x are both based on spark version 3.5.0 +SPARK_PLUGIN_JAR_VERSION=`mvn help:evaluate -q -f $POM_FILE -pl dist -Dexpression=project.version -DforceStdout` +# Append 143 or 173 into the db shim version because Databricks 14.3.x and 15.4.x are both based on spark version 3.5.0 +# and Databricks 17.3 based on Spark 4.0.0 if [[ "$DB_RUNTIME" == "14.3"* ]]; then DB_SHIM_NAME="${SPARK_VERSION_STR}db143" +elif [[ "$DB_RUNTIME" == "17.3"* ]]; then + # Databricks 17.3 based on Spark 4.0.0 + DB_SHIM_NAME="${SPARK_VERSION_STR}db173" else DB_SHIM_NAME="${SPARK_VERSION_STR}db" fi diff --git a/jenkins/databricks/install_deps.py b/jenkins/databricks/install_deps.py index 57c3f98ee45..8185b33318d 100644 --- a/jenkins/databricks/install_deps.py +++ b/jenkins/databricks/install_deps.py @@ -1,4 +1,4 @@ -# Copyright (c) 2023-2024, NVIDIA CORPORATION. +# Copyright (c) 2023-2026, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -45,6 +45,9 @@ def define_deps(spark_version, scala_version): elif spark_version.startswith('3.5'): spark_prefix = '----ws_3_5' mvn_prefix = '--mvn' + elif spark_version.startswith('4.0'): + spark_prefix = '----ws_4_0' + mvn_prefix = '--mvn' else: raise Exception(f"Unsupported Databricks version {spark.version}") @@ -82,9 +85,17 @@ def define_deps(spark_version, scala_version): f'{spark_prefix}--sql--hive--hive-{spark_suffix}_*.jar'), Artifact('org.apache.hive', 'hive-exec', f'{spark_prefix}--patched-hive-with-glue--hive-exec*.jar'), - Artifact('org.apache.hive', 'hive-metastore-client-patched', - f'{spark_prefix}--patched-hive-with-glue--hive-*-patch-{spark_suffix}_deploy.jar'), - + ] + # hive-metastore-client-patched has different naming pattern in Spark 4.0 + if spark_version.startswith('4.0'): + # Spark 4.0: the patch jar uses scala 2.13 + deps += [Artifact('org.apache.hive', 'hive-metastore-client-patched', + f'{spark_prefix}--patched-hive-with-glue--hive-*-patch-hive-2.3__hadoop-3.2_2.*.jar')] + else: + # Spark 3.x versions + deps += [Artifact('org.apache.hive', 'hive-metastore-client-patched', + f'{spark_prefix}--patched-hive-with-glue--hive-*-patch-{spark_suffix}_deploy.jar')] + deps += [ # Hive Artifact('org.apache.hive', 'hive-serde', f'{prefix_ws_sp_mvn_hadoop}--org.apache.hive--hive-serde--org.apache.hive__hive-serde__*.jar'), @@ -129,8 +140,6 @@ def define_deps(spark_version, scala_version): f'{prefix_ws_sp_mvn_hadoop}--com.fasterxml.jackson.core--jackson-core--com.fasterxml.jackson.core__jackson-core__*.jar'), Artifact('com.fasterxml.jackson.core', 'jackson-annotations', f'{prefix_ws_sp_mvn_hadoop}--com.fasterxml.jackson.core--jackson-annotations--com.fasterxml.jackson.core__jackson-annotations__*.jar'), - Artifact('org.apache.spark', f'spark-avro_{scala_version}', - f'{prefix_ws_sp_mvn_hadoop}--org.apache.avro--avro--org.apache.avro*.jar' if spark_version.startswith('3.5') else f'{spark_prefix}--vendor--avro--avro-*.jar'), Artifact('org.apache.avro', 'avro-mapred', f'{prefix_ws_sp_mvn_hadoop}--org.apache.avro--avro-mapred--org.apache.avro__avro-mapred__*.jar'), Artifact('org.apache.avro', 'avro', @@ -147,8 +156,19 @@ def define_deps(spark_version, scala_version): f'{prefix_ws_sp_mvn_hadoop}--io.netty--netty-handler--io.netty__netty-handler__*.jar'), ] + # spark-avro has different locations depending on Spark version + if spark_version.startswith('4.0'): + deps += [Artifact('org.apache.spark', f'spark-avro_{scala_version}', + f'{spark_prefix}--connector--avro--avro-{spark_suffix}_*.jar')] + elif spark_version.startswith('3.5'): + deps += [Artifact('org.apache.spark', f'spark-avro_{scala_version}', + f'{prefix_ws_sp_mvn_hadoop}--org.apache.avro--avro--org.apache.avro*.jar')] + else: + deps += [Artifact('org.apache.spark', f'spark-avro_{scala_version}', + f'{spark_prefix}--vendor--avro--avro-*.jar')] + # Parquet - if spark_version.startswith('3.4') or spark_version.startswith('3.5'): + if spark_version.startswith('3.4') or spark_version.startswith('3.5') or spark_version.startswith('4.0'): deps += [ Artifact('org.apache.parquet', 'parquet-hadoop', f'{spark_prefix}--third_party--parquet-mr--parquet-hadoop--parquet-hadoop-shaded--*--libparquet-hadoop-internal.jar'), @@ -159,7 +179,7 @@ def define_deps(spark_version, scala_version): Artifact('org.apache.parquet', 'parquet-format', f'{spark_prefix}--third_party--parquet-mr--parquet-format-structures--parquet-format-structures-shaded--*--libparquet-format-structures-internal.jar'), Artifact('shaded.parquet.org.apache.thrift', f'shaded-parquet-thrift_{scala_version}', - f'{spark_prefix}--third_party--parquet-mr--parquet-format-structures--parquet-format-structures-shaded--*--org.apache.thrift__libthrift__0.16.0.jar'), + f'{spark_prefix}--third_party--parquet-mr--parquet-format-structures--parquet-format-structures-shaded--*--org.apache.thrift__libthrift__*.jar'), Artifact('org.apache.parquet', f'parquet-format-internal_{scala_version}', f'{spark_prefix}--third_party--parquet-mr--parquet-format-structures--parquet-format-structures-shaded--*--libparquet-thrift.jar') ] @@ -177,7 +197,7 @@ def define_deps(spark_version, scala_version): # log4j-core - if spark_version.startswith('3.3') or spark_version.startswith('3.4') or spark_version.startswith('3.5'): + if spark_version.startswith('3.3') or spark_version.startswith('3.4') or spark_version.startswith('3.5') or spark_version.startswith('4.0'): deps += Artifact('org.apache.logging.log4j', 'log4j-core', f'{prefix_ws_sp_mvn_hadoop}--org.apache.logging.log4j--log4j-core--org.apache.logging.log4j__log4j-core__*.jar'), @@ -187,19 +207,30 @@ def define_deps(spark_version, scala_version): f'{prefix_ws_sp_mvn_hadoop}--org.scala-lang.modules--scala-parser-combinators_{scala_version}-*.jar') ] - if spark_version.startswith('3.4') or spark_version.startswith('3.5'): + if spark_version.startswith('3.4') or spark_version.startswith('3.5') or spark_version.startswith('4.0'): deps += [ # Spark Internal Logging - Artifact('org.apache.spark', f'spark-common-utils_{scala_version}', f'{spark_prefix}--common--utils--common-utils-hive-2.3__hadoop-3.2_2.12_deploy.jar'), + Artifact('org.apache.spark', f'spark-common-utils_{scala_version}', f'{spark_prefix}--common--utils--common-utils-hive-2.3__hadoop-3.2_{scala_version}_deploy.jar'), # Spark SQL API - Artifact('org.apache.spark', f'spark-sql-api_{scala_version}', f'{spark_prefix}--sql--api--sql-api-hive-2.3__hadoop-3.2_2.12_deploy.jar') + Artifact('org.apache.spark', f'spark-sql-api_{scala_version}', f'{spark_prefix}--sql--api--sql-api-hive-2.3__hadoop-3.2_{scala_version}_deploy.jar') ] - if spark_version.startswith('3.5'): + if spark_version.startswith('3.5') or spark_version.startswith('4.0'): deps += [ Artifact('org.scala-lang.modules', f'scala-collection-compat_{scala_version}', f'{prefix_ws_sp_mvn_hadoop}--org.scala-lang.modules--scala-collection-compat_{scala_version}--org.scala-lang.modules__scala-collection-compat_{scala_version}__2.11.0.jar'), - Artifact('org.apache.avro', f'avro-connector', f'{spark_prefix}--connector--avro--avro-hive-2.3__hadoop-3.2_2.12_shaded---606136534--avro-unshaded-hive-2.3__hadoop-3.2_2.12_deploy.jar') + Artifact('org.apache.avro', f'avro-connector', f'{spark_prefix}--connector--avro--avro-hive-2.3__hadoop-3.2_{scala_version}_shaded--*--avro-unshaded-hive-2.3__hadoop-3.2_{scala_version}_deploy.jar') + ] + + # Databricks 17.3 (Spark 4.0) requires additional common-utils JARs + if spark_version.startswith('4.0'): + deps += [ + # Logging class is in common-utils-other + Artifact('org.apache.spark', f'spark-common-utils-other_{scala_version}', f'{spark_prefix}--common--utils--common-utils-other-hive-2.3__hadoop-3.2_{scala_version}_deploy.jar'), + # ConfigEntry and related config classes are in common-config + Artifact('org.apache.spark', f'spark-common-config_{scala_version}', f'{spark_prefix}--common--utils--common-config-hive-2.3__hadoop-3.2_{scala_version}_deploy.jar'), + Artifact('org.apache.orc', 'orc-format', + f'{prefix_ws_sp_mvn_hadoop}--org.apache.orc--orc-format--org.apache.orc__orc-format__*-shaded-protobuf.jar') ] return deps diff --git a/pom.xml b/pom.xml index d5244ff7c50..07824453457 100644 --- a/pom.xml +++ b/pom.xml @@ -707,6 +707,28 @@ delta-lake/delta-40x + + release400db173 + + + buildver + 400db173 + + + + spark400db173 + ${spark400db173.version} + ${spark400db173.version} + 3.3.1 + true + 1.13.1 + rapids-4-spark-delta-stub + + + shim-deps/databricks + delta-lake/delta-stub + + release401 @@ -960,6 +982,7 @@ 3.5.7 3.5.8 4.0.0 + 4.0.0-databricks-173 4.0.1 4.0.2 4.1.1 @@ -1670,9 +1693,9 @@ This will force full Scala code rebuild in downstream modules. [17,) - Unexpected buildver value ${buildver} for a Scala 2.13 build, only Apache Spark versions 3.3.0 (330) and higher are supported, no vendor builds such as 330db + Unexpected buildver value ${buildver} for a Scala 2.13 build, only Apache Spark versions 3.3.0 (330) and higher are supported buildver - (?:[3-9][3-9]|[4-9][0-9])[0-9] + (?:[3-9][3-9]|[4-9][0-9])[0-9](?:db\d*)? --> diff --git a/scala2.13/pom.xml b/scala2.13/pom.xml index 9603e63878c..97bcbd7d58e 100644 --- a/scala2.13/pom.xml +++ b/scala2.13/pom.xml @@ -707,6 +707,28 @@ delta-lake/delta-40x + + release400db173 + + + buildver + 400db173 + + + + spark400db173 + ${spark400db173.version} + ${spark400db173.version} + 3.3.1 + true + 1.13.1 + rapids-4-spark-delta-stub + + + shim-deps/databricks + delta-lake/delta-stub + + release401 @@ -960,6 +982,7 @@ 3.5.7 3.5.8 4.0.0 + 4.0.0-databricks-173 4.0.1 4.0.2 4.1.1 @@ -1670,9 +1693,9 @@ This will force full Scala code rebuild in downstream modules. [17,) - Unexpected buildver value ${buildver} for a Scala 2.13 build, only Apache Spark versions 3.3.0 (330) and higher are supported, no vendor builds such as 330db + Unexpected buildver value ${buildver} for a Scala 2.13 build, only Apache Spark versions 3.3.0 (330) and higher are supported buildver - (?:[3-9][3-9]|[4-9][0-9])[0-9] + (?:[3-9][3-9]|[4-9][0-9])[0-9](?:db\d*)? diff --git a/scala2.13/shim-deps/databricks/pom.xml b/scala2.13/shim-deps/databricks/pom.xml index e61e60bdfb6..5c4a69ede19 100644 --- a/scala2.13/shim-deps/databricks/pom.xml +++ b/scala2.13/shim-deps/databricks/pom.xml @@ -329,4 +329,26 @@ compile + + + + + release400db173 + + + buildver + 400db173 + + + + + org.apache.orc + orc-format + ${spark.version} + compile + + + + + diff --git a/scala2.13/shim-deps/pom.xml b/scala2.13/shim-deps/pom.xml index ca73b5fcb53..902b829b445 100644 --- a/scala2.13/shim-deps/pom.xml +++ b/scala2.13/shim-deps/pom.xml @@ -212,6 +212,73 @@ + + + release400db173 + + + buildver + 400db173 + + + + + org.apache.logging.log4j + log4j-core + ${spark.version} + provided + + + org.apache.parquet + parquet-format-internal_${scala.binary.version} + ${spark.version} + provided + + + org.apache.spark + spark-common-utils_${scala.binary.version} + ${spark.version} + provided + + + org.apache.spark + spark-sql-api_${scala.binary.version} + ${spark.version} + provided + + + org.apache.spark + spark-common-utils-other_${scala.binary.version} + ${spark.version} + provided + + + org.apache.spark + spark-common-config_${scala.binary.version} + ${spark.version} + provided + + + shaded.parquet.org.apache.thrift + shaded-parquet-thrift_${scala.binary.version} + ${spark.version} + provided + + + org.apache.avro + avro-connector + ${spark.version} + provided + + + org.scala-lang.modules + scala-collection-compat_${scala.binary.version} + ${spark.version} + provided + + + + dbdeps diff --git a/shim-deps/databricks/pom.xml b/shim-deps/databricks/pom.xml index bcd256cef92..9cfc290bae8 100644 --- a/shim-deps/databricks/pom.xml +++ b/shim-deps/databricks/pom.xml @@ -329,4 +329,26 @@ compile + + + + diff --git a/shim-deps/pom.xml b/shim-deps/pom.xml index 497cec0844b..9b506041dbf 100644 --- a/shim-deps/pom.xml +++ b/shim-deps/pom.xml @@ -212,6 +212,73 @@ + dbdeps diff --git a/sql-plugin-api/src/main/scala/org/apache/spark/sql/rapids/ProxyRapidsShuffleInternalManagerBase.scala b/sql-plugin-api/src/main/scala/org/apache/spark/sql/rapids/ProxyRapidsShuffleInternalManagerBase.scala index 805eab01443..bb3ec18201c 100644 --- a/sql-plugin-api/src/main/scala/org/apache/spark/sql/rapids/ProxyRapidsShuffleInternalManagerBase.scala +++ b/sql-plugin-api/src/main/scala/org/apache/spark/sql/rapids/ProxyRapidsShuffleInternalManagerBase.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023, NVIDIA CORPORATION. + * Copyright (c) 2023-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -31,11 +31,30 @@ trait RapidsShuffleManagerLike { def initialize: Unit } +/** + * Delegation trait for getReader. This trait has a version-independent + * signature that works across all Spark versions. + */ +trait ProxyShuffleReaderDelegate { + def getReaderImpl[K, C]( + handle: ShuffleHandle, + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int, + context: TaskContext, + metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] +} + /** * A simple proxy wrapper allowing to delay loading of the * real implementation to a later point when ShimLoader * has already updated Spark classloaders. * + * The bytecode will differ across Spark versions where ShuffleManager's + * abstract methods change (e.g., DB 17.3 adds prismMapStatusEnabled to getReader). + * The binary-dedupe check exempts this class. + * * @param conf * @param isDriver */ @@ -46,7 +65,7 @@ class ProxyRapidsShuffleInternalManagerBase( // touched in the plugin code after the shim initialization // is complete - private lazy val realImpl = ShimLoader.newInternalShuffleManager(conf, isDriver) + protected lazy val realImpl = ShimLoader.newInternalShuffleManager(conf, isDriver) .asInstanceOf[ShuffleManager] // This function touches the lazy val `self` so we actually instantiate @@ -69,6 +88,8 @@ class ProxyRapidsShuffleInternalManagerBase( realImpl.getWriter(handle, mapId, context, metrics) } + // 7-parameter getReader (Spark 3.x / 4.x OSS). + // Implements ShuffleManager.getReader for non-DB-17.3 versions. def getReader[K, C]( handle: ShuffleHandle, startMapIndex: Int, @@ -77,9 +98,25 @@ class ProxyRapidsShuffleInternalManagerBase( endPartition: Int, context: TaskContext, metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { - realImpl.getReader(handle, - startMapIndex, endMapIndex, startPartition, endPartition, - context, metrics) + realImpl.asInstanceOf[ProxyShuffleReaderDelegate] + .getReaderImpl(handle, startMapIndex, endMapIndex, + startPartition, endPartition, context, metrics) + } + + // 8-parameter getReader (Databricks 17.3 with prismMapStatusEnabled). + // Implements ShuffleManager.getReader for DB 17.3. + def getReader[K, C]( + handle: ShuffleHandle, + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int, + context: TaskContext, + metrics: ShuffleReadMetricsReporter, + prismMapStatusEnabled: Boolean): ShuffleReader[K, C] = { + realImpl.asInstanceOf[ProxyShuffleReaderDelegate] + .getReaderImpl(handle, startMapIndex, endMapIndex, + startPartition, endPartition, context, metrics) } def registerShuffle[K, V, C]( @@ -95,4 +132,3 @@ class ProxyRapidsShuffleInternalManagerBase( def stop(): Unit = realImpl.stop() } - diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/CostBasedOptimizer.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/CostBasedOptimizer.scala index 81e221befd2..cb79e329e7a 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/CostBasedOptimizer.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/CostBasedOptimizer.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2023, NVIDIA CORPORATION. + * Copyright (c) 2021-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,7 +18,7 @@ package com.nvidia.spark.rapids import scala.collection.mutable.ListBuffer -import com.nvidia.spark.rapids.shims.{GlobalLimitShims, SparkShimImpl} +import com.nvidia.spark.rapids.shims.{GlobalLimitShims, QueryStageRowCountShims, SparkShimImpl} import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Expression, GetStructField, WindowFrame, WindowSpecDefinition} @@ -438,7 +438,7 @@ object RowCountPlanVisitor { def visit(plan: SparkPlanMeta[_]): Option[BigInt] = plan.wrapped match { case p: QueryStageExec => - p.getRuntimeStatistics.rowCount + QueryStageRowCountShims.getRowCount(p) case _: GlobalLimitExec => GlobalLimitShims.visit(plan.asInstanceOf[SparkPlanMeta[GlobalLimitExec]]) case LocalLimitExec(limit, _) => diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala index f7b084b3770..c0e1c87c33b 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala @@ -33,6 +33,7 @@ import com.nvidia.spark.rapids.GpuMetric._ import com.nvidia.spark.rapids.RapidsPluginImplicits._ import com.nvidia.spark.rapids.RmmRapidsRetryIterator.withRetryNoSplit import com.nvidia.spark.rapids.io.async._ +import com.nvidia.spark.rapids.shims.SparkShimImpl import org.apache.commons.io.IOUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} @@ -341,7 +342,7 @@ abstract class MultiFilePartitionReaderFactoryBase( override def createColumnarReader(partition: InputPartition): PartitionReader[ColumnarBatch] = { assert(partition.isInstanceOf[FilePartition]) val filePartition = partition.asInstanceOf[FilePartition] - val files = filePartition.files + val files = SparkShimImpl.getPartitionFiles(filePartition).toArray val filePaths = files.map(_.filePath.toString()) val conf = broadcastedConf.value.value diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index fef7bea85dd..b38ce69ba5e 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -4989,7 +4989,8 @@ case class GpuOverrides() extends Rule[SparkPlan] with Logging { // example filename: "file:/tmp/delta-table/_delta_log/00000000000000000000.json" val found = StaticPartitionShims.getStaticPartitions(f.relation).map { parts => parts.exists { part => - part.files.exists(partFile => checkDeltaFunc(partFile.filePath.toString)) + SparkShimImpl.getPartitionFiles(part).exists(partFile => + checkDeltaFunc(partFile.filePath.toString)) } }.getOrElse { f.relation.location.rootPaths.exists { path => diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala index 11e0a0fe989..17df2c0bc37 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2025, NVIDIA CORPORATION. + * Copyright (c) 2020-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -82,6 +82,12 @@ trait SparkShims { metadataColumns: Seq[AttributeReference] = Seq.empty, fileFormat: Option[FileFormat] = None): RDD[InternalRow] + /** + * Get files from FilePartition. In Spark 4.0+, the files method was deprecated + * in favor of filesWithAbsolutePaths or innerFiles. + */ + def getPartitionFiles(partition: FilePartition): Seq[PartitionedFile] + def shouldFailDivOverflow: Boolean def reusedExchangeExecPfn: PartialFunction[SparkPlan, ReusedExchangeExec] diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala index 3bdc45c1957..8ca82ae436b 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2025, NVIDIA CORPORATION. + * Copyright (c) 2020-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -577,19 +577,19 @@ case class GpuFileSourceScanExec( val prunedPartitions = requiredPartitionSchema.map { partSchema => val idsAndTypes = partSchema.map(f => (relation.partitionSchema.indexOf(f), f.dataType)) partitions.map { p => - val partFiles = p.files.map { pf => + val partFiles = FilePartitionShims.getFiles(p).map { pf => val prunedPartValues = idsAndTypes.map { case (id, dType) => pf.partitionValues.get(id, dType) } pf.copy(partitionValues = InternalRow.fromSeq(prunedPartValues)) } - p.copy(files = partFiles) + FilePartitionShims.copyWithFiles(p, partFiles) } }.getOrElse(partitions) // Update the preferred locations based on the file cache locality val locatedPartitions = prunedPartitions.map { partition => - val newFiles = partition.files.map { partFile => + val newFiles = FilePartitionShims.getFiles(partition).map { partFile => val cacheLocations = FileCacheLocalityManager.get.getLocations(partFile.filePath.toString) if (cacheLocations.nonEmpty) { val newLocations = cacheLocations ++ partFile.locations @@ -598,7 +598,7 @@ case class GpuFileSourceScanExec( partFile } } - partition.copy(files = newFiles) + FilePartitionShims.copyWithFiles(partition, newFiles) } if (isPerFileReadEnabled) { @@ -607,7 +607,7 @@ case class GpuFileSourceScanExec( requiredSchema, fileFormat = Some(relation.fileFormat)) } else { logDebug(s"Using Datasource RDD, files are: " + - s"${prunedPartitions.flatMap(_.files).mkString(",")}") + s"${prunedPartitions.flatMap(FilePartitionShims.getFiles).mkString(",")}") // note we use the v2 DataSourceRDD instead of FileScanRDD so we don't have to copy more code GpuDataSourceRDD(relation.sparkSession.sparkContext, locatedPartitions, readerFactory) } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuScalarSubquery.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuScalarSubquery.scala index 6eb1f487972..42b269d0a7a 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuScalarSubquery.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuScalarSubquery.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2023, NVIDIA CORPORATION. + * Copyright (c) 2021-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,7 +17,7 @@ package org.apache.spark.sql.rapids import com.nvidia.spark.rapids.{GpuColumnVector, GpuExpression, GpuExpressionsUtils, GpuScalar} -import com.nvidia.spark.rapids.shims.ShimExpression +import com.nvidia.spark.rapids.shims.{GpuScalarSubqueryShims, ShimExpression} import org.apache.spark.sql.catalyst.expressions.{Expression, ExprId} import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression} @@ -33,7 +33,8 @@ import org.apache.spark.sql.vectorized.ColumnarBatch case class GpuScalarSubquery( plan: BaseSubqueryExec, exprId: ExprId) - extends ExecSubqueryExpression with GpuExpression with ShimExpression { + extends ExecSubqueryExpression with GpuExpression with ShimExpression + with GpuScalarSubqueryShims { override def dataType: DataType = plan.schema.fields.head.dataType override def children: Seq[Expression] = Seq.empty @@ -43,7 +44,7 @@ case class GpuScalarSubquery( // the first column in first row from `query`. @volatile private var result: Any = _ - @volatile private var updated: Boolean = false + @volatile protected var updated: Boolean = false override def updateResult(): Unit = { val rows = plan.executeCollect() diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManagerBase.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManagerBase.scala index e14685ba088..16013d41b0a 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManagerBase.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManagerBase.scala @@ -1712,7 +1712,8 @@ class RapidsCachingWriter[K, V]( * Apache Spark to use the RAPIDS shuffle manager, */ class RapidsShuffleInternalManagerBase(conf: SparkConf, val isDriver: Boolean) - extends ShuffleManager with RapidsShuffleHeartbeatHandler with Logging { + extends ShuffleManager with RapidsShuffleHeartbeatHandler with Logging + with RapidsShuffleReaderShim with ProxyShuffleReaderDelegate { def getServerId: BlockManagerId = server.fold(blockManager.blockManagerId)(_.getId) @@ -1981,14 +1982,14 @@ class RapidsShuffleInternalManagerBase(conf: SparkConf, val isDriver: Boolean) } } - override def getReader[K, C]( - handle: ShuffleHandle, - startMapIndex: Int, - endMapIndex: Int, - startPartition: Int, - endPartition: Int, - context: TaskContext, - metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { + def getReaderImpl[K, C]( + handle: ShuffleHandle, + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int, + context: TaskContext, + metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { handle match { case gpuHandle: GpuShuffleHandle[_, _] => logInfo(s"Asking map output tracker for dependency ${gpuHandle.dependency}, " + @@ -2056,13 +2057,13 @@ class RapidsShuffleInternalManagerBase(conf: SparkConf, val isDriver: Boolean) numReaderThreads = rapidsConf.shuffleMultiThreadedReaderThreads) case _ => val shuffleHandle = RapidsShuffleInternalManagerBase.unwrapHandle(other) - wrapped.getReader(shuffleHandle, startMapIndex, endMapIndex, startPartition, - endPartition, context, metrics) + ShuffleManagerShims.getReader(wrapped, shuffleHandle, startMapIndex, endMapIndex, + startPartition, endPartition, context, metrics) } case other => val shuffleHandle = RapidsShuffleInternalManagerBase.unwrapHandle(other) - wrapped.getReader(shuffleHandle, startMapIndex, endMapIndex, startPartition, - endPartition, context, metrics) + ShuffleManagerShims.getReader(wrapped, shuffleHandle, startMapIndex, endMapIndex, + startPartition, endPartition, context, metrics) } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/shims/GpuFileScanRDD.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/shims/GpuFileScanRDD.scala index ed8c2e7c228..5598ce0cf82 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/shims/GpuFileScanRDD.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/shims/GpuFileScanRDD.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2023, NVIDIA CORPORATION. + * Copyright (c) 2021-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,6 +18,7 @@ package org.apache.spark.sql.rapids.shims import java.io.{FileNotFoundException, IOException} import com.nvidia.spark.rapids.ScalableTaskCompletion.onTaskCompletion +import com.nvidia.spark.rapids.shims.SparkShimImpl import org.apache.parquet.io.ParquetDecodingException import org.apache.spark.{Partition => RDDPartition, SparkUpgradeException, TaskContext} @@ -62,7 +63,8 @@ class GpuFileScanRDD( inputMetrics.setBytesRead(existingBytesRead + getBytesReadCallback()) } - private[this] val files = split.asInstanceOf[FilePartition].files.toIterator + private[this] val files = SparkShimImpl.getPartitionFiles( + split.asInstanceOf[FilePartition]).toIterator private[this] var currentFile: PartitionedFile = null private[this] var currentIterator: Iterator[Object] = null diff --git a/sql-plugin/src/main/spark321/java/com/nvidia/spark/rapids/shims/ShimSupportsRuntimeFiltering.java b/sql-plugin/src/main/spark321/java/com/nvidia/spark/rapids/shims/ShimSupportsRuntimeFiltering.java index 7174b6b4d91..9b56d2042fc 100644 --- a/sql-plugin/src/main/spark321/java/com/nvidia/spark/rapids/shims/ShimSupportsRuntimeFiltering.java +++ b/sql-plugin/src/main/spark321/java/com/nvidia/spark/rapids/shims/ShimSupportsRuntimeFiltering.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2025, NVIDIA CORPORATION. + * Copyright (c) 2022-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -40,6 +40,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} spark-rapids-shim-json-lines ***/ diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/RapidsShuffleManager.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/RapidsShuffleManager.scala index 99aa4555916..c33c130bb34 100644 --- a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/RapidsShuffleManager.scala +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/RapidsShuffleManager.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2025, NVIDIA CORPORATION. + * Copyright (c) 2021-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRowBase.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRowBase.scala index ac6d378c306..e30069468ef 100644 --- a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRowBase.scala +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRowBase.scala @@ -39,6 +39,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/DynamicPruningShims.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/DynamicPruningShims.scala new file mode 100644 index 00000000000..01e52b5d38b --- /dev/null +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/DynamicPruningShims.scala @@ -0,0 +1,63 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "321"} +{"spark": "330"} +{"spark": "330db"} +{"spark": "331"} +{"spark": "332"} +{"spark": "332db"} +{"spark": "333"} +{"spark": "334"} +{"spark": "340"} +{"spark": "341"} +{"spark": "341db"} +{"spark": "342"} +{"spark": "343"} +{"spark": "344"} +{"spark": "350"} +{"spark": "350db143"} +{"spark": "351"} +{"spark": "352"} +{"spark": "353"} +{"spark": "354"} +{"spark": "355"} +{"spark": "356"} +{"spark": "357"} +{"spark": "358"} +{"spark": "400"} +{"spark": "401"} +{"spark": "402"} +{"spark": "411"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import org.apache.spark.sql.catalyst.expressions.{DynamicPruningExpression, Expression} + +/** + * Shim for DynamicPruningExpression (versions with single parameter). + */ +object DynamicPruningShims { + def unapply(expr: Expression): Option[Expression] = expr match { + case dpe: DynamicPruningExpression => Some(dpe.child) + case _ => None + } + + def apply(child: Expression): DynamicPruningExpression = { + DynamicPruningExpression(child) + } +} diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExecBase.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExecBase.scala index b9ce644191e..5c4f5d52f0e 100644 --- a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExecBase.scala +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExecBase.scala @@ -40,6 +40,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader320Plus.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader320Plus.scala index e1517218d51..fdf28d9c7f8 100644 --- a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader320Plus.scala +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader320Plus.scala @@ -39,6 +39,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReaderBase.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReaderBase.scala index 47c7ae47a92..90f7747f0cb 100644 --- a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReaderBase.scala +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReaderBase.scala @@ -39,6 +39,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/GpuParquetCrypto.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/GpuParquetCrypto.scala index e9630272d38..f24d73d86fe 100644 --- a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/GpuParquetCrypto.scala +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/GpuParquetCrypto.scala @@ -39,6 +39,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/GpuScalarSubqueryShims.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/GpuScalarSubqueryShims.scala new file mode 100644 index 00000000000..b395ebd7799 --- /dev/null +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/GpuScalarSubqueryShims.scala @@ -0,0 +1,49 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "321"} +{"spark": "330"} +{"spark": "330db"} +{"spark": "331"} +{"spark": "332"} +{"spark": "332db"} +{"spark": "333"} +{"spark": "334"} +{"spark": "340"} +{"spark": "341"} +{"spark": "341db"} +{"spark": "342"} +{"spark": "343"} +{"spark": "344"} +{"spark": "350"} +{"spark": "350db143"} +{"spark": "351"} +{"spark": "352"} +{"spark": "353"} +{"spark": "354"} +{"spark": "355"} +{"spark": "356"} +{"spark": "357"} +{"spark": "358"} +{"spark": "400"} +{"spark": "401"} +{"spark": "402"} +{"spark": "411"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +trait GpuScalarSubqueryShims diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/GpuUnionExecShim.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/GpuUnionExecShim.scala index f11332b55b4..af1df0dc36d 100644 --- a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/GpuUnionExecShim.scala +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/GpuUnionExecShim.scala @@ -40,6 +40,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} spark-rapids-shim-json-lines ***/ diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/HashUtils.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/HashUtils.scala index c4931762be1..e1e33a50c28 100644 --- a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/HashUtils.scala +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/HashUtils.scala @@ -39,6 +39,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/HybridFileSourceScanExecMeta.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/HybridFileSourceScanExecMeta.scala index 3ec4ebc9e2a..c5c3a368ffb 100644 --- a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/HybridFileSourceScanExecMeta.scala +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/HybridFileSourceScanExecMeta.scala @@ -40,6 +40,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} @@ -49,7 +50,6 @@ package com.nvidia.spark.rapids.shims import com.nvidia.spark.rapids._ import org.apache.spark.rapids.hybrid.HybridFileSourceScanExec -import org.apache.spark.sql.catalyst.expressions.DynamicPruningExpression import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat @@ -71,12 +71,12 @@ class HybridFileSourceScanExecMeta(plan: FileSourceScanExec, } wrapped.partitionFilters.map { filter => filter.transformDown { - case dpe@DynamicPruningExpression(inSub: InSubqueryExec) => + case dpe@DynamicPruningShims(inSub: InSubqueryExec) => inSub.plan match { case bc: SubqueryBroadcastExec => - dpe.copy(inSub.copy(plan = convertBroadcast(bc))) + DynamicPruningShims(inSub.copy(plan = convertBroadcast(bc))) case reuse@ReusedSubqueryExec(bc: SubqueryBroadcastExec) => - dpe.copy(inSub.copy(plan = reuse.copy(convertBroadcast(bc)))) + DynamicPruningShims(inSub.copy(plan = reuse.copy(convertBroadcast(bc)))) case _ => dpe } diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/InSubqueryShims.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/InSubqueryShims.scala index b257d725e0c..c6ba471d52b 100644 --- a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/InSubqueryShims.scala +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/InSubqueryShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -20,6 +20,7 @@ {"spark": "332db"} {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/OffsetWindowFunctionMeta.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/OffsetWindowFunctionMeta.scala index 30b66598a69..d06bacb6b1c 100644 --- a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/OffsetWindowFunctionMeta.scala +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/OffsetWindowFunctionMeta.scala @@ -40,6 +40,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala index 07bb1101e4f..8fe86fe0032 100644 --- a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala @@ -40,6 +40,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/OrcShims320untilAllBase.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/OrcShims320untilAllBase.scala index 5601e6c49cc..e163fbe9808 100644 --- a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/OrcShims320untilAllBase.scala +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/OrcShims320untilAllBase.scala @@ -40,6 +40,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/QueryStageRowCountShims.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/QueryStageRowCountShims.scala new file mode 100644 index 00000000000..07304f13ac4 --- /dev/null +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/QueryStageRowCountShims.scala @@ -0,0 +1,58 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "321"} +{"spark": "330"} +{"spark": "330db"} +{"spark": "331"} +{"spark": "332"} +{"spark": "332db"} +{"spark": "333"} +{"spark": "334"} +{"spark": "340"} +{"spark": "341"} +{"spark": "341db"} +{"spark": "342"} +{"spark": "343"} +{"spark": "344"} +{"spark": "350"} +{"spark": "350db143"} +{"spark": "351"} +{"spark": "352"} +{"spark": "353"} +{"spark": "354"} +{"spark": "355"} +{"spark": "356"} +{"spark": "357"} +{"spark": "358"} +{"spark": "400"} +{"spark": "401"} +{"spark": "402"} +{"spark": "411"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import org.apache.spark.sql.execution.adaptive.QueryStageExec + +/** + * Helper to get row count from QueryStageExec. + */ +object QueryStageRowCountShims { + def getRowCount(qse: QueryStageExec): Option[BigInt] = { + qse.getRuntimeStatistics.rowCount + } +} diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/RapidsCsvScanMeta.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/RapidsCsvScanMeta.scala index 7a6d3071e99..f80494a677b 100644 --- a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/RapidsCsvScanMeta.scala +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/RapidsCsvScanMeta.scala @@ -40,6 +40,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/RebaseShims.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/RebaseShims.scala index 6dfea3a6f8e..d1ca0f9e7c0 100644 --- a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/RebaseShims.scala +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/RebaseShims.scala @@ -39,6 +39,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/ShimAQEShuffleReadExec.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/ShimAQEShuffleReadExec.scala index dcdfa0b8961..9f885a9b958 100644 --- a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/ShimAQEShuffleReadExec.scala +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/ShimAQEShuffleReadExec.scala @@ -40,6 +40,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/ShimBaseSubqueryExec.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/ShimBaseSubqueryExec.scala index 0638e17a844..b509bdead95 100644 --- a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/ShimBaseSubqueryExec.scala +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/ShimBaseSubqueryExec.scala @@ -40,6 +40,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/ShimPredicateHelper.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/ShimPredicateHelper.scala index ad5eda54a6e..8e882302234 100644 --- a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/ShimPredicateHelper.scala +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/ShimPredicateHelper.scala @@ -40,6 +40,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala index 50b89cbcd1d..197fee1d38e 100644 --- a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala @@ -40,12 +40,15 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims +import scala.annotation.nowarn + import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.GpuOverrides.exec @@ -60,6 +63,7 @@ import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive._ import org.apache.spark.sql.execution.command._ +import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile} import org.apache.spark.sql.execution.datasources.v2.{AppendDataExecV1, AtomicCreateTableAsSelectExec, AtomicReplaceTableAsSelectExec, OverwriteByExpressionExecV1} import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan @@ -77,8 +81,9 @@ import org.apache.spark.sql.rapids.shims.TrampolineConnectShims.SparkSession /** * Shim base class that can be compiled with every supported 3.2.0+ */ -trait Spark320PlusShims extends SparkShims with RebaseShims with WindowInPandasShims - with Logging { +trait Spark320PlusShims extends SparkShims with RebaseShims + with WindowInPandasShims with Logging { + override final def aqeShuffleReaderExec: ExecRule[_ <: SparkPlan] = exec[AQEShuffleReadExec]( "A wrapper of shuffle query stage", @@ -135,15 +140,19 @@ trait Spark320PlusShims extends SparkShims with RebaseShims with WindowInPandasS override def shouldFailDivOverflow: Boolean = SQLConf.get.ansiEnabled + override def getPartitionFiles(partition: FilePartition): Seq[PartitionedFile] = { + (partition.files: @nowarn( + "msg=files in trait FilePartitionBase is deprecated")) + } + def leafNodeDefaultParallelism(ss: SparkSession): Int = { SparkSessionUtils.leafNodeDefaultParallelism(ss) } override def isWindowFunctionExec(plan: SparkPlan): Boolean = plan.isInstanceOf[WindowExecBase] - override def getExprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = { - val baseExprs: Seq[ExprRule[_ <: Expression]] = Seq( - GpuOverrides.expr[Cast]( + override def getExprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = Seq( + GpuOverrides.expr[Cast]( "Convert a column of one type of data into another type", new CastChecks(), (cast, conf, p, r) => { @@ -194,7 +203,11 @@ trait Spark320PlusShims extends SparkShims with RebaseShims with WindowInPandasS // ANSI support for ABS was added in 3.2.0 SPARK-33275 override def convertToGpu(child: Expression): GpuExpression = GpuAbs(child, ansiEnabled) - }), + }) + // TimeAdd moved to TimeAddShims to handle version differences + ).map(r => (r.getClassFor.asSubclass(classOf[Expression]), r)) + .toMap[Class[_ <: Expression], ExprRule[_ <: Expression]] ++ + TimeAddShims.exprs ++ Seq( GpuOverrides.expr[SpecifiedWindowFrame]( "Specification of the width of the group (or \"frame\") of input rows " + "around which a window function is evaluated", @@ -222,12 +235,8 @@ trait Spark320PlusShims extends SparkShims with RebaseShims with WindowInPandasS TypeSig.CALENDAR + TypeSig.NULL + TypeSig.integral + TypeSig.DECIMAL_64 + TypeSig.DAYTIME, TypeSig.numericAndInterval))), (windowExpression, conf, p, r) => new GpuWindowExpressionMeta(windowExpression, conf, p, r)) - ) - val result = new scala.collection.mutable.HashMap[ - Class[_ <: Expression], ExprRule[_ <: Expression]]() - baseExprs.foreach(r => result(r.getClassFor.asSubclass(classOf[Expression])) = r) - result.toMap ++ TimeAddShims.exprs - } + ).map(r => (r.getClassFor.asSubclass(classOf[Expression]), r)) + .toMap[Class[_ <: Expression], ExprRule[_ <: Expression]] override def getExecs: Map[Class[_ <: SparkPlan], ExecRule[_ <: SparkPlan]] = { val maps: Map[Class[_ <: SparkPlan], ExecRule[_ <: SparkPlan]] = Seq( diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/Spark321PlusShims.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/Spark321PlusShims.scala index ef885f4eea2..68e108f9fd4 100644 --- a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/Spark321PlusShims.scala +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/Spark321PlusShims.scala @@ -40,6 +40,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/TreeNode.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/TreeNode.scala index c368946d7e5..6f1a42234b5 100644 --- a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/TreeNode.scala +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/TreeNode.scala @@ -40,6 +40,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/XxHash64Shims.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/XxHash64Shims.scala index 6052f2dc391..1725b1b5e60 100644 --- a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/XxHash64Shims.scala +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/XxHash64Shims.scala @@ -40,6 +40,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/YearParseUtil.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/YearParseUtil.scala index 2e90a8c4743..29e484e45ad 100644 --- a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/YearParseUtil.scala +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/YearParseUtil.scala @@ -39,6 +39,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/gpuWindows.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/gpuWindows.scala index ebdb136cbee..ae2d5524136 100644 --- a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/gpuWindows.scala +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/gpuWindows.scala @@ -40,6 +40,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/v1FallbackWriters.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/v1FallbackWriters.scala index a375263e9d2..aecf63b5c65 100644 --- a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/v1FallbackWriters.scala +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/v1FallbackWriters.scala @@ -40,6 +40,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark321/scala/org/apache/spark/rapids/shims/GpuShuffleBlockResolver.scala b/sql-plugin/src/main/spark321/scala/org/apache/spark/rapids/shims/GpuShuffleBlockResolver.scala index cc2ded370e7..8db03792fe1 100644 --- a/sql-plugin/src/main/spark321/scala/org/apache/spark/rapids/shims/GpuShuffleBlockResolver.scala +++ b/sql-plugin/src/main/spark321/scala/org/apache/spark/rapids/shims/GpuShuffleBlockResolver.scala @@ -40,6 +40,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark321/scala/org/apache/spark/rapids/shims/ShuffledBatchRDDUtil.scala b/sql-plugin/src/main/spark321/scala/org/apache/spark/rapids/shims/ShuffledBatchRDDUtil.scala index 4f05c85016b..a694c9ff1ea 100644 --- a/sql-plugin/src/main/spark321/scala/org/apache/spark/rapids/shims/ShuffledBatchRDDUtil.scala +++ b/sql-plugin/src/main/spark321/scala/org/apache/spark/rapids/shims/ShuffledBatchRDDUtil.scala @@ -40,6 +40,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} @@ -52,6 +53,7 @@ import org.apache.spark.{MapOutputTrackerMaster, Partition, ShuffleDependency, S import org.apache.spark.shuffle.ShuffleReader import org.apache.spark.sql.execution.{CoalescedMapperPartitionSpec, CoalescedPartitionSpec, PartialMapperPartitionSpec, PartialReducerPartitionSpec} import org.apache.spark.sql.execution.metric.SQLShuffleReadMetricsReporter +import org.apache.spark.sql.rapids.ShuffleManagerShims import org.apache.spark.sql.rapids.execution.ShuffledBatchRDDPartition import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.storage.{BlockId, BlockManagerId} @@ -114,7 +116,8 @@ object ShuffledBatchRDDUtil { endReducerIndex) (reader, getPartitionSize(blocksByAddress)) case PartialReducerPartitionSpec(reducerIndex, startMapIndex, endMapIndex, _) => - val reader = SparkEnv.get.shuffleManager.getReader( + val reader = ShuffleManagerShims.getReader( + SparkEnv.get.shuffleManager, dependency.shuffleHandle, startMapIndex, endMapIndex, @@ -130,7 +133,8 @@ object ShuffledBatchRDDUtil { reducerIndex + 1) (reader, getPartitionSize(blocksByAddress)) case PartialMapperPartitionSpec(mapIndex, startReducerIndex, endReducerIndex) => - val reader = SparkEnv.get.shuffleManager.getReader( + val reader = ShuffleManagerShims.getReader( + SparkEnv.get.shuffleManager, dependency.shuffleHandle, mapIndex, mapIndex + 1, @@ -146,7 +150,8 @@ object ShuffledBatchRDDUtil { endReducerIndex) (reader, getPartitionSize(blocksByAddress)) case CoalescedMapperPartitionSpec(startMapIndex, endMapIndex, numReducers) => - val reader = SparkEnv.get.shuffleManager.getReader( + val reader = ShuffleManagerShims.getReader( + SparkEnv.get.shuffleManager, dependency.shuffleHandle, startMapIndex, endMapIndex, diff --git a/sql-plugin/src/main/spark321/scala/org/apache/spark/rapids/shims/storage/ShimDiskBlockManager.scala b/sql-plugin/src/main/spark321/scala/org/apache/spark/rapids/shims/storage/ShimDiskBlockManager.scala index b66f221f607..e32d1cfafb6 100644 --- a/sql-plugin/src/main/spark321/scala/org/apache/spark/rapids/shims/storage/ShimDiskBlockManager.scala +++ b/sql-plugin/src/main/spark321/scala/org/apache/spark/rapids/shims/storage/ShimDiskBlockManager.scala @@ -40,6 +40,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala b/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala index badb1c6bf99..85a97e47def 100644 --- a/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala +++ b/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -50,6 +50,12 @@ object FilePartitionShims { } } + def getFiles(p: FilePartition): Array[PartitionedFile] = p.files + + def copyWithFiles(p: FilePartition, newFiles: Array[PartitionedFile]): FilePartition = { + p.copy(files = newFiles) + } + def splitFiles(sparkSession: SparkSession, hadoopConf: Configuration, selectedPartitions: Array[PartitionDirectory], diff --git a/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/RapidsShuffleReaderShim.scala b/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/RapidsShuffleReaderShim.scala new file mode 100644 index 00000000000..4cd037598f5 --- /dev/null +++ b/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/RapidsShuffleReaderShim.scala @@ -0,0 +1,71 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "321"} +{"spark": "330"} +{"spark": "330db"} +{"spark": "331"} +{"spark": "332"} +{"spark": "332db"} +{"spark": "333"} +{"spark": "334"} +{"spark": "340"} +{"spark": "341"} +{"spark": "341db"} +{"spark": "342"} +{"spark": "343"} +{"spark": "344"} +{"spark": "350"} +{"spark": "350db143"} +{"spark": "351"} +{"spark": "352"} +{"spark": "353"} +{"spark": "354"} +{"spark": "355"} +{"spark": "356"} +{"spark": "357"} +{"spark": "358"} +{"spark": "400"} +{"spark": "401"} +{"spark": "402"} +{"spark": "411"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.rapids + +import org.apache.spark.TaskContext +import org.apache.spark.shuffle.{ShuffleHandle, ShuffleReader, ShuffleReadMetricsReporter} + +/** + * Shim trait providing version-specific getReader signature for ShuffleManager. + */ +trait RapidsShuffleReaderShim { + self: RapidsShuffleInternalManagerBase => + + // 7-parameter version for earlier versions + override def getReader[K, C]( + handle: ShuffleHandle, + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int, + context: TaskContext, + metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { + getReaderImpl(handle, startMapIndex, endMapIndex, + startPartition, endPartition, context, metrics) + } +} + diff --git a/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/ShuffleManagerShims.scala b/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/ShuffleManagerShims.scala new file mode 100644 index 00000000000..5370210874d --- /dev/null +++ b/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/ShuffleManagerShims.scala @@ -0,0 +1,72 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "321"} +{"spark": "330"} +{"spark": "330db"} +{"spark": "331"} +{"spark": "332"} +{"spark": "332db"} +{"spark": "333"} +{"spark": "334"} +{"spark": "340"} +{"spark": "341"} +{"spark": "341db"} +{"spark": "342"} +{"spark": "343"} +{"spark": "344"} +{"spark": "350"} +{"spark": "350db143"} +{"spark": "351"} +{"spark": "352"} +{"spark": "353"} +{"spark": "354"} +{"spark": "355"} +{"spark": "356"} +{"spark": "357"} +{"spark": "358"} +{"spark": "400"} +{"spark": "401"} +{"spark": "402"} +{"spark": "411"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.rapids + +import org.apache.spark.TaskContext +import org.apache.spark.shuffle.{ShuffleHandle, ShuffleManager, ShuffleReader, ShuffleReadMetricsReporter} + +/** + * Shim object to handle version-specific differences in ShuffleManager APIs. + */ +object ShuffleManagerShims { + /** + * Call ShuffleManager.getReader with the appropriate signature for this Spark version. + * This method is overridden in version-specific shims. + */ + def getReader[K, C]( + manager: ShuffleManager, + handle: ShuffleHandle, + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int, + context: TaskContext, + metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { + manager.getReader(handle, startMapIndex, endMapIndex, startPartition, + endPartition, context, metrics) + } +} diff --git a/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuWindowPythonRunnerFactory.scala b/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuWindowPythonRunnerFactory.scala index f969aeea3b4..95df8052f82 100644 --- a/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuWindowPythonRunnerFactory.scala +++ b/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuWindowPythonRunnerFactory.scala @@ -40,6 +40,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} spark-rapids-shim-json-lines ***/ diff --git a/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/execution/python/shims/WindowBoundTypeConfShims.scala b/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/execution/python/shims/WindowBoundTypeConfShims.scala index cdd95af0c8a..086412acfbb 100644 --- a/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/execution/python/shims/WindowBoundTypeConfShims.scala +++ b/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/execution/python/shims/WindowBoundTypeConfShims.scala @@ -40,6 +40,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} spark-rapids-shim-json-lines ***/ diff --git a/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/shims/AvroUtils.scala b/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/shims/AvroUtils.scala index 430cf2834c7..9c1736e0785 100644 --- a/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/shims/AvroUtils.scala +++ b/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/shims/AvroUtils.scala @@ -40,6 +40,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/shims/GpuMapInPandasExecMeta.scala b/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/shims/GpuMapInPandasExecMeta.scala index 97063f2cd26..e35a569eea2 100644 --- a/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/shims/GpuMapInPandasExecMeta.scala +++ b/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/shims/GpuMapInPandasExecMeta.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -30,6 +30,7 @@ {"spark": "343"} {"spark": "344"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala b/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala index aa701af32d7..006a3c083a6 100644 --- a/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala +++ b/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala @@ -40,6 +40,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedWriter.scala b/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedWriter.scala index 571d623b5c8..20b63da74fb 100644 --- a/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedWriter.scala +++ b/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedWriter.scala @@ -40,6 +40,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/shims/Spark32XShimsUtils.scala b/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/shims/Spark32XShimsUtils.scala index 186634f0a38..b6562eb009b 100644 --- a/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/shims/Spark32XShimsUtils.scala +++ b/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/shims/Spark32XShimsUtils.scala @@ -40,6 +40,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/shims/SparkStringUtilsShims.scala b/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/shims/SparkStringUtilsShims.scala index 9116edab692..baa76bc4f83 100644 --- a/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/shims/SparkStringUtilsShims.scala +++ b/sql-plugin/src/main/spark321/scala/org/apache/spark/sql/rapids/shims/SparkStringUtilsShims.scala @@ -40,6 +40,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark321/scala/org/apache/spark/storage/RapidsPushBasedFetchHelper.scala b/sql-plugin/src/main/spark321/scala/org/apache/spark/storage/RapidsPushBasedFetchHelper.scala index c9860d59655..2fcf48adabb 100644 --- a/sql-plugin/src/main/spark321/scala/org/apache/spark/storage/RapidsPushBasedFetchHelper.scala +++ b/sql-plugin/src/main/spark321/scala/org/apache/spark/storage/RapidsPushBasedFetchHelper.scala @@ -40,6 +40,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark321/scala/org/apache/spark/storage/RapidsShuffleBlockFetcherIterator.scala b/sql-plugin/src/main/spark321/scala/org/apache/spark/storage/RapidsShuffleBlockFetcherIterator.scala index 191f626b878..6960a56fb7d 100644 --- a/sql-plugin/src/main/spark321/scala/org/apache/spark/storage/RapidsShuffleBlockFetcherIterator.scala +++ b/sql-plugin/src/main/spark321/scala/org/apache/spark/storage/RapidsShuffleBlockFetcherIterator.scala @@ -40,6 +40,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} @@ -1088,9 +1089,9 @@ final class RapidsShuffleBlockFetcherIterator( try { while (checkedIn.read(buffer) != -1) {} val checksum = checkedIn.getChecksum.getValue - cause = shuffleClient.diagnoseCorruption(address.host, address.port, address.executorId, - shuffleBlock.shuffleId, shuffleBlock.mapId, shuffleBlock.reduceId, checksum, - checksumAlgorithm) + cause = ShuffleClientShims.diagnoseCorruption(shuffleClient, + address.host, address.port, address.executorId, + shuffleBlock, checksum, checksumAlgorithm) } catch { case e: Exception => logWarning("Unable to diagnose the corruption cause of the corrupted block", e) diff --git a/sql-plugin/src/main/spark321/scala/org/apache/spark/storage/ShuffleClientShims.scala b/sql-plugin/src/main/spark321/scala/org/apache/spark/storage/ShuffleClientShims.scala new file mode 100644 index 00000000000..5ce8fb57078 --- /dev/null +++ b/sql-plugin/src/main/spark321/scala/org/apache/spark/storage/ShuffleClientShims.scala @@ -0,0 +1,70 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "321"} +{"spark": "330"} +{"spark": "330db"} +{"spark": "331"} +{"spark": "332"} +{"spark": "332db"} +{"spark": "333"} +{"spark": "334"} +{"spark": "340"} +{"spark": "341"} +{"spark": "341db"} +{"spark": "342"} +{"spark": "343"} +{"spark": "344"} +{"spark": "350"} +{"spark": "350db143"} +{"spark": "351"} +{"spark": "352"} +{"spark": "353"} +{"spark": "354"} +{"spark": "355"} +{"spark": "356"} +{"spark": "357"} +{"spark": "358"} +{"spark": "400"} +{"spark": "401"} +{"spark": "402"} +{"spark": "411"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.storage + +import org.apache.spark.network.shuffle.BlockStoreClient +import org.apache.spark.network.shuffle.checksum.Cause + +object ShuffleClientShims { + def diagnoseCorruption( + client: BlockStoreClient, + host: String, + port: Int, + execId: String, + blockId: BlockId, + checksum: Long, + algorithm: String): Cause = { + blockId match { + case shuffleBlock: ShuffleBlockId => + client.diagnoseCorruption(host, port, execId, + shuffleBlock.shuffleId, shuffleBlock.mapId, shuffleBlock.reduceId, + checksum, algorithm) + case _ => + throw new IllegalArgumentException(s"Unexpected block type: ${blockId.getClass}") + } + } +} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/GpuBloomFilter.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/GpuBloomFilter.scala index 74a37d875be..67779e34b83 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/GpuBloomFilter.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/GpuBloomFilter.scala @@ -39,6 +39,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/GpuBloomFilterMightContain.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/GpuBloomFilterMightContain.scala index f18491fce3c..de345acc28d 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/GpuBloomFilterMightContain.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/GpuBloomFilterMightContain.scala @@ -39,6 +39,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/AnsiUtil.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/AnsiUtil.scala index d36542d86c7..17b7358c4b8 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/AnsiUtil.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/AnsiUtil.scala @@ -39,6 +39,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/BloomFilterShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/BloomFilterShims.scala index 7590a075e89..7a2018d0637 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/BloomFilterShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/BloomFilterShims.scala @@ -39,6 +39,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/BucketingUtilsShim.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/BucketingUtilsShim.scala index 09f6f09f530..40098862798 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/BucketingUtilsShim.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/BucketingUtilsShim.scala @@ -39,6 +39,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/CharVarcharUtilsShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/CharVarcharUtilsShims.scala index e9e8ff3767c..7827063ad09 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/CharVarcharUtilsShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/CharVarcharUtilsShims.scala @@ -39,6 +39,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/DayTimeIntervalShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/DayTimeIntervalShims.scala index b2646226ec0..94e0b85ab64 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/DayTimeIntervalShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/DayTimeIntervalShims.scala @@ -52,7 +52,9 @@ import org.apache.spark.sql.rapids._ import org.apache.spark.sql.rapids.shims.{GpuDivideDTInterval, GpuMultiplyDTInterval} object DayTimeIntervalShims { - def exprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = Seq( + def exprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = + // TimeAdd moved to TimeAddShims to handle version differences + TimeAddShims.exprs ++ Seq( GpuOverrides.expr[Abs]( "Absolute value", ExprChecks.unaryProjectAndAstInputMatchesOutput( @@ -93,5 +95,6 @@ object DayTimeIntervalShims { override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuDivideDTInterval(lhs, rhs) }) - ).map(r => (r.getClassFor.asSubclass(classOf[Expression]), r)).toMap + ).map(r => (r.getClassFor.asSubclass(classOf[Expression]), r)) + .toMap[Class[_ <: Expression], ExprRule[_ <: Expression]] } diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/FilteredPartitions.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/FilteredPartitions.scala index b2e41029c74..02e3f7dfeec 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/FilteredPartitions.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/FilteredPartitions.scala @@ -39,6 +39,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuDataSourceRDD.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuDataSourceRDD.scala index 6b71279abbe..d95f3296200 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuDataSourceRDD.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuDataSourceRDD.scala @@ -39,6 +39,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuHashPartitioning.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuHashPartitioning.scala index 990039fe325..050f9169873 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuHashPartitioning.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuHashPartitioning.scala @@ -39,6 +39,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuIntervalUtilsBase.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuIntervalUtilsBase.scala index aaf7632f1bf..ce6e8bfbc1e 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuIntervalUtilsBase.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuIntervalUtilsBase.scala @@ -38,6 +38,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuRangePartitioning.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuRangePartitioning.scala index 16b69d7ca30..49bb855a682 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuRangePartitioning.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuRangePartitioning.scala @@ -39,6 +39,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuTypeShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuTypeShims.scala index 18e49db71de..5377d7362a1 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuTypeShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuTypeShims.scala @@ -38,6 +38,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/OrcReadingShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/OrcReadingShims.scala index 942aa2aa3cc..fe4203e778c 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/OrcReadingShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/OrcReadingShims.scala @@ -39,6 +39,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/OrcShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/OrcShims.scala index 48ef8d0c1ab..fab63d2953a 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/OrcShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/OrcShims.scala @@ -38,6 +38,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsFileSourceMetaUtils.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsFileSourceMetaUtils.scala index b13d2793951..b7fbc52b788 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsFileSourceMetaUtils.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsFileSourceMetaUtils.scala @@ -39,6 +39,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala index cff5cc18c30..66afb19eb3a 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala @@ -39,6 +39,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala index 8e5c99d5c4b..98c8516acfc 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala @@ -39,6 +39,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RoundingShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RoundingShims.scala index 4970fd9a25f..4cd83dd4562 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RoundingShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RoundingShims.scala @@ -39,6 +39,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ScanExecShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ScanExecShims.scala index bfa92abbaeb..11b7d4f6706 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ScanExecShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ScanExecShims.scala @@ -39,6 +39,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/parquet/ParquetFieldIdShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/parquet/ParquetFieldIdShims.scala index 8008b1f32c9..3fa1afaad64 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/parquet/ParquetFieldIdShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/parquet/ParquetFieldIdShims.scala @@ -38,6 +38,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/parquet/ParquetSchemaClipShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/parquet/ParquetSchemaClipShims.scala index bd21ee13ee4..ec34598809a 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/parquet/ParquetSchemaClipShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/parquet/ParquetSchemaClipShims.scala @@ -39,6 +39,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/execution/datasources/parquet/ShimCurrentBatchIterator.scala b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/execution/datasources/parquet/ShimCurrentBatchIterator.scala index df5da764462..072fdede3dc 100644 --- a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/execution/datasources/parquet/ShimCurrentBatchIterator.scala +++ b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/execution/datasources/parquet/ShimCurrentBatchIterator.scala @@ -39,6 +39,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/aggregate/GpuBloomFilterAggregate.scala b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/aggregate/GpuBloomFilterAggregate.scala index 2e0cab83747..8e31a648dc6 100644 --- a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/aggregate/GpuBloomFilterAggregate.scala +++ b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/aggregate/GpuBloomFilterAggregate.scala @@ -39,6 +39,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtilsFor330plus.scala b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtilsFor330plus.scala index 2cd58c89bc6..18dd5de5990 100644 --- a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtilsFor330plus.scala +++ b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtilsFor330plus.scala @@ -39,6 +39,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedReader.scala b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedReader.scala index 1e355bcdfbc..15119ee24a9 100644 --- a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedReader.scala +++ b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedReader.scala @@ -39,6 +39,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/intervalExpressions.scala b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/intervalExpressions.scala index 8d6e09e1d9b..31aed442bc4 100644 --- a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/intervalExpressions.scala +++ b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/intervalExpressions.scala @@ -39,6 +39,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/DatabricksShimServiceProvider.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/DatabricksShimServiceProvider.scala index 7fb28454c3a..2aae0326162 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/DatabricksShimServiceProvider.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/DatabricksShimServiceProvider.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024-2025, NVIDIA CORPORATION. + * Copyright (c) 2024-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "332db"} {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala index f3a11a9f89c..4e676636f4a 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "332db"} {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims @@ -32,7 +33,8 @@ object AQEUtils { /** Return a new QueryStageExec reuse instance with updated output attributes */ def newReuseInstance(sqse: ShuffleQueryStageExec, newOutput: Seq[Attribute]): QueryStageExec = { val reusedExchange = ReusedExchangeExec(newOutput, sqse.shuffle) - ShuffleQueryStageExec(sqse.id, reusedExchange, sqse.originalPlan, sqse.isSparkExchange) + QueryStageExecShims.createShuffleQueryStageExec( + sqse, sqse.id, reusedExchange, sqse.originalPlan, sqse.isSparkExchange) } // Databricks 10.4 has an issue where if you turn off AQE it can still use it for diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AggregationTagging.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AggregationTagging.scala index efe825394f4..b7191464bae 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AggregationTagging.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AggregationTagging.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "332db"} {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AnsiCastShim.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AnsiCastShim.scala index 7c5eca89dc4..3a585ed1021 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AnsiCastShim.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AnsiCastShim.scala @@ -33,6 +33,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/CastingConfigShim.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/CastingConfigShim.scala index a7d9324f0d4..f1e3d1f7db1 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/CastingConfigShim.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/CastingConfigShim.scala @@ -34,6 +34,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ColumnDefaultValuesShims.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ColumnDefaultValuesShims.scala index ea1025568c7..ceaa89927b0 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ColumnDefaultValuesShims.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ColumnDefaultValuesShims.scala @@ -33,6 +33,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/DecimalArithmeticOverrides.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/DecimalArithmeticOverrides.scala index a16af3014fd..168f362c76e 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/DecimalArithmeticOverrides.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/DecimalArithmeticOverrides.scala @@ -34,6 +34,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/DistributionUtil.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/DistributionUtil.scala index 5634c1910f3..84b23c499ee 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/DistributionUtil.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/DistributionUtil.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "332db"} {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/FileSourceScanExecMeta.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/FileSourceScanExecMeta.scala index 73cd3ded0b8..a2790c22351 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/FileSourceScanExecMeta.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/FileSourceScanExecMeta.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2025, NVIDIA CORPORATION. + * Copyright (c) 2022-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,13 +19,14 @@ {"spark": "332db"} {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims import com.nvidia.spark.rapids._ import org.apache.spark.internal.Logging -import org.apache.spark.sql.catalyst.expressions.{DynamicPruningExpression, Expression} +import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.datasources.HadoopFsRelation import org.apache.spark.sql.execution.datasources.json.JsonFileFormat @@ -90,12 +91,12 @@ class FileSourceScanExecMeta(plan: FileSourceScanExec, private def convertDynamicPruningFilters(filters: Seq[Expression]): Seq[Expression] = { filters.map { filter => filter.transformDown { - case dpe @ DynamicPruningExpression(inSub: InSubqueryExec) => + case dpe @ DynamicPruningShims(inSub: InSubqueryExec) => inSub.plan match { case bc: SubqueryBroadcastExec => - dpe.copy(inSub.copy(plan = convertBroadcast(bc))) + DynamicPruningShims(inSub.copy(plan = convertBroadcast(bc))) case reuse @ ReusedSubqueryExec(bc: SubqueryBroadcastExec) => - dpe.copy(inSub.copy(plan = reuse.copy(convertBroadcast(bc)))) + DynamicPruningShims(inSub.copy(plan = reuse.copy(convertBroadcast(bc)))) case _ => dpe } diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GetMapValueMeta.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GetMapValueMeta.scala index 773d909e390..9ccb12d74e8 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GetMapValueMeta.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GetMapValueMeta.scala @@ -34,6 +34,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuBroadcastJoinMeta.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuBroadcastJoinMeta.scala index 037a1506dfd..e954466439d 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuBroadcastJoinMeta.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuBroadcastJoinMeta.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,6 +18,7 @@ {"spark": "332db"} {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuCastShims.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuCastShims.scala index afc484efd45..1672c49c18a 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuCastShims.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuCastShims.scala @@ -34,6 +34,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuRunningWindowExecMeta.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuRunningWindowExecMeta.scala index 71fefd561eb..5e1e35e7b78 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuRunningWindowExecMeta.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuRunningWindowExecMeta.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "332db"} {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuWindowInPandasExec.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuWindowInPandasExec.scala index 5ec9c5be431..b8b93b6423e 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuWindowInPandasExec.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuWindowInPandasExec.scala @@ -19,6 +19,7 @@ {"spark": "332db"} {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/QueryStageExecShims.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/QueryStageExecShims.scala new file mode 100644 index 00000000000..5430f386496 --- /dev/null +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/QueryStageExecShims.scala @@ -0,0 +1,47 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "330db"} +{"spark": "332db"} +{"spark": "341db"} +{"spark": "350db143"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.adaptive.{BroadcastQueryStageExec, ShuffleQueryStageExec} + +object QueryStageExecShims { + def createShuffleQueryStageExec( + oldStage: ShuffleQueryStageExec, + id: Int, + plan: SparkPlan, + canonicalized: SparkPlan, + isSparkExchange: Boolean): ShuffleQueryStageExec = { + ShuffleQueryStageExec(id, plan, canonicalized, isSparkExchange) + } + + def createBroadcastQueryStageExec( + oldStage: BroadcastQueryStageExec, + id: Int, + plan: SparkPlan, + canonicalized: SparkPlan, + isSparkExchange: Boolean): BroadcastQueryStageExec = { + BroadcastQueryStageExec(id, plan, canonicalized, isSparkExchange) + } +} + diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShimBroadcastExchangeLike.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShimBroadcastExchangeLike.scala index e8d0ccd9150..d37793da205 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShimBroadcastExchangeLike.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShimBroadcastExchangeLike.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "332db"} {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShimFilePartitionReaderFactory.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShimFilePartitionReaderFactory.scala index 3b56b47db22..6b1519fe52a 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShimFilePartitionReaderFactory.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShimFilePartitionReaderFactory.scala @@ -34,6 +34,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShimLeafExecNode.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShimLeafExecNode.scala index d72993e3539..8c9743450cd 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShimLeafExecNode.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShimLeafExecNode.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "332db"} {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShuffleOriginUtil.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShuffleOriginUtil.scala index 7059e1664d3..19e6be9a90f 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShuffleOriginUtil.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShuffleOriginUtil.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "332db"} {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/Spark321PlusDBShims.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/Spark321PlusDBShims.scala index c0ba78474d8..f86b131dcbd 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/Spark321PlusDBShims.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/Spark321PlusDBShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "332db"} {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims @@ -33,13 +34,11 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.plans.physical.BroadcastMode import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive._ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.exchange._ -import org.apache.spark.sql.execution.python._ import org.apache.spark.sql.execution.window._ import org.apache.spark.sql.rapids.GpuSubstring import org.apache.spark.sql.rapids.execution.GpuSubqueryBroadcastMeta @@ -75,7 +74,8 @@ trait Spark321PlusDBShims extends SparkShims override def newBroadcastQueryStageExec( old: BroadcastQueryStageExec, newPlan: SparkPlan): BroadcastQueryStageExec = - BroadcastQueryStageExec(old.id, newPlan, old.originalPlan, old.isSparkExchange) + QueryStageExecShims.createBroadcastQueryStageExec( + old, old.id, newPlan, old.originalPlan, old.isSparkExchange) override def filesFromFileIndex(fileCatalog: PartitioningAwareFileIndex): Seq[FileStatus] = { fileCatalog.allFiles().map(_.toFileStatus) @@ -83,9 +83,6 @@ trait Spark321PlusDBShims extends SparkShims override def neverReplaceShowCurrentNamespaceCommand: ExecRule[_ <: SparkPlan] = null - override def getWindowExpressions(winPy: WindowInPandasExec): Seq[NamedExpression] = - winPy.projectList - override def isWindowFunctionExec(plan: SparkPlan): Boolean = plan.isInstanceOf[WindowExecBase] || plan.isInstanceOf[RunningWindowFunctionExec] @@ -166,4 +163,4 @@ trait Spark321PlusDBShims extends SparkShims } override def reproduceEmptyStringBug: Boolean = true -} \ No newline at end of file +} diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/Spark330PlusDBShims.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/Spark330PlusDBShims.scala index 206161c3171..fac3314c709 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/Spark330PlusDBShims.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/Spark330PlusDBShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "332db"} {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/TypeUtilsShims.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/TypeUtilsShims.scala index 6f7096ba13f..fb47191da72 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/TypeUtilsShims.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/TypeUtilsShims.scala @@ -34,6 +34,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/parquet/ParquetStringPredShims.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/parquet/ParquetStringPredShims.scala index 9b6f8ec5826..d233aa15ee1 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/parquet/ParquetStringPredShims.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/parquet/ParquetStringPredShims.scala @@ -34,6 +34,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/execution/GpuSubqueryBroadcastMeta330DBBase.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/execution/GpuSubqueryBroadcastMeta330DBBase.scala index a6248127bad..e14a7cf3871 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/execution/GpuSubqueryBroadcastMeta330DBBase.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/execution/GpuSubqueryBroadcastMeta330DBBase.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "332db"} {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/execution/shims/ReuseGpuBroadcastExchangeAndSubquery.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/execution/shims/ReuseGpuBroadcastExchangeAndSubquery.scala index cab4f788588..4d07d863419 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/execution/shims/ReuseGpuBroadcastExchangeAndSubquery.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/execution/shims/ReuseGpuBroadcastExchangeAndSubquery.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2026, NVIDIA CORPORATION. * * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -22,6 +22,7 @@ {"spark": "332db"} {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution.shims diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/shims/GpuDatabricksShuffleExchangeExecBase.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/shims/GpuDatabricksShuffleExchangeExecBase.scala index cebe693131a..a51be46d61c 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/shims/GpuDatabricksShuffleExchangeExecBase.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/shims/GpuDatabricksShuffleExchangeExecBase.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,6 +18,7 @@ {"spark": "332db"} {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package org.apache.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ParquetCVShims.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ParquetCVShims.scala index f6bda664c59..9a30d5946b8 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ParquetCVShims.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ParquetCVShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -24,6 +24,7 @@ {"spark": "343"} {"spark": "344"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.execution.datasources.parquet diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/execution/datasources/rapids/DataSourceStrategyUtils.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/execution/datasources/rapids/DataSourceStrategyUtils.scala index 109cbb49077..2f789339d4a 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/execution/datasources/rapids/DataSourceStrategyUtils.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/execution/datasources/rapids/DataSourceStrategyUtils.scala @@ -34,6 +34,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/GpuCheckOverflowInTableInsert.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/GpuCheckOverflowInTableInsert.scala index ff876402b83..c4b2c6048ae 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/GpuCheckOverflowInTableInsert.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/GpuCheckOverflowInTableInsert.scala @@ -38,6 +38,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/aggregate/aggregateFunctions.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/aggregate/aggregateFunctions.scala index 981692c7dfb..c4141c0b6ba 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/aggregate/aggregateFunctions.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/aggregate/aggregateFunctions.scala @@ -34,6 +34,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/arithmetic.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/arithmetic.scala index cef7b1fe9ed..2930ff192b4 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/arithmetic.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/arithmetic.scala @@ -34,6 +34,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala index 18154b6f028..395c5b19bfc 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2025, NVIDIA CORPORATION. + * Copyright (c) 2023-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "332db"} {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala index 1beb4a17864..8eb38b2a330 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2025, NVIDIA CORPORATION. + * Copyright (c) 2023-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "332db"} {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuExecutorBroadcastHelper.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuExecutorBroadcastHelper.scala index 0b16a8be869..ff79630cfe7 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuExecutorBroadcastHelper.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuExecutorBroadcastHelper.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2025, NVIDIA CORPORATION. + * Copyright (c) 2023-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "332db"} {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuShuffleMeta.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuShuffleMeta.scala index 3042d148fae..e9e3e7d1e6f 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuShuffleMeta.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuShuffleMeta.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "332db"} {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/ShimTrampolineUtil.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/ShimTrampolineUtil.scala index bf72e94cb85..e51ea6e5992 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/ShimTrampolineUtil.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/ShimTrampolineUtil.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "332db"} {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/shims/GpuAscii.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/shims/GpuAscii.scala index 27d72f96055..1701a4ea59c 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/shims/GpuAscii.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/shims/GpuAscii.scala @@ -38,6 +38,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtilsBase.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtilsBase.scala index 8ff6522952f..5249afb2508 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtilsBase.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtilsBase.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2025, NVIDIA CORPORATION. + * Copyright (c) 2022-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "332db"} {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala index af037e6e3f7..5935151befa 100644 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala +++ b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala @@ -33,6 +33,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuFileFormatDataWriterShim.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuFileFormatDataWriterShim.scala index d59130a9550..0638fda1935 100644 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuFileFormatDataWriterShim.scala +++ b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuFileFormatDataWriterShim.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,6 +18,7 @@ {"spark": "332db"} {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala index 8af88b68266..99adaf63007 100644 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala +++ b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala @@ -33,6 +33,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuKnownNullable.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuKnownNullable.scala index 9817f142a7f..e91c7e7279c 100644 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuKnownNullable.scala +++ b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuKnownNullable.scala @@ -33,6 +33,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuOptimizedCreateHiveTableAsSelectCommandShims.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuOptimizedCreateHiveTableAsSelectCommandShims.scala index 099ba84f724..6a6e27af879 100644 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuOptimizedCreateHiveTableAsSelectCommandShims.scala +++ b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuOptimizedCreateHiveTableAsSelectCommandShims.scala @@ -33,6 +33,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/Spark332PlusDBShims.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/Spark332PlusDBShims.scala index 3bb59bad787..ad235624b6b 100644 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/Spark332PlusDBShims.scala +++ b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/Spark332PlusDBShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2025, NVIDIA CORPORATION. + * Copyright (c) 2023-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,6 +18,7 @@ {"spark": "332db"} {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/execution/datasources/GpuWriteFiles.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/execution/datasources/GpuWriteFiles.scala index f631180b028..3f81e1a1b29 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/execution/datasources/GpuWriteFiles.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/execution/datasources/GpuWriteFiles.scala @@ -33,6 +33,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/HiveFileUtil.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/HiveFileUtil.scala index 489bfe90434..307f0e14665 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/HiveFileUtil.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/HiveFileUtil.scala @@ -33,6 +33,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/HiveProviderCmdShims.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/HiveProviderCmdShims.scala index 16fa89c280e..634f3e3dab1 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/HiveProviderCmdShims.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/HiveProviderCmdShims.scala @@ -33,6 +33,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala index 4d947322772..6f8df8401d4 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala @@ -33,6 +33,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCastToNumberErrorShim.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCastToNumberErrorShim.scala index 1516366a59e..b5ede00ef96 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCastToNumberErrorShim.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCastToNumberErrorShim.scala @@ -33,6 +33,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala index f4f9d3542c9..bcb9a3c8b8b 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala @@ -33,6 +33,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuDataSource.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuDataSource.scala index 67756d1f992..70f8fba925a 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuDataSource.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuDataSource.scala @@ -33,6 +33,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SchemaMetadataShims.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SchemaMetadataShims.scala index 7d2e108ef9f..92ec1517c89 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SchemaMetadataShims.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SchemaMetadataShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -23,6 +23,7 @@ {"spark": "343"} {"spark": "344"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SchemaUtilsShims.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SchemaUtilsShims.scala index c773ea39fc2..1658d12ca04 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SchemaUtilsShims.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SchemaUtilsShims.scala @@ -33,6 +33,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SparkDateTimeExceptionShims.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SparkDateTimeExceptionShims.scala index 215e47657e8..caa7c84dcd7 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SparkDateTimeExceptionShims.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SparkDateTimeExceptionShims.scala @@ -33,6 +33,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SparkUpgradeExceptionShims.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SparkUpgradeExceptionShims.scala index f763c1203bd..acda623f772 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SparkUpgradeExceptionShims.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SparkUpgradeExceptionShims.scala @@ -33,6 +33,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark334/scala/com/nvidia/spark/rapids/shims/GetSequenceSize.scala b/sql-plugin/src/main/spark334/scala/com/nvidia/spark/rapids/shims/GetSequenceSize.scala index 72f948772a6..aed04d721a1 100644 --- a/sql-plugin/src/main/spark334/scala/com/nvidia/spark/rapids/shims/GetSequenceSize.scala +++ b/sql-plugin/src/main/spark334/scala/com/nvidia/spark/rapids/shims/GetSequenceSize.scala @@ -28,6 +28,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/CastCheckShims.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/CastCheckShims.scala index 10db4b439e7..8ec75bb328e 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/CastCheckShims.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/CastCheckShims.scala @@ -33,6 +33,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ConvShim.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ConvShim.scala index 6fe291eb2a4..42fa49b060b 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ConvShim.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ConvShim.scala @@ -32,6 +32,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GlobalLimitShims.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GlobalLimitShims.scala index a80a19c4104..8fc31886b92 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GlobalLimitShims.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GlobalLimitShims.scala @@ -32,6 +32,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/KeyGroupedPartitioningShim.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/KeyGroupedPartitioningShim.scala index e7cf5a655ad..2c6578cf7f8 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/KeyGroupedPartitioningShim.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/KeyGroupedPartitioningShim.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -21,6 +21,7 @@ {"spark": "343"} {"spark": "344"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/OrcProtoWriterShim.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/OrcProtoWriterShim.scala index 3b0910a5779..047f25fd910 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/OrcProtoWriterShim.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/OrcProtoWriterShim.scala @@ -33,6 +33,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/RoundShims.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/RoundShims.scala index 861526cabc3..21736f441a3 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/RoundShims.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/RoundShims.scala @@ -32,6 +32,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/TagScanForRuntimeFiltering.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/TagScanForRuntimeFiltering.scala index abd8d53e000..a679e40082a 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/TagScanForRuntimeFiltering.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/TagScanForRuntimeFiltering.scala @@ -32,6 +32,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/parquet/GpuParquetUtilsShims.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/parquet/GpuParquetUtilsShims.scala index 0298fe2858d..d844e6d063f 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/parquet/GpuParquetUtilsShims.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/parquet/GpuParquetUtilsShims.scala @@ -32,6 +32,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/parquet/ParquetLegacyNanoAsLongShims.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/parquet/ParquetLegacyNanoAsLongShims.scala index 9daa19acb6e..2a07680036b 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/parquet/ParquetLegacyNanoAsLongShims.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/parquet/ParquetLegacyNanoAsLongShims.scala @@ -32,6 +32,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/parquet/ParquetTimestampAnnotationShims.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/parquet/ParquetTimestampAnnotationShims.scala index c1560ca7d85..0ffcb8b8cc5 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/parquet/ParquetTimestampAnnotationShims.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/parquet/ParquetTimestampAnnotationShims.scala @@ -32,6 +32,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/parquet/ParquetTimestampNTZShims.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/parquet/ParquetTimestampNTZShims.scala index b491f9294cc..3bee9cf16cf 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/parquet/ParquetTimestampNTZShims.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/parquet/ParquetTimestampNTZShims.scala @@ -32,6 +32,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala index 3fa34197ef9..4bf7215e2a0 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala @@ -32,6 +32,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/catalyst/csv/GpuCsvUtils.scala b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/catalyst/csv/GpuCsvUtils.scala index 4d0696e3181..ca1e03fef4a 100644 --- a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/catalyst/csv/GpuCsvUtils.scala +++ b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/catalyst/csv/GpuCsvUtils.scala @@ -32,6 +32,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/catalyst/json/GpuJsonUtils.scala b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/catalyst/json/GpuJsonUtils.scala index 147dfb6d2a7..926af663ce0 100644 --- a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/catalyst/json/GpuJsonUtils.scala +++ b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/catalyst/json/GpuJsonUtils.scala @@ -32,6 +32,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/errors/ConvUtils.scala b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/errors/ConvUtils.scala index 3c0f3cbf405..d9a669771a4 100644 --- a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/errors/ConvUtils.scala +++ b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/errors/ConvUtils.scala @@ -32,6 +32,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/RapidsCachingReader.scala b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/RapidsCachingReader.scala index 61ee96c4138..913857c871f 100644 --- a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/RapidsCachingReader.scala +++ b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/RapidsCachingReader.scala @@ -32,6 +32,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/GpuJsonToStructsShim.scala b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/GpuJsonToStructsShim.scala index ec37ba7041c..c3dc0ccdbfc 100644 --- a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/GpuJsonToStructsShim.scala +++ b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/GpuJsonToStructsShim.scala @@ -31,6 +31,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils340PlusBase.scala b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils340PlusBase.scala index dafac1efa9d..160e03fc552 100644 --- a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils340PlusBase.scala +++ b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils340PlusBase.scala @@ -30,6 +30,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuToPrettyString.scala b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuToPrettyString.scala index 59d4260bf7d..dd80607daf3 100644 --- a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuToPrettyString.scala +++ b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuToPrettyString.scala @@ -27,6 +27,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuWindowGroupLimitExec.scala b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuWindowGroupLimitExec.scala index 0235f3c1b04..e385fee8a63 100644 --- a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuWindowGroupLimitExec.scala +++ b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuWindowGroupLimitExec.scala @@ -27,6 +27,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala index 6cb1004e10a..30a1ffda2d4 100644 --- a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala +++ b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala @@ -18,6 +18,7 @@ {"spark": "341db"} {"spark": "350db143"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/PythonUDFShim.scala b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/PythonUDFShim.scala index 367ca9ff6d5..44d31501157 100644 --- a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/PythonUDFShim.scala +++ b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/PythonUDFShim.scala @@ -27,6 +27,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/Spark341PlusDBShims.scala b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/Spark341PlusDBShims.scala index 7bb4c7be1dd..1daebe521b6 100644 --- a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/Spark341PlusDBShims.scala +++ b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/Spark341PlusDBShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2025, NVIDIA CORPORATION. + * Copyright (c) 2023-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims @@ -108,9 +109,9 @@ trait Spark341PlusDBShims extends Spark332PlusDBShims { (takeExec, conf, p, r) => new SparkPlanMeta[TakeOrderedAndProjectExec](takeExec, conf, p, r) { val sortOrder: Seq[BaseExprMeta[SortOrder]] = - takeExec.sortOrder.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + takeExec.sortOrder.map(GpuOverrides.wrapExpr(_, this.conf, Some(this))) val projectList: Seq[BaseExprMeta[NamedExpression]] = - takeExec.projectList.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + takeExec.projectList.map(GpuOverrides.wrapExpr(_, this.conf, Some(this))) override val childExprs: Seq[BaseExprMeta[_]] = sortOrder ++ projectList override def convertToGpu(): GpuExec = { @@ -203,10 +204,15 @@ trait Spark341PlusDBShims extends Spark332PlusDBShims { override def checkCToRWithExecBroadcastAQECoalPart(p: SparkPlan, parent: Option[SparkPlan]): Boolean = { p match { - case ColumnarToRowExec(AQEShuffleReadExec(_: ShuffleQueryStageExec, _, _)) => - parent match { - case Some(bhje: BroadcastHashJoinExec) if bhje.isExecutorBroadcast => true - case Some(bhnlj: BroadcastNestedLoopJoinExec) if bhnlj.isExecutorBroadcast => true + case ColumnarToRowExec(aqe: AQEShuffleReadExec) => + aqe.child match { + case _: ShuffleQueryStageExec => + parent match { + case Some(bhje: BroadcastHashJoinExec) if bhje.isExecutorBroadcast => true + case Some(bhnlj: BroadcastNestedLoopJoinExec) + if bhnlj.isExecutorBroadcast => true + case _ => false + } case _ => false } case _ => false @@ -221,7 +227,11 @@ trait Spark341PlusDBShims extends Spark332PlusDBShims { */ override def getShuffleFromCToRWithExecBroadcastAQECoalPart(p: SparkPlan): Option[SparkPlan] = { p match { - case ColumnarToRowExec(AQEShuffleReadExec(s: ShuffleQueryStageExec, _, _)) => Some(s) + case ColumnarToRowExec(aqe: AQEShuffleReadExec) => + aqe.child match { + case s: ShuffleQueryStageExec => Some(s) + case _ => None + } case _ => None } } diff --git a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/StaticPartitionShims.scala b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/StaticPartitionShims.scala index 9a94f367b55..4846c117342 100644 --- a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/StaticPartitionShims.scala +++ b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/StaticPartitionShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/datasources/v2/rapids/GpuAtomicCreateTableAsSelectExec.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/datasources/v2/rapids/GpuAtomicCreateTableAsSelectExec.scala index 103185a12e0..db0233b11b5 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/datasources/v2/rapids/GpuAtomicCreateTableAsSelectExec.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/datasources/v2/rapids/GpuAtomicCreateTableAsSelectExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,9 +17,11 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.execution.datasources.v2.rapids +import scala.annotation.nowarn import scala.collection.JavaConverters._ import com.nvidia.spark.rapids.GpuExec @@ -69,9 +71,11 @@ case class GpuAtomicCreateTableAsSelectExec( throw QueryCompilationErrors.tableAlreadyExistsError(ident) } val schema = CharVarcharUtils.getRawSchema(query.schema, conf).asNullable - val stagedTable = catalog.stageCreate( + val stagedTable = (catalog.stageCreate( ident, getV2Columns(schema, catalog.useNullableQuerySchema), - partitioning.toArray, properties.asJava) + partitioning.toArray, + properties.asJava): @nowarn( + "msg=stageCreate in trait StagingTableCatalog is deprecated")) writeToTable(catalog, stagedTable, writeOptions, ident, query) } diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/datasources/v2/rapids/GpuAtomicReplaceTableAsSelectExec.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/datasources/v2/rapids/GpuAtomicReplaceTableAsSelectExec.scala index f9098c9d468..e436bea0ea6 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/datasources/v2/rapids/GpuAtomicReplaceTableAsSelectExec.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/datasources/v2/rapids/GpuAtomicReplaceTableAsSelectExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,9 +17,11 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.execution.datasources.v2.rapids +import scala.annotation.nowarn import scala.collection.JavaConverters._ import com.nvidia.spark.rapids.GpuExec @@ -73,12 +75,18 @@ case class GpuAtomicReplaceTableAsSelectExec( invalidateCache(catalog, table, ident) } val staged = if (orCreate) { - catalog.stageCreateOrReplace( - ident, schema, partitioning.toArray, properties.asJava) + (catalog.stageCreateOrReplace( + ident, schema, partitioning.toArray, + properties.asJava + ): @nowarn( + "msg=stageCreateOrReplace in trait StagingTableCatalog is deprecated")) } else if (catalog.tableExists(ident)) { try { - catalog.stageReplace( - ident, schema, partitioning.toArray, properties.asJava) + (catalog.stageReplace( + ident, schema, partitioning.toArray, + properties.asJava + ): @nowarn( + "msg=stageReplace in trait StagingTableCatalog is deprecated")) } catch { case e: NoSuchTableException => throw QueryCompilationErrors.cannotReplaceMissingTableError(ident, Some(e)) diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala index 429ea342fd8..5753adfe167 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -33,4 +33,10 @@ object FilePartitionShims extends SplitFiles { } } } -} \ No newline at end of file + + def getFiles(p: FilePartition): Array[PartitionedFile] = p.files + + def copyWithFiles(p: FilePartition, newFiles: Array[PartitionedFile]): FilePartition = { + p.copy(files = newFiles) + } +} diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/rapids/shims/SplitFiles.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/rapids/shims/SplitFiles.scala index ae26f7e0863..6482c58a57b 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/rapids/shims/SplitFiles.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/rapids/shims/SplitFiles.scala @@ -28,6 +28,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/CreateFunctions.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/CreateFunctions.scala index 327560f3a8d..f0bfb0cf69b 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/CreateFunctions.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/CreateFunctions.scala @@ -27,6 +27,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/FileSinkDescShim.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/FileSinkDescShim.scala index 240f25ed593..16278342518 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/FileSinkDescShim.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/FileSinkDescShim.scala @@ -28,6 +28,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/HiveInspectorsShim.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/HiveInspectorsShim.scala index a982ee797dd..a7c30aee2bc 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/HiveInspectorsShim.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/HiveInspectorsShim.scala @@ -28,6 +28,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonOutput.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonOutput.scala index 5a0294d31f8..3f83fa6b9d5 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonOutput.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonOutput.scala @@ -18,6 +18,7 @@ {"spark": "341db"} {"spark": "350db143"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonRunner.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonRunner.scala index e8b1e929ce8..6ffa0abab53 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonRunner.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonRunner.scala @@ -17,6 +17,7 @@ {"spark": "341db"} {"spark": "350db143"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuCoGroupedArrowPythonRunner.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuCoGroupedArrowPythonRunner.scala index f6aa8597f5e..e665435eb25 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuCoGroupedArrowPythonRunner.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuCoGroupedArrowPythonRunner.scala @@ -18,6 +18,7 @@ {"spark": "341db"} {"spark": "350db143"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupUDFArrowPythonRunner.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupUDFArrowPythonRunner.scala index 7c942e6753f..1946f031e98 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupUDFArrowPythonRunner.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupUDFArrowPythonRunner.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2025, NVIDIA CORPORATION. + * Copyright (c) 2021-2026, NVIDIA CORPORATION. * * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -20,6 +20,7 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution.python.shims diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupedPythonRunnerFactory.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupedPythonRunnerFactory.scala index d07e7668500..ea75745de6c 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupedPythonRunnerFactory.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupedPythonRunnerFactory.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2025, NVIDIA CORPORATION. + * Copyright (c) 2023-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution.python.shims diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils341DBPlusBase.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils341DBPlusBase.scala index ee2c2c6462b..8700afbc013 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils341DBPlusBase.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils341DBPlusBase.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ /*** spark-rapids-shim-json-lines {"spark": "341db"} {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark342/scala/com/nvidia/spark/rapids/shims/DecimalMultiply128.scala b/sql-plugin/src/main/spark342/scala/com/nvidia/spark/rapids/shims/DecimalMultiply128.scala index 63373f700b9..82c3aa73acf 100644 --- a/sql-plugin/src/main/spark342/scala/com/nvidia/spark/rapids/shims/DecimalMultiply128.scala +++ b/sql-plugin/src/main/spark342/scala/com/nvidia/spark/rapids/shims/DecimalMultiply128.scala @@ -28,6 +28,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/BatchScanExecMetaBase.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/BatchScanExecMetaBase.scala index d1ff11ec109..4a141e44aa1 100644 --- a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/BatchScanExecMetaBase.scala +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/BatchScanExecMetaBase.scala @@ -26,6 +26,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} @@ -34,7 +35,6 @@ package com.nvidia.spark.rapids.shims import com.nvidia.spark.rapids._ -import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.datasources.v2.BatchScanExec @@ -57,12 +57,12 @@ abstract class BatchScanExecMetaBase(p: BatchScanExec, } wrapped.runtimeFilters.map { filter => filter.transformDown { - case dpe @ DynamicPruningExpression(inSub: InSubqueryExec) => + case dpe @ DynamicPruningShims(inSub: InSubqueryExec) => inSub.plan match { case bc: SubqueryBroadcastExec => - dpe.copy(inSub.copy(plan = convertBroadcast(bc))) + DynamicPruningShims(inSub.copy(plan = convertBroadcast(bc))) case reuse @ ReusedSubqueryExec(bc: SubqueryBroadcastExec) => - dpe.copy(inSub.copy(plan = reuse.copy(convertBroadcast(bc)))) + DynamicPruningShims(inSub.copy(plan = reuse.copy(convertBroadcast(bc)))) case _ => dpe } diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/GpuIntervalUtils.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/GpuIntervalUtils.scala index c666f3e17f1..9ff897b76bd 100644 --- a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/GpuIntervalUtils.scala +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/GpuIntervalUtils.scala @@ -24,6 +24,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/LegacyBehaviorPolicyShim.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/LegacyBehaviorPolicyShim.scala index ff3d390ea31..a3220b961cf 100644 --- a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/LegacyBehaviorPolicyShim.scala +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/LegacyBehaviorPolicyShim.scala @@ -26,6 +26,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/NullOutputStreamShim.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/NullOutputStreamShim.scala index 43b05da8aaf..319964ee6e9 100644 --- a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/NullOutputStreamShim.scala +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/NullOutputStreamShim.scala @@ -26,6 +26,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala index 613b3606532..5e50261afb9 100644 --- a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala +++ b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2025, NVIDIA CORPORATION. + * Copyright (c) 2023-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -34,4 +34,10 @@ object FilePartitionShims extends SplitFiles { } } } + + def getFiles(p: FilePartition): Array[PartitionedFile] = p.files + + def copyWithFiles(p: FilePartition, newFiles: Array[PartitionedFile]): FilePartition = { + p.copy(files = newFiles) + } } diff --git a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/ArrowUtilsShim.scala b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/ArrowUtilsShim.scala index cd59e5034fc..8bddfcd2750 100644 --- a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/ArrowUtilsShim.scala +++ b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/ArrowUtilsShim.scala @@ -26,6 +26,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/DataTypeUtilsShim.scala b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/DataTypeUtilsShim.scala index 66994e9ef61..bd49fba8dce 100644 --- a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/DataTypeUtilsShim.scala +++ b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/DataTypeUtilsShim.scala @@ -26,6 +26,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/RapidsShuffleManager.scala b/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/RapidsShuffleManager.scala index 10d60dc65a8..8c80c7e81ec 100644 --- a/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/RapidsShuffleManager.scala +++ b/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/RapidsShuffleManager.scala @@ -17,6 +17,7 @@ /*** spark-rapids-shim-json-lines {"spark": "350db143"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/BatchScanExecMeta.scala b/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/BatchScanExecMeta.scala index 6abaad160b5..ef6b456253d 100644 --- a/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/BatchScanExecMeta.scala +++ b/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/BatchScanExecMeta.scala @@ -17,6 +17,7 @@ /*** spark-rapids-shim-json-lines {"spark": "350db143"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/CastTimeToIntShim.scala b/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/CastTimeToIntShim.scala index ce31cc38bb2..5c529226fea 100644 --- a/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/CastTimeToIntShim.scala +++ b/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/CastTimeToIntShim.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "350db143"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRow.scala b/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRow.scala index 4b1315f1075..af0080df686 100644 --- a/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRow.scala +++ b/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRow.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -15,6 +15,7 @@ */ /*** spark-rapids-shim-json-lines {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/DeltaLakeUtils.scala b/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/DeltaLakeUtils.scala index 2ca639f8879..95dc0fd713f 100644 --- a/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/DeltaLakeUtils.scala +++ b/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/DeltaLakeUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2025, NVIDIA CORPORATION. + * Copyright (c) 2022-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala b/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala index 5867e4c15b1..9b77115565f 100644 --- a/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala +++ b/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala @@ -17,6 +17,7 @@ /*** spark-rapids-shim-json-lines {"spark": "350db143"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} @@ -65,7 +66,7 @@ case class GpuBatchScanExec( @transient override protected lazy val filteredPartitions: Seq[Seq[InputPartition]] = { val dataSourceFilters = runtimeFilters.flatMap { - case DynamicPruningExpression(e) => DataSourceStrategyUtils.translateRuntimeFilter(e) + case DynamicPruningShims(e) => DataSourceStrategyUtils.translateRuntimeFilter(e) case _ => None } diff --git a/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShimBase.scala b/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShimBase.scala index 8a66e9559a6..01856ae95d0 100644 --- a/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShimBase.scala +++ b/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShimBase.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/PythonMapInArrowExecShims.scala b/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/PythonMapInArrowExecShims.scala index 4de832eab39..a202a43440a 100644 --- a/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/PythonMapInArrowExecShims.scala +++ b/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/PythonMapInArrowExecShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/RaiseErrorShim.scala b/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/RaiseErrorShim.scala index e0e9c5daf37..a398ac6d6c4 100644 --- a/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/RaiseErrorShim.scala +++ b/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/RaiseErrorShim.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "350db143"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/ShuffleManagerShimUtils.scala b/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/ShuffleManagerShimUtils.scala index 9014a1bbde8..68c8c6cca9a 100644 --- a/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/ShuffleManagerShimUtils.scala +++ b/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/ShuffleManagerShimUtils.scala @@ -17,6 +17,7 @@ /*** spark-rapids-shim-json-lines {"spark": "350db143"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/StoragePartitionJoinShims.scala b/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/StoragePartitionJoinShims.scala index 594ff43b581..3295b3d43bc 100644 --- a/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/StoragePartitionJoinShims.scala +++ b/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/StoragePartitionJoinShims.scala @@ -28,10 +28,12 @@ import org.apache.spark.sql.execution.datasources.v2.StoragePartitionJoinParams /** * Shim for StoragePartitionJoinParams to handle package location change. * In Spark 3.5.0-db143 and 4.0.x, it's in org.apache.spark.sql.execution.datasources.v2 - * In Spark 4.1.0+, it moved to org.apache.spark.sql.execution.joins + * In Spark 4.1.0+ and 400db173, it moved to org.apache.spark.sql.execution.joins */ object StoragePartitionJoinShims { type SpjParams = StoragePartitionJoinParams - + def default(): SpjParams = StoragePartitionJoinParams() + + def fromBatchScan(spjParams: StoragePartitionJoinParams): SpjParams = spjParams } diff --git a/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/spark350db143/SparkShimServiceProvider.scala b/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/spark350db143/SparkShimServiceProvider.scala index c0aa49290e8..55bcad35ea9 100644 --- a/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/spark350db143/SparkShimServiceProvider.scala +++ b/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/spark350db143/SparkShimServiceProvider.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2025, NVIDIA CORPORATION. + * Copyright (c) 2023-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims.spark350db143 diff --git a/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/execution/datasources/parquet/RapidsVectorizedColumnReader.scala b/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/execution/datasources/parquet/RapidsVectorizedColumnReader.scala index 271d9a6dae7..e1d3ad630eb 100644 --- a/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/execution/datasources/parquet/RapidsVectorizedColumnReader.scala +++ b/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/execution/datasources/parquet/RapidsVectorizedColumnReader.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.execution.datasources.parquet diff --git a/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/RapidsShuffleWriter.scala b/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/RapidsShuffleWriter.scala index c080ff9ae9c..f9dcbc8aa9d 100644 --- a/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/RapidsShuffleWriter.scala +++ b/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/RapidsShuffleWriter.scala @@ -17,6 +17,7 @@ /*** spark-rapids-shim-json-lines {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids diff --git a/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala b/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala index 10e3fa68b76..fd37a1bbd49 100644 --- a/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala +++ b/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -15,6 +15,7 @@ */ /*** spark-rapids-shim-json-lines {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution diff --git a/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/execution/python/shims/PythonArgumentsUtils.scala b/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/execution/python/shims/PythonArgumentsUtils.scala index dc3fb39b1ec..f67afdd0015 100644 --- a/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/execution/python/shims/PythonArgumentsUtils.scala +++ b/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/execution/python/shims/PythonArgumentsUtils.scala @@ -17,6 +17,7 @@ /*** spark-rapids-shim-json-lines {"spark": "350db143"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/shims/ArrayInvalidArgumentErrorUtils.scala b/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/shims/ArrayInvalidArgumentErrorUtils.scala index b21779e674f..c65836f2f09 100644 --- a/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/shims/ArrayInvalidArgumentErrorUtils.scala +++ b/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/shims/ArrayInvalidArgumentErrorUtils.scala @@ -17,6 +17,7 @@ /*** spark-rapids-shim-json-lines {"spark": "350db143"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExec.scala b/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExec.scala index ea1af0ecbbb..e82e3083958 100644 --- a/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExec.scala +++ b/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExecMeta.scala b/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExecMeta.scala index 8ddd15f7171..afe9d14d9f2 100644 --- a/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExecMeta.scala +++ b/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExecMeta.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index b081cb89826..7f148eeec72 100644 --- a/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024-2025, NVIDIA CORPORATION. + * Copyright (c) 2024-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "350db143"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/shims/SequenceSizeExceededLimitErrorBuilder.scala b/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/shims/SequenceSizeExceededLimitErrorBuilder.scala index b18517ea046..c42f21207a1 100644 --- a/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/shims/SequenceSizeExceededLimitErrorBuilder.scala +++ b/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/shims/SequenceSizeExceededLimitErrorBuilder.scala @@ -17,6 +17,7 @@ /*** spark-rapids-shim-json-lines {"spark": "350db143"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/shims/misc.scala b/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/shims/misc.scala index e4922d4f2ff..8862115569d 100644 --- a/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/shims/misc.scala +++ b/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/shims/misc.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "350db143"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark352/scala/com/nvidia/spark/rapids/shims/InMemoryTableScanExecLikeShim.scala b/sql-plugin/src/main/spark352/scala/com/nvidia/spark/rapids/shims/InMemoryTableScanExecLikeShim.scala index 6faaa29ef01..5e88e13a381 100644 --- a/sql-plugin/src/main/spark352/scala/com/nvidia/spark/rapids/shims/InMemoryTableScanExecLikeShim.scala +++ b/sql-plugin/src/main/spark352/scala/com/nvidia/spark/rapids/shims/InMemoryTableScanExecLikeShim.scala @@ -23,6 +23,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark355/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala b/sql-plugin/src/main/spark355/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala index b1ae218a21c..eaf8ca5be29 100644 --- a/sql-plugin/src/main/spark355/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala +++ b/sql-plugin/src/main/spark355/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2025, NVIDIA CORPORATION. + * Copyright (c) 2023-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -33,4 +33,10 @@ object FilePartitionShims extends SplitFiles { } } } + + def getFiles(p: FilePartition): Array[PartitionedFile] = p.files + + def copyWithFiles(p: FilePartition, newFiles: Array[PartitionedFile]): FilePartition = { + p.copy(files = newFiles) + } } diff --git a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/DateTimeUtilsShims.scala b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/DateTimeUtilsShims.scala index 1b2fac82b83..21254c4b39a 100644 --- a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/DateTimeUtilsShims.scala +++ b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/DateTimeUtilsShims.scala @@ -17,6 +17,7 @@ /*** spark-rapids-shim-json-lines {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/GetJsonObjectShim.scala b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/GetJsonObjectShim.scala index bb47ece1d0a..fa10aa6e489 100644 --- a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/GetJsonObjectShim.scala +++ b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/GetJsonObjectShim.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader.scala b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader.scala index 80b40e40c3d..6bd9f9c99ae 100644 --- a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader.scala +++ b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader.scala @@ -15,6 +15,7 @@ */ /*** spark-rapids-shim-json-lines {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/LogicalPlanShims.scala b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/LogicalPlanShims.scala index 90c03500f28..f31ca918539 100644 --- a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/LogicalPlanShims.scala +++ b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/LogicalPlanShims.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/NullIntolerantShim.scala b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/NullIntolerantShim.scala index dd0d3608ae7..842846d6aae 100644 --- a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/NullIntolerantShim.scala +++ b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/NullIntolerantShim.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/OperatorsUtilShims.scala b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/OperatorsUtilShims.scala index dfaf2bc636d..79911923a26 100644 --- a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/OperatorsUtilShims.scala +++ b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/OperatorsUtilShims.scala @@ -17,6 +17,7 @@ /*** spark-rapids-shim-json-lines {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala index 9afda0ed7e9..61c7ad036be 100644 --- a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala +++ b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala @@ -17,6 +17,7 @@ /*** spark-rapids-shim-json-lines {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/spark400/SparkShimServiceProvider.scala b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/spark400/SparkShimServiceProvider.scala index b7e1bd26923..95525c673f7 100644 --- a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/spark400/SparkShimServiceProvider.scala +++ b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/spark400/SparkShimServiceProvider.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024-2025, NVIDIA CORPORATION. + * Copyright (c) 2024-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "400"} +{"spark": "400db173"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims.spark400 diff --git a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala index 3e45b3ffcd6..5287a7ea08b 100644 --- a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala +++ b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala @@ -33,4 +33,11 @@ object FilePartitionShims extends SplitFiles { } } } + + // Methods for FilePartition (overloaded) + def getFiles(p: FilePartition): Array[PartitionedFile] = p.files + + def copyWithFiles(p: FilePartition, newFiles: Array[PartitionedFile]): FilePartition = { + p.copy(files = newFiles) + } } diff --git a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/hive/rapids/shims/CommandUtilsShim.scala b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/hive/rapids/shims/CommandUtilsShim.scala index df12aea0b0b..ace99a17807 100644 --- a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/hive/rapids/shims/CommandUtilsShim.scala +++ b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/hive/rapids/shims/CommandUtilsShim.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/nvidia/DFUDFShims.scala b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/nvidia/DFUDFShims.scala index 0e46911f8b3..f3f48170060 100644 --- a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/nvidia/DFUDFShims.scala +++ b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/nvidia/DFUDFShims.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala index 498b9b52851..9302e1c869c 100644 --- a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala +++ b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala @@ -15,6 +15,7 @@ */ /*** spark-rapids-shim-json-lines {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuBasePythonRunner.scala b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuBasePythonRunner.scala index 5427e69966f..dcde32f3dbb 100644 --- a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuBasePythonRunner.scala +++ b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuBasePythonRunner.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/GpuMapInArrowExecMeta.scala b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/GpuMapInArrowExecMeta.scala index e91f8822e57..50028c85019 100644 --- a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/GpuMapInArrowExecMeta.scala +++ b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/GpuMapInArrowExecMeta.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/InvokeExprMeta.scala b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/InvokeExprMeta.scala index d178393d62d..d66e879667a 100644 --- a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/InvokeExprMeta.scala +++ b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/InvokeExprMeta.scala @@ -17,6 +17,7 @@ /*** spark-rapids-shim-json-lines {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index 17110f644f8..4b78b0d01fa 100644 --- a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/SparkSessionUtils.scala b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/SparkSessionUtils.scala index 18d78b8d77c..6de8f1d6165 100644 --- a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/SparkSessionUtils.scala +++ b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/SparkSessionUtils.scala @@ -16,6 +16,7 @@ /*** spark-rapids-shim-json-lines {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/TrampolineConnectShims.scala b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/TrampolineConnectShims.scala index d5dc45f8ab3..9a753370800 100644 --- a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/TrampolineConnectShims.scala +++ b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/TrampolineConnectShims.scala @@ -17,6 +17,7 @@ /*** spark-rapids-shim-json-lines {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/sql-plugin/src/main/spark400db173/java/com/nvidia/spark/rapids/shims/QueryStageRowCountHelper.java b/sql-plugin/src/main/spark400db173/java/com/nvidia/spark/rapids/shims/QueryStageRowCountHelper.java new file mode 100644 index 00000000000..7d08492f2c7 --- /dev/null +++ b/sql-plugin/src/main/spark400db173/java/com/nvidia/spark/rapids/shims/QueryStageRowCountHelper.java @@ -0,0 +1,33 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400db173"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims; + +import org.apache.spark.sql.catalyst.plans.logical.Statistics; +import org.apache.spark.sql.execution.adaptive.QueryStageExec; + +/** + * Java helper to access getRuntimeStatistics on Databricks 17.3. + * The method is Scala protected (compiles to JVM public). + */ +public class QueryStageRowCountHelper { + public static Statistics getRuntimeStatistics(QueryStageExec qse) { + return qse.getRuntimeStatistics(); + } +} diff --git a/sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/AggregateInPandasExecShims.scala b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/AggregateInPandasExecShims.scala similarity index 94% rename from sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/AggregateInPandasExecShims.scala rename to sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/AggregateInPandasExecShims.scala index 46148f178e8..8f4dad61c26 100644 --- a/sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/AggregateInPandasExecShims.scala +++ b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/AggregateInPandasExecShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2026, NVIDIA CORPORATION. + * Copyright (c) 2025-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -15,6 +15,7 @@ */ /*** spark-rapids-shim-json-lines +{"spark": "400db173"} {"spark": "411"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims @@ -26,7 +27,7 @@ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.python.ArrowAggregatePythonExec /** - * AggregateInPandasExec was renamed to ArrowAggregatePythonExec in Spark 4.1.0. + * AggregateInPandasExec was renamed to ArrowAggregatePythonExec in Spark 4.1.0 and Databricks 17.3. */ object AggregateInPandasExecShims { val execRule: Option[ExecRule[_ <: SparkPlan]] = Some( diff --git a/sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/BroadcastExchangeShims.scala b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/BroadcastExchangeShims.scala similarity index 94% rename from sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/BroadcastExchangeShims.scala rename to sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/BroadcastExchangeShims.scala index e7415fb2b63..b8de584054e 100644 --- a/sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/BroadcastExchangeShims.scala +++ b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/BroadcastExchangeShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2026, NVIDIA CORPORATION. + * Copyright (c) 2025-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -15,6 +15,7 @@ */ /*** spark-rapids-shim-json-lines +{"spark": "400db173"} {"spark": "411"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/DayTimeIntervalShims.scala b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/DayTimeIntervalShims.scala similarity index 99% rename from sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/DayTimeIntervalShims.scala rename to sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/DayTimeIntervalShims.scala index cc225c0acd0..aa957363b62 100644 --- a/sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/DayTimeIntervalShims.scala +++ b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/DayTimeIntervalShims.scala @@ -15,6 +15,7 @@ */ /*** spark-rapids-shim-json-lines +{"spark": "400db173"} {"spark": "411"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/DynamicPruningShims.scala b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/DynamicPruningShims.scala new file mode 100644 index 00000000000..66b7c12e7d3 --- /dev/null +++ b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/DynamicPruningShims.scala @@ -0,0 +1,36 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400db173"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import org.apache.spark.sql.catalyst.expressions.{DynamicPruningExpression, Expression} + +/** + * Databricks 17.3 version where DynamicPruningExpression added dynamicPruningInfo parameter. + */ +object DynamicPruningShims { + def unapply(expr: Expression): Option[Expression] = expr match { + case dpe: DynamicPruningExpression => Some(dpe.child) + case _ => None + } + + def apply(child: Expression): DynamicPruningExpression = { + DynamicPruningExpression(child, None) + } +} diff --git a/sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/GpuArrowAggregatePythonExecMeta.scala b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/GpuArrowAggregatePythonExecMeta.scala similarity index 94% rename from sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/GpuArrowAggregatePythonExecMeta.scala rename to sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/GpuArrowAggregatePythonExecMeta.scala index c95ccd5b5ac..6ae883e02c5 100644 --- a/sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/GpuArrowAggregatePythonExecMeta.scala +++ b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/GpuArrowAggregatePythonExecMeta.scala @@ -15,6 +15,7 @@ */ /*** spark-rapids-shim-json-lines +{"spark": "400db173"} {"spark": "411"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims @@ -26,7 +27,8 @@ import org.apache.spark.sql.execution.python.ArrowAggregatePythonExec import org.apache.spark.sql.rapids.execution.python.{GpuAggregateInPandasExec, GpuPythonUDAF} /** - * Meta class for ArrowAggregatePythonExec (renamed from AggregateInPandasExec in Spark 4.1.0). + * Meta class for ArrowAggregatePythonExec (renamed from + * AggregateInPandasExec in Spark 4.1.0 and Databricks 17.3). */ class GpuArrowAggregatePythonExecMeta( aggPandas: ArrowAggregatePythonExec, diff --git a/sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/GpuOneRowRelationExec.scala b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/GpuOneRowRelationExec.scala similarity index 99% rename from sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/GpuOneRowRelationExec.scala rename to sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/GpuOneRowRelationExec.scala index ec22c7014dd..d55ed600ac4 100644 --- a/sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/GpuOneRowRelationExec.scala +++ b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/GpuOneRowRelationExec.scala @@ -15,6 +15,7 @@ */ /*** spark-rapids-shim-json-lines +{"spark": "400db173"} {"spark": "411"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/GpuScalarSubqueryShims.scala b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/GpuScalarSubqueryShims.scala new file mode 100644 index 00000000000..a4c55634645 --- /dev/null +++ b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/GpuScalarSubqueryShims.scala @@ -0,0 +1,30 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400db173"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +/** + * Shim trait for GpuScalarSubquery. + * Databricks 17.3 adds the resultUpdated() method to ExecSubqueryExpression. + */ +trait GpuScalarSubqueryShims { + protected def updated: Boolean + + def resultUpdated(): Boolean = updated +} diff --git a/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/QueryStageExecShims.scala b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/QueryStageExecShims.scala new file mode 100644 index 00000000000..ededb416653 --- /dev/null +++ b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/QueryStageExecShims.scala @@ -0,0 +1,49 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400db173"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.adaptive.{BroadcastQueryStageExec, ShuffleQueryStageExec} + +/** + * Databricks 17.3 version where QueryStageExec.apply requires adaptiveContext parameter. + */ +object QueryStageExecShims { + def createShuffleQueryStageExec( + oldStage: ShuffleQueryStageExec, + id: Int, + plan: SparkPlan, + canonicalized: SparkPlan, + isSparkExchange: Boolean): ShuffleQueryStageExec = { + // DB 17.3 requires adaptiveContext from the old stage + ShuffleQueryStageExec(id, plan, canonicalized, isSparkExchange)(oldStage.adaptiveContext) + } + + def createBroadcastQueryStageExec( + oldStage: BroadcastQueryStageExec, + id: Int, + plan: SparkPlan, + canonicalized: SparkPlan, + isSparkExchange: Boolean): BroadcastQueryStageExec = { + // DB 17.3 requires adaptiveContext from the old stage + BroadcastQueryStageExec(id, plan, canonicalized, isSparkExchange)(oldStage.adaptiveContext) + } +} + diff --git a/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/QueryStageRowCountShims.scala b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/QueryStageRowCountShims.scala new file mode 100644 index 00000000000..e4a2e5c6f84 --- /dev/null +++ b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/QueryStageRowCountShims.scala @@ -0,0 +1,33 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400db173"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import org.apache.spark.sql.execution.adaptive.QueryStageExec + +/** + * Databricks 17.3 version where getRuntimeStatistics is Scala protected. + * Since Scala protected compiles to JVM public, we use a Java helper + * (QueryStageRowCountHelper) to call it directly. + */ +object QueryStageRowCountShims { + def getRowCount(qse: QueryStageExec): Option[BigInt] = { + QueryStageRowCountHelper.getRuntimeStatistics(qse).rowCount + } +} diff --git a/sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/ShowNamespacesExecShims.scala b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/ShowNamespacesExecShims.scala similarity index 87% rename from sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/ShowNamespacesExecShims.scala rename to sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/ShowNamespacesExecShims.scala index 46a7c8d854a..021cca7f20d 100644 --- a/sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/ShowNamespacesExecShims.scala +++ b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/ShowNamespacesExecShims.scala @@ -15,16 +15,17 @@ */ /*** spark-rapids-shim-json-lines +{"spark": "400db173"} {"spark": "411"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims -import com.nvidia.spark.rapids.ExecRule +import com.nvidia.spark.rapids._ import org.apache.spark.sql.execution.SparkPlan /** - * ShowNamespacesExec was removed/renamed in Spark 4.1.0. + * ShowNamespacesExec was removed/renamed in Spark 4.1.0 and Databricks 17.3. * Return None to skip this exec rule. */ object ShowNamespacesExecShims { diff --git a/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/Spark400PlusDBShims.scala b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/Spark400PlusDBShims.scala new file mode 100644 index 00000000000..a73c64f2f09 --- /dev/null +++ b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/Spark400PlusDBShims.scala @@ -0,0 +1,47 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400db173"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import com.nvidia.spark.rapids._ + +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions.objects.Invoke +import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile} +import org.apache.spark.sql.rapids.shims.InvokeExprMeta + +trait Spark400PlusDBShims extends Spark341PlusDBShims { + override def getExprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = { + val shimExprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = Seq( + GpuOverrides.expr[Invoke]( + "Calls the specified function on an object. This is a wrapper to other expressions, so " + + "can not know the details in advance. E.g.: between is replaced by " + + "And(GreaterThanOrEqual(ref, lower), LessThanOrEqual(ref, upper); StructToJson is " + + "replaced by Invoke(Literal(StructsToJsonEvaluator), evaluate, string_type, arguments)", + InvokeCheck, + InvokeExprMeta) + .note("The supported types are not deterministic since it's a dynamic expression") + ).map(r => (r.getClassFor.asSubclass(classOf[Expression]), r)).toMap + super.getExprs ++ shimExprs + } + + override def getPartitionFiles(partition: FilePartition): Seq[PartitionedFile] = { + partition.innerFiles + } +} diff --git a/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/SparkShims.scala b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/SparkShims.scala new file mode 100644 index 00000000000..b82cd95d968 --- /dev/null +++ b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/SparkShims.scala @@ -0,0 +1,39 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400db173"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import com.nvidia.spark.rapids._ + +import org.apache.spark.sql.execution.{OneRowRelationExec, SparkPlan} + +object SparkShimImpl extends Spark400PlusDBShims { + + override def getExecs: Map[Class[_ <: SparkPlan], ExecRule[_ <: SparkPlan]] = { + val shimExecs: Map[Class[_ <: SparkPlan], ExecRule[_ <: SparkPlan]] = Seq( + // OneRowRelationExec is a new class in Spark 4.1.0 for single-row queries (e.g. SELECT 1) + // GPU version produces a single ColumnarBatch with one row and zero columns + GpuOverrides.exec[OneRowRelationExec]( + "Single row relation for literal queries without FROM clause", + ExecChecks(TypeSig.all, TypeSig.all), + (exec, conf, parent, rule) => new GpuOneRowRelationExecMeta(exec, conf, parent, rule)) + ).map(r => (r.getClassFor.asSubclass(classOf[SparkPlan]), r)).toMap + super.getExecs ++ shimExecs + } +} diff --git a/sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/StoragePartitionJoinShims.scala b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/StoragePartitionJoinShims.scala similarity index 98% rename from sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/StoragePartitionJoinShims.scala rename to sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/StoragePartitionJoinShims.scala index 2bfe24ebcf9..f3624d7d7ae 100644 --- a/sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/StoragePartitionJoinShims.scala +++ b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/StoragePartitionJoinShims.scala @@ -15,6 +15,7 @@ */ /*** spark-rapids-shim-json-lines +{"spark": "400db173"} {"spark": "411"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/TimeAddShims.scala b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/TimeAddShims.scala similarity index 92% rename from sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/TimeAddShims.scala rename to sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/TimeAddShims.scala index 000b39e7a81..83d9298bfa2 100644 --- a/sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/TimeAddShims.scala +++ b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/TimeAddShims.scala @@ -15,6 +15,7 @@ */ /*** spark-rapids-shim-json-lines +{"spark": "400db173"} {"spark": "411"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims @@ -24,7 +25,7 @@ import com.nvidia.spark.rapids._ import org.apache.spark.sql.catalyst.expressions.Expression /** - * Empty TimeAddShims for Spark 4.1.0+. + * Empty TimeAddShims for Spark 4.1.0+ and Databricks 17.3. * TimeAdd was renamed to TimestampAddInterval and is handled by DayTimeIntervalShims. */ object TimeAddShims { diff --git a/sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/TryModeShim.scala b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/TryModeShim.scala similarity index 92% rename from sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/TryModeShim.scala rename to sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/TryModeShim.scala index 907bcbdb861..9d155cecb91 100644 --- a/sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/TryModeShim.scala +++ b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/TryModeShim.scala @@ -15,6 +15,7 @@ */ /*** spark-rapids-shim-json-lines +{"spark": "400db173"} {"spark": "411"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims @@ -23,7 +24,8 @@ import org.apache.spark.sql.catalyst.expressions.{Add, Divide, EvalMode, Express import org.apache.spark.sql.catalyst.expressions.aggregate.{Average, Sum} /** - * Spark 4.1.0 version where evalMode changed to evalContext.evalMode for arithmetic. + * Spark 4.1.0+ and Databricks 17.3 version where evalMode + * changed to evalContext.evalMode for arithmetic. * See: https://github.com/apache/spark/commit/a96e9ca81518bff31b0089d459fe78804ca1aa38 */ object TryModeShim { diff --git a/sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/WindowInPandasExecShims.scala b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/WindowInPandasExecShims.scala similarity index 99% rename from sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/WindowInPandasExecShims.scala rename to sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/WindowInPandasExecShims.scala index 44486add156..8a64a717bcd 100644 --- a/sql-plugin/src/main/spark411/scala/com/nvidia/spark/rapids/shims/WindowInPandasExecShims.scala +++ b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/WindowInPandasExecShims.scala @@ -15,6 +15,7 @@ */ /*** spark-rapids-shim-json-lines +{"spark": "400db173"} {"spark": "411"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/WindowInPandasShims.scala b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/WindowInPandasShims.scala new file mode 100644 index 00000000000..9c79f88a0f6 --- /dev/null +++ b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/WindowInPandasShims.scala @@ -0,0 +1,32 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400db173"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import org.apache.spark.sql.catalyst.expressions.NamedExpression +import org.apache.spark.sql.execution.python.ArrowWindowPythonExec + +/** + * WindowInPandasExec was renamed to ArrowWindowPythonExec in Spark 4.1. + * This trait provides the implementation for 4.1+. + */ +trait WindowInPandasShims { + def getWindowExpressions(winPy: ArrowWindowPythonExec): Seq[NamedExpression] = + winPy.projectList +} diff --git a/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/shimExpressions.scala b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/shimExpressions.scala new file mode 100644 index 00000000000..2ec102ddfe6 --- /dev/null +++ b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/shimExpressions.scala @@ -0,0 +1,38 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400db173"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.trees.TreePattern._ + +trait ShimGetArrayStructFields extends ExtractValue { + override def nodePatternsInternal: Seq[TreePattern] = Seq(EXTRACT_ARRAY_SUBFIELDS) +} + +trait ShimGetArrayItem extends ExtractValue { + override def nodePatternsInternal: Seq[TreePattern] = Seq(GET_ARRAY_ITEM) +} + +trait ShimGetStructField extends ExtractValue { + override def nodePatternsInternal: Seq[TreePattern] = Seq(GET_STRUCT_FIELD) +} + +// Fallback to the default definition of `deterministic` +trait GpuDeterministicFirstLastCollectShim extends Expression diff --git a/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/spark400db173/SparkShimServiceProvider.scala b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/spark400db173/SparkShimServiceProvider.scala new file mode 100644 index 00000000000..8f793d05165 --- /dev/null +++ b/sql-plugin/src/main/spark400db173/scala/com/nvidia/spark/rapids/shims/spark400db173/SparkShimServiceProvider.scala @@ -0,0 +1,49 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400db173"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims.spark400db173 + +import com.nvidia.spark.rapids._ + +import org.apache.spark.SparkEnv + +object SparkShimServiceProvider { + // DB version should conform to "major.minor" and has no patch version. + // Refer to VersionUtils.getVersionForJni + val VERSION = DatabricksShimVersion(4, 0, 0, "17.3") +} + +class SparkShimServiceProvider extends com.nvidia.spark.rapids.SparkShimServiceProvider { + + override def getShimVersion: ShimVersion = SparkShimServiceProvider.VERSION + + def matchesVersion(version: String): Boolean = { + val shimEnabledProp = "spark.rapids.shims.spark400db173" + ".enabled" + val shimEnabled = Option(SparkEnv.get) + .flatMap(_.conf.getOption(shimEnabledProp).map(_.toBoolean)) + .getOrElse(true) + + DatabricksShimServiceProvider.matchesVersion( + dbrVersion = "17.3.x", + shimMatchEnabled = shimEnabled, + disclaimer = "Development of support for Databricks 17.3.x is still in progress: " + + "https://github.com/NVIDIA/spark-rapids/issues/14015" + ) + } +} diff --git a/sql-plugin/src/main/spark400db173/scala/org/apache/spark/rapids/shims/GpuShuffleExchangeExec.scala b/sql-plugin/src/main/spark400db173/scala/org/apache/spark/rapids/shims/GpuShuffleExchangeExec.scala new file mode 100644 index 00000000000..9c91cd3a5ed --- /dev/null +++ b/sql-plugin/src/main/spark400db173/scala/org/apache/spark/rapids/shims/GpuShuffleExchangeExec.scala @@ -0,0 +1,88 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/*** spark-rapids-shim-json-lines +{"spark": "400db173"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.rapids.shims + +import com.nvidia.spark.rapids.{GpuMetric, GpuPartitioning} + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.catalyst.trees.TreeNodeTag +import org.apache.spark.sql.execution.{ShufflePartitionSpec, SparkPlan} +import org.apache.spark.sql.execution.adaptive.AdaptiveRepartitioningStatus +import org.apache.spark.sql.execution.exchange.{ShuffleExchangeLike, ShuffleOrigin} +import org.apache.spark.sql.execution.metric.SQLShuffleWriteMetricsReporter +import org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBase.createAdditionalExchangeMetrics +import org.apache.spark.sql.rapids.execution.ShuffledBatchRDD + +case class GpuShuffleExchangeExec( + gpuOutputPartitioning: GpuPartitioning, + child: SparkPlan, + shuffleOrigin: ShuffleOrigin, + adaptiveRepartitioningStatus: AdaptiveRepartitioningStatus = + AdaptiveRepartitioningStatus.DEFAULT_STATUS)( + cpuOutputPartitioning: Partitioning) + extends GpuDatabricksShuffleExchangeExecBase(gpuOutputPartitioning, child, shuffleOrigin)( + cpuOutputPartitioning) { + + override lazy val additionalMetrics: Map[String, GpuMetric] = { + createAdditionalExchangeMetrics(this) ++ + GpuMetric.wrap(readMetrics) ++ + GpuMetric.wrap( + SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext)) ++ + // DBR 17.3 specific metrics from ShuffleExchangeLike's parent traits + GpuMetric.wrap(skewMetrics) ++ + GpuMetric.wrap(spillFallbackMetrics) ++ + GpuMetric.wrap(ensReqDPMetrics) ++ + GpuMetric.wrap(adpMetrics) ++ + GpuMetric.wrap(aosMetrics) + } + + // Databricks 17.3: Added stageShuffleCount parameter + override def getShuffleRDD( + partitionSpecs: Array[ShufflePartitionSpec], + lazyFetching: Boolean, + stageShuffleCount: Int): RDD[_] = { + new ShuffledBatchRDD(shuffleDependencyColumnar, metrics ++ readMetrics, partitionSpecs) + } + + // In Databricks ShuffleExchangeExec, targetOutputPartitioning is the first + // constructor parameter (the CPU partitioning). Our GPU version stores this as + // cpuOutputPartitioning. + override def targetOutputPartitioning: Partitioning = cpuOutputPartitioning + + override def withNewNumPartitions(numPartitions: Int): ShuffleExchangeLike = { + val newCpuPartitioning = cpuOutputPartitioning.withNewNumPartitions(numPartitions) + val newExec = copy(gpuOutputPartitioning, child, shuffleOrigin, + adaptiveRepartitioningStatus)(newCpuPartitioning) + newExec.copyTagsFrom(this) + newExec + } + + def repartition(numPartitions: Int, + updatedRepartitioningStatus: AdaptiveRepartitioningStatus): + ShuffleExchangeLike = { + val newCpuPartitioning = cpuOutputPartitioning.withNewNumPartitions(numPartitions) + copy(gpuOutputPartitioning, child, shuffleOrigin, + updatedRepartitioningStatus)(newCpuPartitioning) + } + + // not sure how it is used, so try to return one at first. + // For more details, refer to https://github.com/NVIDIA/spark-rapids/issues/13242. + override val ensReqDPMetricTag: TreeNodeTag[Int] = TreeNodeTag[Int]("GpuShuffleExchangeExec") +} diff --git a/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ShimParquetColumnVector.scala b/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ShimParquetColumnVector.scala new file mode 100644 index 00000000000..11c5f7a2193 --- /dev/null +++ b/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ShimParquetColumnVector.scala @@ -0,0 +1,37 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400db173"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.execution.datasources.parquet + +import org.apache.spark.memory.MemoryMode +import org.apache.spark.sql.execution.vectorized.WritableColumnVector + +object ShimParquetColumnVector { + def apply( + column: ParquetColumn, + vector: WritableColumnVector, + capacity: Int, + memoryMode: MemoryMode, + missingColumns: java.util.Set[ParquetColumn], + isTopLevel: Boolean, + defaultValue: Any): ParquetColumnVector = { + new ParquetColumnVector(column, vector, capacity, missingColumns, isTopLevel, + defaultValue, "") + } +} diff --git a/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ShimVectorizedColumnReader.scala b/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ShimVectorizedColumnReader.scala new file mode 100644 index 00000000000..f1f55b764b5 --- /dev/null +++ b/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ShimVectorizedColumnReader.scala @@ -0,0 +1,90 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400db173"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.execution.datasources.parquet.rapids.shims + +import java.time.ZoneId +import java.util.TimeZone + +import org.apache.parquet.VersionParser.ParsedVersion +import org.apache.parquet.column.ColumnDescriptor +import org.apache.parquet.column.page.PageReadStore +import org.apache.parquet.schema.{GroupType, Type} + +import org.apache.spark.sql.catalyst.util.RebaseDateTime.RebaseSpec +import org.apache.spark.sql.execution.datasources.parquet.{ParentContainerUpdater, + ParquetIdExternalMapping, ParquetRowConverter, ParquetToSparkSchemaConverter, + VectorizedColumnReader} +import org.apache.spark.sql.internal.LegacyBehaviorPolicy +import org.apache.spark.sql.types.StructType + +/** + * Identity mapping implementation for ParquetIdExternalMapping in Databricks 17.3. + * Based on the interface methods: getRootId(), getChild(name), getFieldId(name) + */ +object IdentityParquetIdMapping extends ParquetIdExternalMapping { + override def getRootId: Option[Int] = None + override def getChild(name: String): ParquetIdExternalMapping = this + override def getFieldId(name: String): Option[Int] = None +} + +/** + * Databricks 17.3 version where ParquetRowConverter requires + * 8 parameters including externalIdMapping. + */ +class ShimParquetRowConverter( + schemaConverter: ParquetToSparkSchemaConverter, + parquetType: GroupType, + catalystType: StructType, + convertTz: Option[ZoneId], + datetimeRebaseMode: String, + int96RebaseMode: String, + int96CDPHive3Compatibility: Boolean, + updater: ParentContainerUpdater +) extends ParquetRowConverter( + schemaConverter, + parquetType, + catalystType, + convertTz, + RebaseSpec(LegacyBehaviorPolicy.withName(datetimeRebaseMode)), + RebaseSpec(LegacyBehaviorPolicy.withName(int96RebaseMode)), + updater, + IdentityParquetIdMapping) + +class ShimVectorizedColumnReader( + index: Int, + columns: java.util.List[ColumnDescriptor], + types: java.util.List[Type], + pageReadStore: PageReadStore, + convertTz: ZoneId, + datetimeRebaseMode: String, + int96RebaseMode: String, + int96CDPHive3Compatibility: Boolean, + writerVersion: ParsedVersion +) extends VectorizedColumnReader( + columns.get(index), + true, + false, + pageReadStore, + convertTz, + datetimeRebaseMode, + TimeZone.getDefault.getID, + int96RebaseMode, + TimeZone.getDefault.getID, + writerVersion) diff --git a/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala b/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala new file mode 100644 index 00000000000..8e993d2c004 --- /dev/null +++ b/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala @@ -0,0 +1,39 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400db173"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.execution.rapids.shims + +import org.apache.spark.sql.execution.PartitionedFileUtil +import org.apache.spark.sql.execution.datasources._ + +object FilePartitionShims extends SplitFiles { + def getPartitions(selectedPartitions: Array[PartitionDirectory]): Array[PartitionedFile] = { + selectedPartitions.flatMap { p => + p.files.map { f => + PartitionedFileUtil.getPartitionedFile(f, f.getPath, p.values, 0, f.getLen) + } + } + } + + def getFiles(p: FilePartition): Array[PartitionedFile] = p.innerFiles + + def copyWithFiles(p: FilePartition, newFiles: Array[PartitionedFile]): FilePartition = { + p.copy(innerFiles = newFiles) + } +} diff --git a/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/hive/rapids/shims/GpuRowBasedHiveGenericUDFShim.scala b/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/hive/rapids/shims/GpuRowBasedHiveGenericUDFShim.scala new file mode 100644 index 00000000000..38d2f5ea8de --- /dev/null +++ b/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/hive/rapids/shims/GpuRowBasedHiveGenericUDFShim.scala @@ -0,0 +1,35 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400db173"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.hive.rapids.shims + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.SpecializedGetters +import org.apache.spark.sql.hive.DeferredObjectAdapter + +object GpuRowBasedHiveGenericUDFShim { + def setDeferredObject( + o: DeferredObjectAdapter, + childRowAccessors: Array[SpecializedGetters => Any], + idx: Int, + childrenRow: InternalRow): Unit = { + o.set(() => childRowAccessors(idx)(childrenRow)) + } +} + diff --git a/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/rapids/RapidsShuffleReaderShim.scala b/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/rapids/RapidsShuffleReaderShim.scala new file mode 100644 index 00000000000..a708d19d56d --- /dev/null +++ b/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/rapids/RapidsShuffleReaderShim.scala @@ -0,0 +1,42 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400db173"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.rapids + +import org.apache.spark.TaskContext +import org.apache.spark.shuffle.{ShuffleHandle, ShuffleReader, ShuffleReadMetricsReporter} + +trait RapidsShuffleReaderShim { + self: RapidsShuffleInternalManagerBase => + + // 8-parameter version for Databricks 17.3 (added prismMapStatusEnabled) + override def getReader[K, C]( + handle: ShuffleHandle, + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int, + context: TaskContext, + metrics: ShuffleReadMetricsReporter, + prismMapStatusEnabled: Boolean): ShuffleReader[K, C] = { + // Ignore prismMapStatusEnabled and call common implementation + getReaderImpl(handle, startMapIndex, endMapIndex, + startPartition, endPartition, context, metrics) + } +} diff --git a/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/rapids/ShuffleManagerShims.scala b/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/rapids/ShuffleManagerShims.scala new file mode 100644 index 00000000000..2f4f8b5feda --- /dev/null +++ b/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/rapids/ShuffleManagerShims.scala @@ -0,0 +1,38 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400db173"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.rapids + +import org.apache.spark.TaskContext +import org.apache.spark.shuffle.{ShuffleHandle, ShuffleManager, ShuffleReader, ShuffleReadMetricsReporter} + +object ShuffleManagerShims { + def getReader[K, C]( + manager: ShuffleManager, + handle: ShuffleHandle, + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int, + context: TaskContext, + metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { + manager.getReader(handle, startMapIndex, endMapIndex, startPartition, + endPartition, context, metrics, false) + } +} diff --git a/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/rapids/execution/ShuffleExchangeShim.scala b/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/rapids/execution/ShuffleExchangeShim.scala new file mode 100644 index 00000000000..4b24549f5d1 --- /dev/null +++ b/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/rapids/execution/ShuffleExchangeShim.scala @@ -0,0 +1,39 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400db173"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.rapids.execution + +import org.apache.spark.rapids.shims.GpuShuffleExchangeExec +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.execution.CoalescedPartitionSpec +import org.apache.spark.sql.vectorized.ColumnarBatch + +/** + * Databricks 17.3 version where getShuffleRDD requires stageShuffleCount parameter. + */ +object ShuffleExchangeShim { + def getShuffleRDD( + shuffleExchange: GpuShuffleExchangeExec, + partitionSpecs: Seq[CoalescedPartitionSpec]): RDD[ColumnarBatch] = { + // Databricks 17.3: Pass 0 as default for stageShuffleCount (not used) + shuffleExchange.getShuffleRDD(partitionSpecs.toArray, + lazyFetching = true, stageShuffleCount = 0) + .asInstanceOf[RDD[ColumnarBatch]] + } +} diff --git a/sql-plugin/src/main/spark411/scala/org/apache/spark/sql/rapids/execution/python/shims/WindowInPandasExecTypeShim.scala b/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/rapids/execution/python/shims/WindowInPandasExecTypeShim.scala similarity index 97% rename from sql-plugin/src/main/spark411/scala/org/apache/spark/sql/rapids/execution/python/shims/WindowInPandasExecTypeShim.scala rename to sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/rapids/execution/python/shims/WindowInPandasExecTypeShim.scala index a8c7d28b630..8c5ed60a4c0 100644 --- a/sql-plugin/src/main/spark411/scala/org/apache/spark/sql/rapids/execution/python/shims/WindowInPandasExecTypeShim.scala +++ b/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/rapids/execution/python/shims/WindowInPandasExecTypeShim.scala @@ -15,6 +15,7 @@ */ /*** spark-rapids-shim-json-lines +{"spark": "400db173"} {"spark": "411"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution.python.shims diff --git a/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/rapids/execution/python/shims/WritePythonUDFUtils.scala b/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/rapids/execution/python/shims/WritePythonUDFUtils.scala new file mode 100644 index 00000000000..559fc537a2d --- /dev/null +++ b/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/rapids/execution/python/shims/WritePythonUDFUtils.scala @@ -0,0 +1,53 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400db173"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.rapids.execution.python.shims + +import java.io.DataOutputStream + +import org.apache.spark.api.python.ChainedPythonFunctions +import org.apache.spark.sql.execution.python.EvalPythonExec.ArgumentMetadata +import org.apache.spark.sql.execution.python.PythonUDFRunner + +object WritePythonUDFUtils { + def writeUDFs( + dataOut: DataOutputStream, + funcs: Seq[(ChainedPythonFunctions, Long)], + argOffsets: Array[Array[Int]], + argNames: Option[Array[Array[Option[String]]]] = None, + profiler: Option[String] = None): Unit = { + // spark.sql.pyspark.udf.logging.maxEntries (default 0 = no logs collected) + // spark.sql.pyspark.udf.logging.logLevel (default "WARNING") + val udfLogMaxEntries = 0 + val udfLogLevel = "WARNING" + + if (argNames.isDefined) { + val argMetas = argOffsets.zip(argNames.get).map { case (idxs, names) => + idxs.zip(names).map { case (idx, name) => + ArgumentMetadata(idx, name) + } + } + PythonUDFRunner.writeUDFs(dataOut, funcs, argMetas, profiler, + udfLogMaxEntries, udfLogLevel) + } else { + PythonUDFRunner.writeUDFs(dataOut, funcs, argOffsets, profiler, + udfLogMaxEntries, udfLogLevel) + } + } +} diff --git a/sql-plugin/src/main/spark411/scala/org/apache/spark/sql/rapids/shims/FileStreamSinkShims.scala b/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/rapids/shims/FileStreamSinkShims.scala similarity index 98% rename from sql-plugin/src/main/spark411/scala/org/apache/spark/sql/rapids/shims/FileStreamSinkShims.scala rename to sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/rapids/shims/FileStreamSinkShims.scala index cd98593fe81..268db40678d 100644 --- a/sql-plugin/src/main/spark411/scala/org/apache/spark/sql/rapids/shims/FileStreamSinkShims.scala +++ b/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/rapids/shims/FileStreamSinkShims.scala @@ -15,6 +15,7 @@ */ /*** spark-rapids-shim-json-lines +{"spark": "400db173"} {"spark": "411"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark411/scala/org/apache/spark/sql/rapids/shims/datetimeExpressions.scala b/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/rapids/shims/datetimeExpressions.scala similarity index 99% rename from sql-plugin/src/main/spark411/scala/org/apache/spark/sql/rapids/shims/datetimeExpressions.scala rename to sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/rapids/shims/datetimeExpressions.scala index 7b82072b5db..25ed0a853ca 100644 --- a/sql-plugin/src/main/spark411/scala/org/apache/spark/sql/rapids/shims/datetimeExpressions.scala +++ b/sql-plugin/src/main/spark400db173/scala/org/apache/spark/sql/rapids/shims/datetimeExpressions.scala @@ -15,6 +15,7 @@ */ /*** spark-rapids-shim-json-lines +{"spark": "400db173"} {"spark": "411"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark400db173/scala/org/apache/spark/storage/ShuffleClientShims.scala b/sql-plugin/src/main/spark400db173/scala/org/apache/spark/storage/ShuffleClientShims.scala new file mode 100644 index 00000000000..aedfb30f7e9 --- /dev/null +++ b/sql-plugin/src/main/spark400db173/scala/org/apache/spark/storage/ShuffleClientShims.scala @@ -0,0 +1,37 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400db173"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.storage + +import org.apache.spark.network.shuffle.BlockStoreClient +import org.apache.spark.network.shuffle.checksum.Cause + +object ShuffleClientShims { + def diagnoseCorruption( + client: BlockStoreClient, + host: String, + port: Int, + execId: String, + blockId: BlockId, + checksum: Long, + algorithm: String): Cause = { + client.diagnoseCorruption(host, port, execId, blockId.name, checksum, algorithm) + } +} + diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/MetricsEventLogValidationSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/MetricsEventLogValidationSuite.scala index c84b2e2dfd9..b4cfcb37940 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/MetricsEventLogValidationSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/MetricsEventLogValidationSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2025, NVIDIA CORPORATION. + * Copyright (c) 2025-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -13,7 +13,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package com.nvidia.spark.rapids import java.io.{BufferedReader, File, InputStreamReader} @@ -21,8 +20,8 @@ import java.io.{BufferedReader, File, InputStreamReader} import scala.collection.mutable import scala.io.Source +import com.nvidia.spark.rapids.shims.EventLogJsonShims import org.json4s._ -import org.json4s.jackson.JsonMethods._ import org.scalatest.BeforeAndAfterEach import org.scalatest.funsuite.AnyFunSuite @@ -190,7 +189,7 @@ class MetricsEventLogValidationSuite extends AnyFunSuite with BeforeAndAfterEach lines.foreach { line => try { - val json = parse(line) + val json = EventLogJsonShims.parseJson(line) val eventType = (json \ "Event").extractOpt[String] eventType match { @@ -199,11 +198,12 @@ class MetricsEventLogValidationSuite extends AnyFunSuite with BeforeAndAfterEach val taskInfo = (json \ "Task Info") // Extract task execution time from Task Metrics - val taskId = (taskInfo \ "Task ID").extractOpt[Long] + val taskId = EventLogJsonShims.extractLong(taskInfo \ "Task ID") val taskMetrics = (json \ "Task Metrics") // https://github.com/apache/spark/blob/450b415028c3b00f3a002126cd11318d3932e28f/ // core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala#L151 - val executorRunTime = (taskMetrics \ "Executor Run Time").extractOpt[Long] + val executorRunTime = + EventLogJsonShims.extractLong(taskMetrics \ "Executor Run Time") (taskId, executorRunTime) match { case (Some(tId), Some(runTime)) => diff --git a/tests/src/test/scala/org/apache/spark/sql/rapids/metrics/source/MockTaskContext.scala b/tests/src/test/scala/org/apache/spark/sql/rapids/metrics/source/MockTaskContext.scala index 6f535b04357..c9b724d254b 100644 --- a/tests/src/test/scala/org/apache/spark/sql/rapids/metrics/source/MockTaskContext.scala +++ b/tests/src/test/scala/org/apache/spark/sql/rapids/metrics/source/MockTaskContext.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2025, NVIDIA CORPORATION. + * Copyright (c) 2020-2026, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -30,7 +30,7 @@ import org.apache.spark.scheduler.TaskLocality import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.util.{AccumulatorV2, TaskCompletionListener, TaskFailureListener} -class MockTaskContext(taskAttemptId: Long, partitionId: Int) extends TaskContext { +abstract class MockTaskContextBase(taskAttemptId: Long, partitionId: Int) extends TaskContext { val listeners = new ListBuffer[TaskCompletionListener] @@ -115,4 +115,12 @@ class MockTaskContext(taskAttemptId: Long, partitionId: Int) extends TaskContext private[spark] def createResourceUninterruptibly[T <: Closeable]( resourceBuilder: => T): T = resourceBuilder + + /** + * These below methods were introduced in Spark-4.1 / Databricks 17.3. Not shimmed and added + * to the common class by removing the override keyword. + * Note: addTaskInterruptListener is omitted because TaskInterruptListener type does not exist + * in Spark <= 4.0. + */ + private[spark] def getTaskFailure(): Option[Throwable] = None } diff --git a/tests/src/test/spark321/scala/com/nvidia/spark/rapids/shims/EventLogJsonShims.scala b/tests/src/test/spark321/scala/com/nvidia/spark/rapids/shims/EventLogJsonShims.scala new file mode 100644 index 00000000000..fbfa7bf814f --- /dev/null +++ b/tests/src/test/spark321/scala/com/nvidia/spark/rapids/shims/EventLogJsonShims.scala @@ -0,0 +1,58 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "321"} +{"spark": "330"} +{"spark": "330db"} +{"spark": "331"} +{"spark": "332"} +{"spark": "332db"} +{"spark": "333"} +{"spark": "334"} +{"spark": "340"} +{"spark": "341"} +{"spark": "341db"} +{"spark": "342"} +{"spark": "343"} +{"spark": "344"} +{"spark": "350"} +{"spark": "350db143"} +{"spark": "351"} +{"spark": "352"} +{"spark": "353"} +{"spark": "354"} +{"spark": "355"} +{"spark": "356"} +{"spark": "357"} +{"spark": "358"} +{"spark": "400"} +{"spark": "401"} +{"spark": "402"} +{"spark": "411"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import org.json4s._ +import org.json4s.jackson.JsonMethods._ + +object EventLogJsonShims { + private implicit val formats: Formats = org.json4s.DefaultFormats + + def parseJson(line: String): JValue = parse(line) + + def extractLong(jval: JValue): Option[Long] = jval.extractOpt[Long] +} diff --git a/tests/src/test/spark321/scala/com/nvidia/spark/rapids/shims/OrcStatisticShim.scala b/tests/src/test/spark321/scala/com/nvidia/spark/rapids/shims/OrcStatisticShim.scala index 20090f1ccb7..804877fa973 100644 --- a/tests/src/test/spark321/scala/com/nvidia/spark/rapids/shims/OrcStatisticShim.scala +++ b/tests/src/test/spark321/scala/com/nvidia/spark/rapids/shims/OrcStatisticShim.scala @@ -40,6 +40,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/tests/src/test/spark321/scala/org/apache/spark/sql/rapids/metrics/source/MockTaskContext.scala b/tests/src/test/spark321/scala/org/apache/spark/sql/rapids/metrics/source/MockTaskContext.scala new file mode 100644 index 00000000000..ad80436ef60 --- /dev/null +++ b/tests/src/test/spark321/scala/org/apache/spark/sql/rapids/metrics/source/MockTaskContext.scala @@ -0,0 +1,51 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "321"} +{"spark": "330"} +{"spark": "330db"} +{"spark": "331"} +{"spark": "332"} +{"spark": "332db"} +{"spark": "333"} +{"spark": "334"} +{"spark": "340"} +{"spark": "341"} +{"spark": "341db"} +{"spark": "342"} +{"spark": "343"} +{"spark": "344"} +{"spark": "350"} +{"spark": "350db143"} +{"spark": "351"} +{"spark": "352"} +{"spark": "353"} +{"spark": "354"} +{"spark": "355"} +{"spark": "356"} +{"spark": "357"} +{"spark": "358"} +{"spark": "400"} +{"spark": "401"} +{"spark": "402"} +{"spark": "411"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.rapids.metrics.source + +class MockTaskContext(taskAttemptId: Long, partitionId: Int) + extends MockTaskContextBase(taskAttemptId, partitionId) { +} diff --git a/tests/src/test/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTestHelper.scala b/tests/src/test/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTestHelper.scala index 0d17c884cf6..66c0f4925de 100644 --- a/tests/src/test/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTestHelper.scala +++ b/tests/src/test/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTestHelper.scala @@ -32,6 +32,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/tests/src/test/spark341db/scala/com/nvidia/spark/rapids/ToPrettyStringSuite.scala b/tests/src/test/spark341db/scala/com/nvidia/spark/rapids/ToPrettyStringSuite.scala index 3b46b03b18f..b2c11310ff2 100644 --- a/tests/src/test/spark341db/scala/com/nvidia/spark/rapids/ToPrettyStringSuite.scala +++ b/tests/src/test/spark341db/scala/com/nvidia/spark/rapids/ToPrettyStringSuite.scala @@ -27,6 +27,7 @@ {"spark": "357"} {"spark": "358"} {"spark": "400"} +{"spark": "400db173"} {"spark": "401"} {"spark": "402"} {"spark": "411"} diff --git a/tests/src/test/spark400db173/scala/com/nvidia/spark/rapids/shims/EventLogJsonShims.scala b/tests/src/test/spark400db173/scala/com/nvidia/spark/rapids/shims/EventLogJsonShims.scala new file mode 100644 index 00000000000..5d2dcee4a65 --- /dev/null +++ b/tests/src/test/spark400db173/scala/com/nvidia/spark/rapids/shims/EventLogJsonShims.scala @@ -0,0 +1,32 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400db173"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import org.json4s._ +import org.json4s.jackson.Serialization + +object EventLogJsonShims { + private implicit val formats: Formats = org.json4s.DefaultFormats + + def parseJson(line: String): JValue = Serialization.read[JValue](line) + + def extractLong(jval: JValue): Option[Long] = + jval.extractOpt[BigInt].map(_.toLong) +} diff --git a/tests/src/test/spark400db173/scala/org/apache/spark/sql/rapids/metrics/source/MockTaskContext.scala b/tests/src/test/spark400db173/scala/org/apache/spark/sql/rapids/metrics/source/MockTaskContext.scala new file mode 100644 index 00000000000..2f8edbc5630 --- /dev/null +++ b/tests/src/test/spark400db173/scala/org/apache/spark/sql/rapids/metrics/source/MockTaskContext.scala @@ -0,0 +1,30 @@ +/* + * Copyright (c) 2026, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400db173"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.rapids.metrics.source + +import org.apache.spark.TaskContext +import org.apache.spark.util.TaskInterruptListener + +class MockTaskContext(taskAttemptId: Long, partitionId: Int) + extends MockTaskContextBase(taskAttemptId, partitionId) { + + override def addTaskInterruptListener( + listener: TaskInterruptListener): TaskContext = this +}