diff --git a/.github/workflows/build_branch35.yml b/.github/workflows/build_branch35.yml new file mode 100644 index 0000000000000..9509706a28805 --- /dev/null +++ b/.github/workflows/build_branch35.yml @@ -0,0 +1,48 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +name: "Build (branch-3.5, Scala 2.13, Hadoop 3, JDK 8)" + +on: + pull_request: + +jobs: + run-build: + permissions: + packages: write + name: Run + uses: ./.github/workflows/build_and_test.yml + if: github.repository == 'logicalclocks/spark' + with: + java: 8 + branch: branch-3.5 + hadoop: hadoop3 + envs: >- + { + "SCALA_PROFILE": "scala2.13" + } + jobs: >- + { + "build": "true", + "pyspark": "true", + "sparkr": "true", + "tpcds-1g": "true", + "docker-integration-tests": "true", + "lint" : "true" + } diff --git a/.github/workflows/build_main.yml b/.github/workflows/build_main.yml index 9ef52f326375b..6a8d60117414e 100644 --- a/.github/workflows/build_main.yml +++ b/.github/workflows/build_main.yml @@ -23,6 +23,7 @@ on: push: branches: - '**' + pull_request: jobs: call-build-and-test: diff --git a/.github/workflows/build_spark_with_hopsfs.yaml b/.github/workflows/build_spark_with_hopsfs.yaml new file mode 100644 index 0000000000000..434162a796428 --- /dev/null +++ b/.github/workflows/build_spark_with_hopsfs.yaml @@ -0,0 +1,204 @@ +name: Build Spark with hopsfs + +on: + workflow_call: + inputs: + ref: + description: 'The ref to checkout for the spark repo, default is branch-3.5' + required: false + type: string + default: 'branch-3.5' + jira_tag: + description: 'The tag to use for the jira release, default is the version from version.txt' + required: false + type: string + default: 'NOJIRA' + runner: + description: 'The type of runner to use, default is ghrunner-ee8' + required: false + type: string + default: 'ghrunner-ee8' + build: + description: 'Whether to build spark or not, default is false. If this is false then the workflow will only prepare the versioning related outputs.' + required: false + type: boolean + default: true + secrets: + NEXUS_HARBOR_PASSWORD: + required: true + outputs: + pom_version_no_jira: + value: ${{ jobs.build-spark.outputs.pom_version_no_jira }} + description: 'The pom version without the jira tag' + pom_version: + value: ${{ jobs.build-spark.outputs.pom_version }} + description: 'The pom version with the jira tag' + commit_hash: + value: ${{ jobs.build-spark.outputs.commit_hash }} + description: 'The commit hash of the spark repo' + jira_tag: + value: ${{ jobs.build-spark.outputs.jira_tag }} + description: 'The jira tag used for the build' + spark_tar_name: + value: ${{ jobs.build-spark.outputs.spark_tar_name }} + description: 'The name of the spark tar file' + spark_tar_url: + value: ${{ jobs.build-spark.outputs.spark_tar_url }} + description: 'The url of the spark tar file' + hopsfs_version: + value: ${{ jobs.build-spark.outputs.hopsfs_version }} + description: 'The version of hopsfs used in the build' + workflow_dispatch: + inputs: + ref: + description: 'The ref to checkout for the spark repo, default is branch-3.5' + required: false + type: string + default: 'branch-3.5' + jira_tag: + description: 'The tag to use for the jira release, default is the version from version.txt' + required: false + type: string + default: 'NOJIRA' + runner: + description: 'The type of runner to use, default is ghrunner-ee8' + required: false + type: string + default: 'ghrunner-ee8' + build: + description: 'Whether to build spark or not, default is false. If this is false then the workflow will only prepare the versioning related outputs.' + required: false + type: boolean + default: true + pull_request: + +concurrency: + group: build-spark-${{ github.workflow }}-${{ github.job }}-${{ inputs.jira_tag || 'NOJIRA' }} + cancel-in-progress: true + +# Used to avoid error on PRs +env: + # SPARK_REF: ${{ inputs.ref || 'branch-3.5' }} + SPARK_REF: ${{ inputs.ref || 'HWORKS-2203-vatj' }} + JIRA_TAG: ${{ inputs.jira_tag || 'NOJIRA' }} + +jobs: + build-spark: + runs-on: ${{ inputs.runner }} + outputs: + pom_version_no_jira: ${{ steps.prep_version.outputs.pom_version_no_jira }} + pom_version: ${{ steps.prep_version.outputs.pom_version }} + commit_hash: ${{ steps.prep_version.outputs.commit_hash }} + jira_tag: ${{ env.JIRA_TAG }} + spark_tar_name: ${{ steps.prep_version.outputs.spark_tar_name }} + spark_tar_url: ${{ steps.prep_version.outputs.spark_tar_url }} + hopsfs_version: ${{ steps.prep_version.outputs.hopsfs_version }} + steps: + - name: Checkout spark repo + uses: actions/checkout@v4 + with: + repository: logicalclocks/spark + ref: ${{ env.SPARK_REF }} + path: ${{ github.workspace }}/spark + + - name: To build or not to build + id: to_build_or_not_to_build + shell: bash + env: + BUILD_SPARK: ${{ (github.event_name == 'pull_request' && contains(join(github.event.pull_request.labels.*.name, ','), 'build-spark')) || inputs.build }} + run: | + if [[ "${{ env.BUILD_SPARK }}" != "true" ]]; then + echo "# :recycle: Not building Spark" >> $GITHUB_STEP_SUMMARY + if [[ "${{ github.event_name }}" == "pull_request" ]]; then + echo "This is a pull request and the 'build-spark' label is not present." >> $GITHUB_STEP_SUMMARY + echo "pull_request_labels=${{ join(github.event.pull_request.labels.*.name, ', ') }}" >> $GITHUB_STEP_SUMMARY + elif [[ "${{ inputs.build || 'false'}}" != "true" ]]; then + echo "The input 'build' is set to false." >> $GITHUB_STEP_SUMMARY + fi + echo "BUILD_SPARK=$BUILD_SPARK" >> $GITHUB_OUTPUT + else + echo "# :white_check_mark: Building Spark" >> $GITHUB_STEP_SUMMARY + echo "BUILD_SPARK=$BUILD_SPARK" >> $GITHUB_OUTPUT + fi + + - name: Prep step version + shell: bash + id: prep_version + working-directory: ${{ github.workspace }}/spark + run: | + COMMIT_HASH=$(git rev-parse --short HEAD) + POM_VERSION_NO_JIRA=$(mvn -q -Dexec.executable="echo" -Dexec.args='${project.version}' --non-recursive exec:exec) + find . -name "pom.xml" -exec sed -i "s|${POM_VERSION_NO_JIRA}|${POM_VERSION_NO_JIRA%-SNAPSHOT}-${JIRA_TAG}-SNAPSHOT|g" {} \; + POM_VERSION=$(mvn -q -Dexec.executable="echo" -Dexec.args='${project.version}' --non-recursive exec:exec) + SPARK_TAR_NAME=spark-${POM_VERSION}-bin-without-hadoop-with-hive.tgz + SPARK_TAR_URL="${{ vars.NEXUS_DEV_SPARK_URL }}/${JIRA_TAG}/${SPARK_TAR_NAME}" + HOPSFS_VERSION=$(mvn -q -Dexec.executable="echo" -Dexec.args='${hadoop.version}' --non-recursive exec:exec) + + echo "POM_VERSION_NO_JIRA=${POM_VERSION_NO_JIRA}" >> $GITHUB_ENV + echo "POM_VERSION=${POM_VERSION}" >> $GITHUB_ENV + echo "COMMIT_HASH=$COMMIT_HASH" >> $GITHUB_ENV + echo "SPARK_TAR_NAME=${SPARK_TAR_NAME}" >> $GITHUB_ENV + echo "SPARK_TAR_URL=${SPARK_TAR_URL}" >> $GITHUB_ENV + echo "HOPSFS_VERSION=${HOPSFS_VERSION}" >> $GITHUB_ENV + + echo "POM_VERSION_NO_JIRA=${POM_VERSION_NO_JIRA}" >> $GITHUB_STEP_SUMMARY + echo "POM_VERSION=${POM_VERSION}" >> $GITHUB_STEP_SUMMARY + echo "COMMIT_HASH=$COMMIT_HASH" >> $GITHUB_STEP_SUMMARY + echo "SPARK_TAR_NAME=${SPARK_TAR_NAME}" >> $GITHUB_STEP_SUMMARY + echo "SPARK_TAR_URL=${SPARK_TAR_URL}" >> $GITHUB_STEP_SUMMARY + echo "HOPSFS_VERSION=${HOPSFS_VERSION}" >> $GITHUB_STEP_SUMMARY + + echo "POM_VERSION=${POM_VERSION}" >> $GITHUB_OUTPUT + echo "POM_VERSION_NO_JIRA=${POM_VERSION_NO_JIRA}" >> $GITHUB_OUTPUT + echo "COMMIT_HASH=$COMMIT_HASH" >> $GITHUB_OUTPUT + echo "SPARK_TAR_NAME=${SPARK_TAR_NAME}" >> $GITHUB_OUTPUT + echo "SPARK_TAR_URL=${SPARK_TAR_URL}" >> $GITHUB_OUTPUT + echo "HOPSFS_VERSION=${HOPSFS_VERSION}" >> $GITHUB_OUTPUT + + - name: Set up .m2 settings.xml + shell: bash + if: steps.to_build_or_not_to_build.outputs.BUILD_SPARK == 'true' + env: + M2_HOME: ~/.m2 + run: | + echo "M2_HOME var is $M2_HOME" >> $GITHUB_STEP_SUMMARY + mkdir -p ~/.m2 + echo "" > ~/.m2/settings.xml + echo "HopsEE${{ vars.NEXUS_HARBOR_USER }}${{ secrets.NEXUS_HARBOR_PASSWORD }}" >> ~/.m2/settings.xml + echo "HiveEE${{ vars.NEXUS_HARBOR_USER }}${{ secrets.NEXUS_HARBOR_PASSWORD }}" >> ~/.m2/settings.xml + echo "" >> ~/.m2/settings.xml + + + - name: Cache maven + id: cache-maven + if: steps.to_build_or_not_to_build.outputs.BUILD_SPARK == 'true' + uses: actions/cache@v4 + with: + path: | + ~/.m2 + !~/.m2/settings.xml + key: ${{ runner.os }}-maven-spark-${{ hashFiles('spark/**/pom.xml') }} + restore-keys: | + ${{ runner.os }}-maven-spark- + + + - name: Build spark and spark-packaging + shell: bash + if: steps.to_build_or_not_to_build.outputs.BUILD_SPARK == 'true' + working-directory: ${{ github.workspace }}/spark + env: + POM_VERSION: ${{ env.POM_VERSION }} + M2_HOME: ~/.m2 + run: | + ./dev/make-distribution.sh --name without-hadoop-with-hive --tgz "-Pkubernetes,hadoop-provided,parquet-provided,hive,hadoop-cloud,bigtop-dist" + + - name: Upload spark-packaging artifact to Nexus + shell: bash + if: steps.to_build_or_not_to_build.outputs.BUILD_SPARK == 'true' + working-directory: ${{ github.workspace }}/spark + env: + M2_HOME: ~/.m2 + run: | + curl -u ${{ vars.NEXUS_HARBOR_USER }}:${{ secrets.NEXUS_HARBOR_PASSWORD }} --upload-file spark-$POM_VERSION-bin-without-hadoop-with-hive.tgz "${SPARK_TAR_URL}" + export MAVEN_OPTS="${MAVEN_OPTS:--Xss128m -Xmx4g -XX:ReservedCodeCacheSize=128m}" + ./build/mvn deploy -DskipTests -Dmaven.javadoc.skip=true -Dmaven.scaladoc.skip=true -Dmaven.source.skip -Dcyclonedx.skip=true -Pkubernetes,hadoop-provided,parquet-provided,hive,hadoop-cloud \ No newline at end of file diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index d8b7f76b262f3..53f40d803e1a0 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 3.5.8 +Version: 3.5.7 Title: R Front End for 'Apache Spark' Description: Provides an R Front end for 'Apache Spark' . Authors@R: diff --git a/assembly/pom.xml b/assembly/pom.xml index 7c4c5d84792bc..6eb1c9c341b81 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../pom.xml @@ -137,7 +137,7 @@ ${project.version} - org.apache.hadoop + ${hadoop.group} hadoop-yarn-server-web-proxy diff --git a/assembly/src/main/assembly/assembly.xml b/assembly/src/main/assembly/assembly.xml index 009d4b92f406c..7db5b432aaf50 100644 --- a/assembly/src/main/assembly/assembly.xml +++ b/assembly/src/main/assembly/assembly.xml @@ -83,6 +83,8 @@ false org.apache.hadoop:*:jar + io.hops:*:jar + io.hops.metadata:*:jar org.apache.spark:*:jar org.apache.zookeeper:*:jar org.apache.avro:*:jar diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index a543c74ca9003..a1ec2748329b9 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 60d6066b3cc0e..6ae7863161b1e 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index a8750506edcb2..7537e39d93ea5 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index ee1492783cd9b..f458b6c4e7e15 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../pom.xml @@ -64,12 +64,12 @@ - org.apache.hadoop + ${hadoop.group} hadoop-client-api ${hadoop.version} - org.apache.hadoop + ${hadoop.group} hadoop-client-runtime ${hadoop.version} diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 258b786ed308b..43313bd0ec28f 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index b237b1e272ecc..471b499c37297 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 4f2ffa892d523..4dac8356b77b6 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../pom.xml diff --git a/common/utils/pom.xml b/common/utils/pom.xml index 7c445789d8fac..2e4e0dcdaa2eb 100644 --- a/common/utils/pom.xml +++ b/common/utils/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../pom.xml diff --git a/connector/avro/pom.xml b/connector/avro/pom.xml index a6b310bdefa72..11811ed080bca 100644 --- a/connector/avro/pom.xml +++ b/connector/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../pom.xml diff --git a/connector/connect/client/jvm/pom.xml b/connector/connect/client/jvm/pom.xml index 1f309e2db75ab..f2630bfb9303f 100644 --- a/connector/connect/client/jvm/pom.xml +++ b/connector/connect/client/jvm/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../../../pom.xml diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 10d2af094a08c..f138ca93760cf 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -45,7 +45,7 @@ class DataFrameReader private[sql] (sparkSession: SparkSession) extends Logging * @since 3.4.0 */ def format(source: String): DataFrameReader = { - this.source = source + this.source = Some(source) this } @@ -179,8 +179,7 @@ class DataFrameReader private[sql] (sparkSession: SparkSession) extends Logging def load(paths: String*): DataFrame = { sparkSession.newDataFrame { builder => val dataSourceBuilder = builder.getReadBuilder.getDataSourceBuilder - assertSourceFormatSpecified() - dataSourceBuilder.setFormat(source) + source.foreach(dataSourceBuilder.setFormat) userSpecifiedSchema.foreach(schema => dataSourceBuilder.setSchema(schema.toDDL)) extraOptions.foreach { case (k, v) => dataSourceBuilder.putOptions(k, v) @@ -285,7 +284,7 @@ class DataFrameReader private[sql] (sparkSession: SparkSession) extends Logging sparkSession.newDataFrame { builder => val dataSourceBuilder = builder.getReadBuilder.getDataSourceBuilder format("jdbc") - dataSourceBuilder.setFormat(source) + source.foreach(dataSourceBuilder.setFormat) predicates.foreach(predicate => dataSourceBuilder.addPredicates(predicate)) this.extraOptions ++= Seq("url" -> url, "dbtable" -> table) val params = extraOptions ++ connectionProperties.asScala @@ -539,12 +538,6 @@ class DataFrameReader private[sql] (sparkSession: SparkSession) extends Logging text(paths: _*).select("value").as(StringEncoder) } - private def assertSourceFormatSpecified(): Unit = { - if (source == null) { - throw new IllegalArgumentException("The source format must be specified.") - } - } - private def parse(ds: Dataset[String], format: ParseFormat): DataFrame = { sparkSession.newDataFrame { builder => val parseBuilder = builder.getParseBuilder @@ -571,7 +564,7 @@ class DataFrameReader private[sql] (sparkSession: SparkSession) extends Logging // Builder pattern config options /////////////////////////////////////////////////////////////////////////////////////// - private var source: String = _ + private var source: Option[String] = None private var userSpecifiedSchema: Option[StructType] = None diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala index feefd19000d1d..d53a472723b71 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala @@ -1325,6 +1325,18 @@ class ClientE2ETestSuite extends RemoteSparkSession with SQLHelper with PrivateM .dropDuplicatesWithinWatermark("newcol") testAndVerify(result2) } + + test("SPARK-53054: DataFrameReader defaults to spark.sql.sources.default") { + withTempPath { file => + val path = file.getAbsoluteFile.toURI.toString + spark.range(100).write.parquet(file.toPath.toAbsolutePath.toString) + + spark.conf.set("spark.sql.sources.default", "parquet") + + val df = spark.read.load(path) + assert(df.count == 100) + } + } } private[sql] case class ClassData(a: String, b: Int) diff --git a/connector/connect/common/pom.xml b/connector/connect/common/pom.xml index f676a5eccbad5..a4f010f7076d4 100644 --- a/connector/connect/common/pom.xml +++ b/connector/connect/common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../../pom.xml diff --git a/connector/connect/server/pom.xml b/connector/connect/server/pom.xml index 97ff8140a3632..54c63a6f6ded8 100644 --- a/connector/connect/server/pom.xml +++ b/connector/connect/server/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../../pom.xml diff --git a/connector/docker-integration-tests/pom.xml b/connector/docker-integration-tests/pom.xml index 91b89665d4700..d35c4809b529f 100644 --- a/connector/docker-integration-tests/pom.xml +++ b/connector/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../pom.xml @@ -92,7 +92,7 @@ test - org.apache.hadoop + ${hadoop.group} hadoop-minikdc test diff --git a/connector/kafka-0-10-assembly/pom.xml b/connector/kafka-0-10-assembly/pom.xml index 73f5c23a9f5c7..4cd2f31a94645 100644 --- a/connector/kafka-0-10-assembly/pom.xml +++ b/connector/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../pom.xml @@ -70,13 +70,13 @@ provided - org.apache.hadoop + ${hadoop.group} hadoop-client-api ${hadoop.version} provided - org.apache.hadoop + ${hadoop.group} hadoop-client-runtime ${hadoop.version} provided diff --git a/connector/kafka-0-10-sql/pom.xml b/connector/kafka-0-10-sql/pom.xml index 89ce0a2ff5cef..ebb6f15ad3697 100644 --- a/connector/kafka-0-10-sql/pom.xml +++ b/connector/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../pom.xml @@ -121,7 +121,7 @@ - org.apache.hadoop + ${hadoop.group} hadoop-minikdc diff --git a/connector/kafka-0-10-token-provider/pom.xml b/connector/kafka-0-10-token-provider/pom.xml index a139af88905ac..cde5d2c4c1348 100644 --- a/connector/kafka-0-10-token-provider/pom.xml +++ b/connector/kafka-0-10-token-provider/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../pom.xml @@ -65,7 +65,7 @@ test - org.apache.hadoop + ${hadoop.group} hadoop-client-runtime ${hadoop.deps.scope} diff --git a/connector/kafka-0-10/pom.xml b/connector/kafka-0-10/pom.xml index 3d804b66248dd..b59e6401191be 100644 --- a/connector/kafka-0-10/pom.xml +++ b/connector/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../pom.xml diff --git a/connector/kinesis-asl-assembly/pom.xml b/connector/kinesis-asl-assembly/pom.xml index e6c3f0219f3e1..1b5ee194c268a 100644 --- a/connector/kinesis-asl-assembly/pom.xml +++ b/connector/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../pom.xml @@ -100,13 +100,13 @@ provided - org.apache.hadoop + ${hadoop.group} hadoop-client-api ${hadoop.version} provided - org.apache.hadoop + ${hadoop.group} hadoop-client-runtime ${hadoop.version} provided diff --git a/connector/kinesis-asl/pom.xml b/connector/kinesis-asl/pom.xml index 1bfc3384fa35c..608671f47a0c3 100644 --- a/connector/kinesis-asl/pom.xml +++ b/connector/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../pom.xml diff --git a/connector/protobuf/pom.xml b/connector/protobuf/pom.xml index de2e490c00341..91df2118e6092 100644 --- a/connector/protobuf/pom.xml +++ b/connector/protobuf/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../pom.xml diff --git a/connector/spark-ganglia-lgpl/pom.xml b/connector/spark-ganglia-lgpl/pom.xml index 2fd4109af2d4f..572766941ed93 100644 --- a/connector/spark-ganglia-lgpl/pom.xml +++ b/connector/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index f0771a62db3a3..29747b80a431e 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../pom.xml @@ -71,12 +71,12 @@ xbean-asm9-shaded - org.apache.hadoop + ${hadoop.group} hadoop-client-api ${hadoop.version} - org.apache.hadoop + ${hadoop.group} hadoop-client-runtime ${hadoop.version} @@ -426,7 +426,7 @@ test - org.apache.hadoop + ${hadoop.group} hadoop-minikdc test @@ -472,13 +472,13 @@ - org.apache.hadoop + ${hadoop.group} hadoop-aws ${hadoop.version} test - org.apache.hadoop + ${hadoop.group} hadoop-common diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 441bf60e4891e..c7e0deb4d61ed 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -205,7 +205,9 @@ private[spark] class ExecutorAllocationManager( s"s${DYN_ALLOCATION_SUSTAINED_SCHEDULER_BACKLOG_TIMEOUT.key} must be > 0!") } if (!conf.get(config.SHUFFLE_SERVICE_ENABLED) && !reliableShuffleStorage) { - if (conf.get(config.DYN_ALLOCATION_SHUFFLE_TRACKING_ENABLED)) { + if (conf.isRssEnable()) { + logInfo("Dynamic allocation will use remote shuffle service") + } else if (conf.get(config.DYN_ALLOCATION_SHUFFLE_TRACKING_ENABLED)) { logInfo("Dynamic allocation is enabled without a shuffle service.") } else if (decommissionEnabled && conf.get(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED)) { diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index f49e9e357c84d..e1927f3a6df3d 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -580,6 +580,13 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria Utils.redact(this, getAll).sorted.map { case (k, v) => k + "=" + v }.mkString("\n") } + /** + * Return true if remote shuffle service is enabled. + */ + def isRssEnable(): Boolean = { + val shuffleMgr = get("spark.shuffle.manager", "sort") + shuffleMgr.contains("RssShuffleManager") || shuffleMgr.contains("UniffleShuffleManager") + } } private[spark] object SparkConf extends Logging { diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index f3f7db6bb0aba..89f0d12935ce1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -99,6 +99,15 @@ private[history] abstract class ApplicationHistoryProvider { */ def getListing(): Iterator[ApplicationInfo] + /** + * Returns a list of applications available for the history server to show. + * + * @param max The maximum number of applications to return + * @param predicate A function that filters the applications to be returned + * @return An iterator of matching applications up to the specified maximum + */ + def getListing(max: Int)(predicate: ApplicationInfo => Boolean): Iterator[ApplicationInfo] + /** * Returns the Spark UI for a specific application. * diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 387bc7d9e45b3..9841b21861d42 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -300,6 +300,14 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) .index("endTime").reverse())(_.toApplicationInfo()).iterator } + override def getListing(max: Int)( + predicate: ApplicationInfo => Boolean): Iterator[ApplicationInfo] = { + // Return the filtered listing in end time descending order. + KVUtils.mapToSeqWithFilter( + listing.view(classOf[ApplicationInfoWrapper]).index("endTime").reverse(), + max)(_.toApplicationInfo())(predicate).iterator + } + override def getApplicationInfo(appId: String): Option[ApplicationInfo] = { try { Some(load(appId).toApplicationInfo()) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index f2cd5b7e240dd..dd4921207c961 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -91,7 +91,7 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") } def shouldDisplayApplications(requestedIncomplete: Boolean): Boolean = { - parent.getApplicationList().exists(isApplicationCompleted(_) != requestedIncomplete) + parent.getApplicationInfoList(1)(isApplicationCompleted(_) != requestedIncomplete).nonEmpty } private def makePageLink(request: HttpServletRequest, showIncomplete: Boolean): String = { diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index bea3f9ec84a4b..97425ba2339fd 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -220,6 +220,11 @@ class HistoryServer( getApplicationList() } + override def getApplicationInfoList(max: Int)( + filter: ApplicationInfo => Boolean): Iterator[ApplicationInfo] = { + provider.getListing(max)(filter) + } + def getApplicationInfo(appId: String): Option[ApplicationInfo] = { provider.getApplicationInfo(appId) } diff --git a/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala b/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala index 5448d7da6d6c1..bff8007bccf7d 100644 --- a/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala +++ b/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala @@ -176,7 +176,14 @@ private[spark] class ProcfsMetricsGetter(procfsDir: String = "/proc/") extends L } Utils.tryWithResource(openReader) { in => val procInfo = in.readLine - val procInfoSplit = procInfo.split(" ") + // The comm field, which is inside parentheses, could contain spaces. We should not split + // by those spaces as doing so could cause the numbers after it to be shifted. + val commStartIndex = procInfo.indexOf('(') + val commEndIndex = procInfo.lastIndexOf(')') + 1 + val pidArray = Array(procInfo.substring(0, commStartIndex).trim) + val commArray = Array(procInfo.substring(commStartIndex, commEndIndex)) + val splitAfterComm = procInfo.substring(commEndIndex).trim.split(" ") + val procInfoSplit = pidArray ++ commArray ++ splitAfterComm val vmem = procInfoSplit(22).toLong val rssMem = procInfoSplit(23).toLong * pageSize if (procInfoSplit(1).toLowerCase(Locale.US).contains("java")) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 5ae29a5cd0230..9cc4dbafbcc10 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -2628,7 +2628,8 @@ private[spark] class DAGScheduler( // we know to unregister shuffle output. (Note that "worker" specifically refers to the process // from a Standalone cluster, where the shuffle service lives in the Worker.) val fileLost = !sc.shuffleDriverComponents.supportsReliableStorage() && - (workerHost.isDefined || !env.blockManager.externalShuffleServiceEnabled) + (workerHost.isDefined || !env.blockManager.externalShuffleServiceEnabled) && + !sc.getConf.isRssEnable() removeExecutorAndUnregisterOutputs( execId = execId, fileLost = fileLost, diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index b52a0f2f999dd..6e41e41fab14a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -152,96 +152,96 @@ private[spark] class EventLoggingListener( } // log stage completed event - logEvent(event, flushLogger = true) + logEvent(event) } override def onJobStart(event: SparkListenerJobStart): Unit = { - logEvent(event.copy(properties = redactProperties(event.properties)), flushLogger = true) + logEvent(event.copy(properties = redactProperties(event.properties))) } override def onJobEnd(event: SparkListenerJobEnd): Unit = logEvent(event, flushLogger = true) override def onBlockManagerAdded(event: SparkListenerBlockManagerAdded): Unit = { - logEvent(event, flushLogger = true) + logEvent(event) } override def onBlockManagerRemoved(event: SparkListenerBlockManagerRemoved): Unit = { - logEvent(event, flushLogger = true) + logEvent(event) } override def onUnpersistRDD(event: SparkListenerUnpersistRDD): Unit = { - logEvent(event, flushLogger = true) + logEvent(event) } override def onApplicationStart(event: SparkListenerApplicationStart): Unit = { - logEvent(event, flushLogger = true) + logEvent(event) } override def onApplicationEnd(event: SparkListenerApplicationEnd): Unit = { logEvent(event, flushLogger = true) } override def onExecutorAdded(event: SparkListenerExecutorAdded): Unit = { - logEvent(event, flushLogger = true) + logEvent(event) } override def onExecutorRemoved(event: SparkListenerExecutorRemoved): Unit = { - logEvent(event, flushLogger = true) + logEvent(event) } override def onExecutorBlacklisted(event: SparkListenerExecutorBlacklisted): Unit = { - logEvent(event, flushLogger = true) + logEvent(event) } override def onExecutorExcluded(event: SparkListenerExecutorExcluded): Unit = { - logEvent(event, flushLogger = true) + logEvent(event) } override def onExecutorBlacklistedForStage( event: SparkListenerExecutorBlacklistedForStage): Unit = { - logEvent(event, flushLogger = true) + logEvent(event) } override def onExecutorExcludedForStage( event: SparkListenerExecutorExcludedForStage): Unit = { - logEvent(event, flushLogger = true) + logEvent(event) } override def onNodeBlacklistedForStage(event: SparkListenerNodeBlacklistedForStage): Unit = { - logEvent(event, flushLogger = true) + logEvent(event) } override def onNodeExcludedForStage(event: SparkListenerNodeExcludedForStage): Unit = { - logEvent(event, flushLogger = true) + logEvent(event) } override def onExecutorUnblacklisted(event: SparkListenerExecutorUnblacklisted): Unit = { - logEvent(event, flushLogger = true) + logEvent(event) } override def onExecutorUnexcluded(event: SparkListenerExecutorUnexcluded): Unit = { - logEvent(event, flushLogger = true) + logEvent(event) } override def onNodeBlacklisted(event: SparkListenerNodeBlacklisted): Unit = { - logEvent(event, flushLogger = true) + logEvent(event) } override def onNodeExcluded(event: SparkListenerNodeExcluded): Unit = { - logEvent(event, flushLogger = true) + logEvent(event) } override def onNodeUnblacklisted(event: SparkListenerNodeUnblacklisted): Unit = { - logEvent(event, flushLogger = true) + logEvent(event) } override def onNodeUnexcluded(event: SparkListenerNodeUnexcluded): Unit = { - logEvent(event, flushLogger = true) + logEvent(event) } override def onBlockUpdated(event: SparkListenerBlockUpdated): Unit = { if (shouldLogBlockUpdates) { - logEvent(event, flushLogger = true) + logEvent(event) } } @@ -266,12 +266,12 @@ private[spark] class EventLoggingListener( } override def onResourceProfileAdded(event: SparkListenerResourceProfileAdded): Unit = { - logEvent(event, flushLogger = true) + logEvent(event) } override def onOtherEvent(event: SparkListenerEvent): Unit = { if (event.logEvent) { - logEvent(event, flushLogger = true) + logEvent(event) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 69b626029e4f4..13198f758d797 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -1046,7 +1046,7 @@ private[spark] class TaskSetManager( // could serve the shuffle outputs or the executor lost is caused by decommission (which // can destroy the whole host). The reason is the next stage wouldn't be able to fetch the // data from this dead executor so we would need to rerun these tasks on other executors. - val maybeShuffleMapOutputLoss = isShuffleMapTasks && + val maybeShuffleMapOutputLoss = isShuffleMapTasks && !conf.isRssEnable() && !sched.sc.shuffleDriverComponents.supportsReliableStorage() && (reason.isInstanceOf[ExecutorDecommission] || !env.blockManager.externalShuffleServiceEnabled) if (maybeShuffleMapOutputLoss && !isZombie) { diff --git a/core/src/main/scala/org/apache/spark/status/KVUtils.scala b/core/src/main/scala/org/apache/spark/status/KVUtils.scala index 0dd40962309a4..e7773b8a4350c 100644 --- a/core/src/main/scala/org/apache/spark/status/KVUtils.scala +++ b/core/src/main/scala/org/apache/spark/status/KVUtils.scala @@ -210,6 +210,20 @@ private[spark] object KVUtils extends Logging { } } + /** + * Maps all values of KVStoreView to new values using a transformation function + * and filtered by a filter function. + */ + def mapToSeqWithFilter[T, B]( + view: KVStoreView[T], + max: Int) + (mapFunc: T => B) + (filterFunc: B => Boolean): Seq[B] = { + Utils.tryWithResource(view.closeableIterator()) { iter => + iter.asScala.map(mapFunc).filter(filterFunc).take(max).toList + } + } + def size[T](view: KVStoreView[T]): Int = { Utils.tryWithResource(view.closeableIterator()) { iter => iter.asScala.size diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala index cc21c1488f67c..44db9f2eec53e 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala @@ -82,6 +82,10 @@ private[spark] trait UIRoot { def withSparkUI[T](appId: String, attemptId: Option[String])(fn: SparkUI => T): T def getApplicationInfoList: Iterator[ApplicationInfo] + + def getApplicationInfoList(max: Int)( + filter: ApplicationInfo => Boolean): Iterator[ApplicationInfo] + def getApplicationInfo(appId: String): Option[ApplicationInfo] /** diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala index 6eb8b2bfd55a3..04a25a818c3c6 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala @@ -37,7 +37,7 @@ private[v1] class ApplicationListResource extends ApiRequestContext { val includeCompleted = status.isEmpty || status.contains(ApplicationStatus.COMPLETED) val includeRunning = status.isEmpty || status.contains(ApplicationStatus.RUNNING) - uiRoot.getApplicationInfoList.filter { app => + uiRoot.getApplicationInfoList(numApps) { app => val anyRunning = app.attempts.isEmpty || !app.attempts.head.completed // if any attempt is still running, we consider the app to also still be running; // keep the app if *any* attempts fall in the right time window @@ -45,7 +45,7 @@ private[v1] class ApplicationListResource extends ApiRequestContext { app.attempts.exists { attempt => isAttemptInRange(attempt, minDate, maxDate, minEndDate, maxEndDate, anyRunning) } - }.take(numApps) + } } private def isAttemptInRange( diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 685407c11208f..4cbedc057c16b 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -192,6 +192,11 @@ private[spark] class SparkUI private ( )) } + override def getApplicationInfoList(max: Int)( + filter: ApplicationInfo => Boolean): Iterator[ApplicationInfo] = { + getApplicationInfoList.filter(filter).take(max) + } + def getApplicationInfo(appId: String): Option[ApplicationInfo] = { getApplicationInfoList.find(_.id == appId) } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 377caf776deb0..8fde30fd18024 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -22,7 +22,7 @@ import java.util.{Properties, UUID} import scala.collection.JavaConverters._ import scala.collection.Map -import com.fasterxml.jackson.core.JsonGenerator +import com.fasterxml.jackson.core.{JsonGenerator, StreamReadConstraints} import com.fasterxml.jackson.databind.JsonNode import org.json4s.jackson.JsonMethods.compact @@ -54,6 +54,11 @@ import org.apache.spark.util.Utils.weakIntern private[spark] object JsonProtocol extends JsonUtils { // TODO: Remove this file and put JSON serialization into each individual class. + // SPARK-49872: Remove jackson JSON string length limitation. + mapper.getFactory.setStreamReadConstraints( + StreamReadConstraints.builder().maxStringLength(Int.MaxValue).build() + ) + /** ------------------------------------------------- * * JSON serialization methods for SparkListenerEvents | * -------------------------------------------------- */ @@ -874,8 +879,14 @@ private[spark] object JsonProtocol extends JsonUtils { case `stageExecutorMetrics` => stageExecutorMetricsFromJson(json) case `blockUpdate` => blockUpdateFromJson(json) case `resourceProfileAdded` => resourceProfileAddedFromJson(json) - case other => mapper.readValue(json.toString, Utils.classForName(other)) - .asInstanceOf[SparkListenerEvent] + case other => + val otherClass = Utils.classForName(other) + if (classOf[SparkListenerEvent].isAssignableFrom(otherClass)) { + mapper.readValue(json.toString, otherClass) + .asInstanceOf[SparkListenerEvent] + } else { + throw new SparkException(s"Unknown event type: $other") + } } } diff --git a/core/src/test/resources/ProcfsMetrics/487713/stat b/core/src/test/resources/ProcfsMetrics/487713/stat new file mode 100644 index 0000000000000..63640b58155b5 --- /dev/null +++ b/core/src/test/resources/ProcfsMetrics/487713/stat @@ -0,0 +1 @@ +487713 ((Executor) task l)) D 474416 474398 474398 0 -1 4194368 5 0 0 0 0 0 0 0 25 5 1 0 1542745216 7469137920 120815 18446744073709551615 104424108929024 104424108932808 140734257079632 0 0 0 4 3 553671884 1 0 0 17 58 0 0 0 0 0 104424108940536 104424108941336 104424532111360 140734257083781 140734257085131 140734257085131 140734257102797 0 \ No newline at end of file diff --git a/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala b/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala index 571110784818f..f0e41046ae48b 100644 --- a/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala @@ -26,7 +26,7 @@ import org.apache.commons.io.filefilter.TrueFileFilter import org.scalatest.BeforeAndAfterAll import org.scalatest.matchers.should.Matchers._ -import org.apache.spark.internal.config.SHUFFLE_MANAGER +import org.apache.spark.internal.config.{SHUFFLE_CHECKSUM_ALGORITHM, SHUFFLE_MANAGER} import org.apache.spark.rdd.ShuffledRDD import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.shuffle.sort.SortShuffleManager @@ -69,8 +69,9 @@ class SortShuffleSuite extends ShuffleSuite with BeforeAndAfterAll { shuffledRdd.count() // Ensure that the shuffle actually created files that will need to be cleaned up val filesCreatedByShuffle = getAllFiles -- filesBeforeShuffle - filesCreatedByShuffle.map(_.getName) should be - Set("shuffle_0_0_0.data", "shuffle_0_0_0.index") + filesCreatedByShuffle.map(_.getName) should be( + Set("shuffle_0_0_0.data", s"shuffle_0_0_0.checksum.${conf.get(SHUFFLE_CHECKSUM_ALGORITHM)}", + "shuffle_0_0_0.index")) // Check that the cleanup actually removes the files sc.env.blockManager.master.removeShuffle(0, blocking = true) for (file <- filesCreatedByShuffle) { diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 8e2d6e6cf5ff3..dd8bb3b96480e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -1095,14 +1095,14 @@ class SparkSubmitSuite val appArgs = new SparkSubmitArguments(args) val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs) - conf.get("spark.yarn.dist.jars").split(",").toSet should be - (Set(jar1.toURI.toString, jar2.toURI.toString)) - conf.get("spark.yarn.dist.files").split(",").toSet should be - (Set(file1.toURI.toString, file2.toURI.toString)) - conf.get("spark.yarn.dist.pyFiles").split(",").toSet should be - (Set(pyFile1.getAbsolutePath, pyFile2.getAbsolutePath)) - conf.get("spark.yarn.dist.archives").split(",").toSet should be - (Set(archive1.toURI.toString, archive2.toURI.toString)) + conf.get("spark.yarn.dist.jars").split(",").toSet should be( + Set(jar1.toURI.toString, jar2.toURI.toString)) + conf.get("spark.yarn.dist.files").split(",").toSet should be( + Set(file1.toURI.toString, file2.toURI.toString)) + conf.get("spark.yarn.dist.pyFiles").split(",").toSet should be( + Set(pyFile1.toURI.toString, pyFile2.toURI.toString)) + conf.get("spark.yarn.dist.archives").split(",").toSet should be( + Set(archive1.toURI.toString, archive2.toURI.toString)) } } } diff --git a/core/src/test/scala/org/apache/spark/executor/ProcfsMetricsGetterSuite.scala b/core/src/test/scala/org/apache/spark/executor/ProcfsMetricsGetterSuite.scala index d583afdf07c49..bcafe153be0d5 100644 --- a/core/src/test/scala/org/apache/spark/executor/ProcfsMetricsGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ProcfsMetricsGetterSuite.scala @@ -62,4 +62,16 @@ class ProcfsMetricsGetterSuite extends SparkFunSuite { assert(r.pythonVmemTotal == 0) assert(r.pythonRSSTotal == 0) } + + test("SPARK-52776: Whitespace and parentheses in the comm field") { + val p = new ProcfsMetricsGetter(getTestResourcePath("ProcfsMetrics")) + var r = ProcfsMetrics(0, 0, 0, 0, 0, 0) + r = p.addProcfsMetricsFromOneProcess(r, 487713) + assert(r.jvmVmemTotal == 0) + assert(r.jvmRSSTotal == 0) + assert(r.pythonVmemTotal == 0) + assert(r.pythonRSSTotal == 0) + assert(r.otherVmemTotal == 7469137920L) + assert(r.otherRSSTotal == 494858240) + } } diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 8105df64705a4..8af8c8579232f 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -874,6 +874,36 @@ class JsonProtocolSuite extends SparkFunSuite { val jobFailedEvent = JsonProtocol.sparkEventFromJson(exJobFailureNoStackJson) testEvent(jobFailedEvent, exJobFailureExpectedJson) } + + test("SPARK-52381: handle class not found") { + val unknownJson = + """{ + | "Event" : "com.example.UnknownEvent", + | "foo" : "foo" + |}""".stripMargin + try { + JsonProtocol.sparkEventFromJson(unknownJson) + fail("Expected ClassNotFoundException for unknown event type") + } catch { + case e: ClassNotFoundException => + } + } + + test("SPARK-52381: only read classes that extend SparkListenerEvent") { + val unknownJson = + """{ + | "Event" : "org.apache.spark.SparkException", + | "foo" : "foo" + |}""".stripMargin + try { + JsonProtocol.sparkEventFromJson(unknownJson) + fail("Expected SparkException for unknown event type") + } catch { + case e: SparkException => + assert(e.getMessage.startsWith("Unknown event type")) + } + } + } diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index d3f8c509910e9..a9dd229f6c69e 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -142,9 +142,322 @@ if [[ "$1" == "finalize" ]]; then git commit -m "Add docs for Apache Spark $RELEASE_VERSION" git push origin HEAD:asf-site cd .. - rm -rf spark-website echo "docs uploaded" + echo "Uploading release docs to spark-website" + cd spark-website + + # 1. Add download link to documentation.md + python3 <Spark {release_version}' +preview_newline = f'
  • Spark {release_version}
  • ' + +inserted = False + +def parse_version(v): + return [int(p) for p in v.strip().split(".")] + +def vercmp(v1, v2): + a = parse_version(v1) + b = parse_version(v2) + return (a > b) - (a < b) + +with open("documentation.md") as f: + lines = f.readlines() + +with open("documentation.md", "w") as f: + if is_preview: + in_preview_section = False + for i, line in enumerate(lines): + if '

    Documentation for preview releases:

    ' in line: + in_preview_section = True + f.write(line) + continue + + if in_preview_section and re.search(r'docs/\d+\.\d+\.\d+-preview\d*/', line): + existing_version = re.search(r'docs/(\d+\.\d+\.\d+-preview\d*)/', line).group(1) + + if existing_version == release_version: + inserted = True # Already exists, don't add + elif not inserted: + base_existing = re.sub(r'-preview\d*$', '', existing_version) + preview_num_existing = int(re.search(r'preview(\d*)', existing_version).group(1) or "0") + preview_num_new = int(re.search(r'preview(\d*)', release_version).group(1) or "0") + + if (vercmp(base_version, base_existing) > 0) or \ + (vercmp(base_version, base_existing) == 0 and preview_num_new >= preview_num_existing): + f.write(preview_newline + "\n") + inserted = True + + f.write(line) + continue + + if in_preview_section and "" in line and not inserted: + f.write(preview_newline + "\n") + inserted = True + f.write(line) + else: + for line in lines: + match = re.search(r'docs/(\d+\.\d+\.\d+)/', line) + if not inserted and match: + existing_version = match.group(1) + if vercmp(release_version, existing_version) >= 0: + f.write(stable_newline + "\n") + inserted = True + f.write(line) + if not inserted: + f.write(stable_newline + "\n") +EOF + + echo "Edited documentation.md" + + # 2. Add download link to js/downloads.js + if [[ "$RELEASE_VERSION" =~ -preview[0-9]*$ ]]; then + echo "Skipping js/downloads.js for preview release: $RELEASE_VERSION" + else + RELEASE_DATE=$(TZ=America/Los_Angeles date +"%m/%d/%Y") + IFS='.' read -r rel_maj rel_min rel_patch <<< "$RELEASE_VERSION" + NEW_PACKAGES="packagesV14" + if [[ "$rel_maj" -ge 4 ]]; then + NEW_PACKAGES="packagesV15" + fi + + python3 < b) - (a < b) + +inserted = replaced = False + +with open("js/downloads.js") as f: + lines = f.readlines() + +with open("js/downloads.js", "w") as f: + for line in lines: + m = re.search(r'addRelease\("(\d+\.\d+\.\d+)"', line) + if m: + existing_version = m.group(1) + cmp_result = vercmp(release_version, existing_version) + ex_major, ex_minor, ex_patch = parse_version(existing_version) + + if cmp_result == 0: + f.write(newline + "\n") + replaced = True + elif not replaced and ex_major == new_major and ex_minor == new_minor: + f.write(newline + "\n") + replaced = True + elif not replaced and not inserted and cmp_result > 0: + f.write(newline + "\n") + f.write(line) + inserted = True + else: + f.write(line) + else: + f.write(line) + if not replaced and not inserted: + f.write(newline + "\n") +EOF + + echo "Edited js/downloads.js" + fi + + # 3. Add news post + RELEASE_DATE=$(TZ=America/Los_Angeles date +"%Y-%m-%d") + FILENAME="news/_posts/${RELEASE_DATE}-spark-${RELEASE_VERSION//./-}-released.md" + mkdir -p news/_posts + + if [[ "$RELEASE_VERSION" =~ -preview[0-9]*$ ]]; then + BASE_VERSION="${RELEASE_VERSION%%-preview*}" + cat > "$FILENAME" <Spark ${RELEASE_VERSION} release. +This preview is not a stable release in terms of either API or functionality, but it is meant to give the community early +access to try the code that will become Spark ${BASE_VERSION}. If you would like to test the release, +please download it, and send feedback using either +mailing lists or +JIRA. +The documentation is available at the link. + +We'd like to thank our contributors and users for their contributions and early feedback to this release. This release would not have been possible without you. +EOF + + else + cat > "$FILENAME" <Apache Spark ${RELEASE_VERSION}! Visit the release notes to read about the new features, or download the release today. +EOF + fi + + echo "Created $FILENAME" + + # 4. Add release notes with Python to extract JIRA version ID + if [[ "$RELEASE_VERSION" =~ -preview[0-9]*$ ]]; then + echo "Skipping JIRA release notes for preview release: $RELEASE_VERSION" + else + RELEASE_DATE=$(TZ=America/Los_Angeles date +"%Y-%m-%d") + JIRA_PROJECT_ID=12315420 + JIRA_URL="https://issues.apache.org/jira/rest/api/2/project/SPARK/versions" + JSON=$(curl -s "$JIRA_URL") + + VERSION_ID=$(python3 - <&2 + fi + + JIRA_LINK="https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=${JIRA_PROJECT_ID}&version=${VERSION_ID}" + + IFS='.' read -r rel_maj rel_min rel_patch <<< "$RELEASE_VERSION" + if [[ "$rel_patch" -eq 0 ]]; then + ACKNOWLEDGE="patches and features to this release." + BODY="Apache Spark ${RELEASE_VERSION} is a new feature release. It introduces new functionality and improvements. We encourage users to try it and provide feedback." + else + ACKNOWLEDGE="patches to this release." + BODY="Apache Spark ${RELEASE_VERSION} is a maintenance release containing security and correctness fixes. This release is based on the branch-${rel_maj}.${rel_min} maintenance branch of Spark. We strongly recommend all ${rel_maj}.${rel_min} users to upgrade to this stable release." + fi + + BODY+=" + +You can find the list of resolved issues and detailed changes in the [JIRA release notes](${JIRA_LINK}). + +We would like to acknowledge all community members for contributing ${ACKNOWLEDGE}" + + FILENAME="releases/_posts/${RELEASE_DATE}-spark-release-${RELEASE_VERSION}.md" + mkdir -p releases/_posts + cat > "$FILENAME" < $RELEASE_VERSION (preview release)" + + else + LINK_PATH="site/docs/latest" + + if [[ "$rel_patch" -eq 0 ]]; then + if [[ -L "$LINK_PATH" ]]; then + CURRENT_TARGET=$(readlink "$LINK_PATH") + else + CURRENT_TARGET="" + fi + + if [[ "$CURRENT_TARGET" =~ ^[0-9]+\.[0-9]+\.[0-9]+$ ]]; then + IFS='.' read -r cur_maj cur_min cur_patch <<< "$CURRENT_TARGET" + + if [[ "$rel_maj" -gt "$cur_maj" ]]; then + ln -sfn "$RELEASE_VERSION" "$LINK_PATH" + echo "Updated symlink $LINK_PATH -> $RELEASE_VERSION (major version increased)" + elif [[ "$rel_maj" -eq "$cur_maj" && "$rel_min" -gt "$cur_min" ]]; then + ln -sfn "$RELEASE_VERSION" "$LINK_PATH" + echo "Updated symlink $LINK_PATH -> $RELEASE_VERSION (minor version increased)" + else + echo "Symlink $LINK_PATH points to $CURRENT_TARGET with equal or newer major.minor, no change" + fi + else + echo "No valid existing version target." + fi + else + echo "Patch release detected ($RELEASE_VERSION), not updating symlink" + fi + fi + + git add . + git commit -m "Add release docs for Apache Spark $RELEASE_VERSION" + git push origin HEAD:asf-site + cd .. + echo "release docs uploaded" + rm -rf spark-website + # Moves the docs from dev directory to release directory. echo "Moving Spark docs to the release directory" svn mv --username "$ASF_USERNAME" --password "$ASF_PASSWORD" -m"Apache Spark $RELEASE_VERSION" \ @@ -216,7 +529,6 @@ if [[ "$1" == "finalize" ]]; then echo "Done." - # TODO: Test it in the actual official release # Remove old releases from the mirror # Extract major.minor prefix RELEASE_SERIES=$(echo "$RELEASE_VERSION" | cut -d. -f1-2) @@ -615,13 +927,15 @@ if [[ "$1" == "publish-release" ]]; then # Calculate deadline in Pacific Time (PST/PDT) DEADLINE=$(TZ=America/Los_Angeles date -d "+4 days" "+%a, %d %b %Y %H:%M:%S %Z") + PYSPARK_VERSION=`echo "$RELEASE_VERSION" | sed -e "s/-/./" -e "s/preview/dev/"` JIRA_API_URL="https://issues.apache.org/jira/rest/api/2/project/SPARK/versions" + SPARK_VERSION_BASE=$(echo "$SPARK_VERSION" | sed 's/-preview[0-9]*//') JIRA_VERSION_ID=$(curl -s "$JIRA_API_URL" | \ # Split JSON objects by replacing '},{' with a newline-separated pattern tr '}' '\n' | \ # Find the block containing the exact version name - grep -F "\"name\":\"$SPARK_VERSION\"" -A 5 | \ + grep -F "\"name\":\"$SPARK_VERSION_BASE\"" -A 5 | \ # Extract the line with "id" grep '"id"' | \ # Extract the numeric id value (assuming "id":"123456") @@ -693,7 +1007,7 @@ EOF echo "reporting any regressions." echo echo "If you're working in PySpark you can set up a virtual env and install" - echo "the current RC via \"pip install https://dist.apache.org/repos/dist/dev/spark/${GIT_REF}-bin/pyspark-${SPARK_VERSION}.tar.gz\"" + echo "the current RC via \"pip install https://dist.apache.org/repos/dist/dev/spark/${GIT_REF}-bin/pyspark-${PYSPARK_VERSION}.tar.gz\"" echo "and see if anything important breaks." echo "In the Java/Scala, you can add the staging repository to your project's resolvers and test" echo "with the RC (make sure to clean up the artifact cache before/after so" diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index dbf0cb34c5353..b4034c171fd3a 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -212,9 +212,9 @@ opencsv/2.3//opencsv-2.3.jar opentracing-api/0.33.0//opentracing-api-0.33.0.jar opentracing-noop/0.33.0//opentracing-noop-0.33.0.jar opentracing-util/0.33.0//opentracing-util-0.33.0.jar -orc-core/1.9.6/shaded-protobuf/orc-core-1.9.6-shaded-protobuf.jar -orc-mapreduce/1.9.6/shaded-protobuf/orc-mapreduce-1.9.6-shaded-protobuf.jar -orc-shims/1.9.6//orc-shims-1.9.6.jar +orc-core/1.9.7/shaded-protobuf/orc-core-1.9.7-shaded-protobuf.jar +orc-mapreduce/1.9.7/shaded-protobuf/orc-mapreduce-1.9.7-shaded-protobuf.jar +orc-shims/1.9.7//orc-shims-1.9.7.jar oro/2.0.8//oro-2.0.8.jar osgi-resource-locator/1.0.3//osgi-resource-locator-1.0.3.jar paranamer/2.8//paranamer-2.8.jar diff --git a/dev/infra/Dockerfile b/dev/infra/Dockerfile index f0b88666c040d..811b2dba3768e 100644 --- a/dev/infra/Dockerfile +++ b/dev/infra/Dockerfile @@ -43,7 +43,7 @@ RUN mkdir -p /usr/local/pypy/pypy3.8 && \ ln -sf /usr/local/pypy/pypy3.8/bin/pypy /usr/local/bin/pypy3.8 && \ ln -sf /usr/local/pypy/pypy3.8/bin/pypy /usr/local/bin/pypy3 -RUN curl -sS https://bootstrap.pypa.io/get-pip.py | pypy3 +RUN curl -sS https://bootstrap.pypa.io/pip/3.8/get-pip.py | pypy3 RUN $APT_INSTALL gnupg ca-certificates pandoc RUN echo 'deb https://cloud.r-project.org/bin/linux/ubuntu focal-cran40/' >> /etc/apt/sources.list diff --git a/dev/make-distribution.sh b/dev/make-distribution.sh index ef7c010e930a1..a182981dfa063 100755 --- a/dev/make-distribution.sh +++ b/dev/make-distribution.sh @@ -167,6 +167,7 @@ export MAVEN_OPTS="${MAVEN_OPTS:--Xss128m -Xmx4g -XX:ReservedCodeCacheSize=128m} # Normal quoting tricks don't work. # See: http://mywiki.wooledge.org/BashFAQ/050 BUILD_COMMAND=("$MVN" clean package \ + -s ~/.m2/settings.xml \ -DskipTests \ -Dmaven.javadoc.skip=true \ -Dmaven.scaladoc.skip=true \ @@ -187,7 +188,13 @@ echo "Spark $VERSION$GITREVSTRING built for Hadoop $SPARK_HADOOP_VERSION" > "$DI echo "Build flags: $@" >> "$DISTDIR/RELEASE" # Copy jars -cp "$SPARK_HOME"/assembly/target/scala*/jars/* "$DISTDIR/jars/" +# Fabio: copy jars from the spark-assemmbly-*-dist directory which +# contains the distribution prepared by the maven-assembly-plugin +# The maven-assembly-plugin has rules to remove the hadoop/hops dependencies +# from the final distribution +# You need to run the -Pbigtop-dist profile for this to work +cp "$SPARK_HOME"/assembly/target/spark-assembly_"$SCALA_VERSION"-"$VERSION"-dist/lib/* "$DISTDIR/jars/" +cp "$SPARK_HOME"/assembly/target/spark-assembly_"$SCALA_VERSION"-"$VERSION"-dist/*.jar "$DISTDIR/jars/" # Only create the yarn directory if the yarn artifacts were built. if [ -f "$SPARK_HOME"/common/network-yarn/target/scala*/spark-*-yarn-shuffle.jar ]; then diff --git a/dev/test-dependencies.sh b/dev/test-dependencies.sh index 36cc7a4f994dc..636ed0ffb0cad 100755 --- a/dev/test-dependencies.sh +++ b/dev/test-dependencies.sh @@ -37,6 +37,9 @@ HADOOP_HIVE_PROFILES=( hadoop-3-hive-2.3 ) +MVN_EXEC_PLUGIN_VERSION=$(build/mvn help:evaluate \ + -Dexpression=exec-maven-plugin.version -q -DforceStdout | grep -E "[0-9]+\.[0-9]+\.[0-9]+") + # We'll switch the version to a temp. one, publish POMs using that new version, then switch back to # the old version. We need to do this because the `dependency:build-classpath` task needs to # resolve Spark's internal submodule dependencies. @@ -47,7 +50,7 @@ OLD_VERSION=$($MVN -q \ -Dexec.executable="echo" \ -Dexec.args='${project.version}' \ --non-recursive \ - org.codehaus.mojo:exec-maven-plugin:1.6.0:exec | grep -E '[0-9]+\.[0-9]+\.[0-9]+') + org.codehaus.mojo:exec-maven-plugin:${MVN_EXEC_PLUGIN_VERSION}:exec | grep -E '[0-9]+\.[0-9]+\.[0-9]+') # dependency:get for guava and jetty-io are workaround for SPARK-37302. GUAVA_VERSION=$(build/mvn help:evaluate -Dexpression=guava.version -q -DforceStdout | grep -E "^[0-9.]+$") build/mvn dependency:get -Dartifact=com.google.guava:guava:${GUAVA_VERSION} -q @@ -61,7 +64,7 @@ SCALA_BINARY_VERSION=$($MVN -q \ -Dexec.executable="echo" \ -Dexec.args='${scala.binary.version}' \ --non-recursive \ - org.codehaus.mojo:exec-maven-plugin:1.6.0:exec | grep -E '[0-9]+\.[0-9]+') + org.codehaus.mojo:exec-maven-plugin:${MVN_EXEC_PLUGIN_VERSION}:exec | grep -E '[0-9]+\.[0-9]+') if [[ "$SCALA_BINARY_VERSION" != "2.12" ]]; then echo "Skip dependency testing on $SCALA_BINARY_VERSION" exit 0 diff --git a/docs/_config.yml b/docs/_config.yml index d19efdb99ba06..acfa53576a9ee 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -19,8 +19,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 3.5.8-SNAPSHOT -SPARK_VERSION_SHORT: 3.5.8 +SPARK_VERSION: 3.5.7-SNAPSHOT +SPARK_VERSION_SHORT: 3.5.7 SCALA_BINARY_VERSION: "2.12" SCALA_VERSION: "2.12.18" MESOS_VERSION: 1.0.0 @@ -40,7 +40,7 @@ DOCSEARCH_SCRIPT: | inputSelector: '#docsearch-input', enhancedSearchInput: true, algoliaOptions: { - 'facetFilters': ["version:3.5.8"] + 'facetFilters': ["version:3.5.7"] }, debug: false // Set debug to true if you want to inspect the dropdown }); diff --git a/docs/_layouts/redirect.html b/docs/_layouts/redirect.html index 72a0462fc6a30..6177f91b7d793 100644 --- a/docs/_layouts/redirect.html +++ b/docs/_layouts/redirect.html @@ -19,10 +19,11 @@ Redirecting… - - - +{% assign redirect_url = page.redirect.to | replace_first: '/', '' | prepend: rel_path_to_root | append: '.html' %} + + +

    Redirecting…

    -Click here if you are not redirected. - \ No newline at end of file +Click here if you are not redirected. + diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index bafc479974551..28d5e0d82c93a 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -136,7 +136,6 @@ mkdir_p "api/python" puts "cp -r ../python/docs/build/html/. api/python" - rm_r("../python/docs/build/html/_sources") cp_r("../python/docs/build/html/.", "api/python") end diff --git a/examples/pom.xml b/examples/pom.xml index 00bc0d7bca367..4cf1847cc16c3 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 9f9ff9b0d9157..287116cca802a 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index fc2d9b5799adb..94cb21db3a01d 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../pom.xml @@ -59,13 +59,13 @@ test
    - org.apache.hadoop + ${hadoop.group} hadoop-client-api ${hadoop.version} provided - org.apache.hadoop + ${hadoop.group} hadoop-client-runtime ${hadoop.version} @@ -74,7 +74,7 @@ intra-jackson-module version problems. --> - org.apache.hadoop + ${hadoop.group} hadoop-aws ${hadoop.version} ${hadoop.deps.scope} @@ -130,7 +130,7 @@ ${hadoop.deps.scope} - org.apache.hadoop + ${hadoop.group} hadoop-azure ${hadoop.version} ${hadoop.deps.scope} @@ -146,13 +146,13 @@ but it still needs some selective exclusion across versions, especially 3.0.x. --> - org.apache.hadoop + ${hadoop.group} hadoop-cloud-storage ${hadoop.version} ${hadoop.deps.scope} - org.apache.hadoop + ${hadoop.group} hadoop-common @@ -167,7 +167,7 @@ manually exclude it to avoid recurring issues similar to HADOOP-18159 in Spark. --> - org.apache.hadoop + ${hadoop.group} hadoop-cos diff --git a/launcher/pom.xml b/launcher/pom.xml index 5a56efc2d168f..97e74a0998958 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../pom.xml @@ -91,13 +91,13 @@ - org.apache.hadoop + ${hadoop.group} hadoop-client-api ${hadoop.version} test - org.apache.hadoop + ${hadoop.group} hadoop-client-runtime ${hadoop.version} test diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 0f1e9fa843757..a7020fabd259a 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index adfe9b29141fa..a9e8fca6e1b19 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../pom.xml diff --git a/pom.xml b/pom.xml index 4f912329beed6..b09b6345f7cc3 100644 --- a/pom.xml +++ b/pom.xml @@ -22,11 +22,11 @@ org.apache apache - 18 + 31 org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 pom Spark Project Parent POM https://spark.apache.org/ @@ -119,29 +119,30 @@ 3.1.0 spark 9.5 - 2.0.7 - 2.20.0 + 2.0.17 + 2.24.3 - 3.3.4 + 3.2.0.17-EE-RC1 + io.hops 3.23.4 3.11.4 ${hadoop.version} 3.6.3 2.13.0 - org.apache.hive + io.hops.hive core - 2.3.9 - 2.3.9 + 3.0.0.13.10-HWORKS-2203-SNAPSHOT + 3.0.0.13.10-HWORKS-2203-SNAPSHOT - 2.3 + 3.0 3.4.1 10.14.2.0 1.13.1 - 1.9.6 + 1.9.7 shaded-protobuf 9.4.56.v20240826 4.0.3 @@ -226,7 +227,7 @@ If you are changing Arrow version specification, please check ./python/pyspark/sql/pandas/utils.py, and ./python/setup.py too. --> - 12.0.1 + 17.0.0 2.5.9 @@ -266,12 +267,14 @@ --> compile compile - 2.8.1 + 2.6.1.2 compile compile compile compile compile + compile + compile compile compile test @@ -348,6 +351,39 @@ false + + HopsEE + Hops Release Repository + https://nexus.hops.works/repository/hops-artifacts + + true + + + true + + + + Hops + Hops Repo + https://archiva.hops.works/repository/Hops/ + + true + + + true + + + + HiveEE + HiveEE Repo + https://nexus.hops.works/repository/hive-artifacts + + true + + + true + + @@ -773,6 +809,11 @@ log4j-slf4j2-impl ${log4j.version} + + org.apache.logging.log4j + log4j-slf4j-impl + ${log4j.version} + org.apache.logging.log4j log4j-api @@ -1326,26 +1367,26 @@ - org.apache.hadoop + ${hadoop.group} hadoop-client-api ${hadoop.version} ${hadoop.deps.scope} - org.apache.hadoop + ${hadoop.group} hadoop-client-runtime ${hadoop.version} ${hadoop.deps.scope} - org.apache.hadoop + ${hadoop.group} hadoop-client-minicluster ${yarn.version} test - org.apache.hadoop + ${hadoop.group} hadoop-client ${hadoop.version} ${hadoop.deps.scope} @@ -1435,7 +1476,7 @@ - org.apache.hadoop + ${hadoop.group} hadoop-minikdc ${hadoop.version} test @@ -1567,7 +1608,7 @@ ${hadoop.deps.scope} - org.apache.hadoop + ${hadoop.group} hadoop-yarn-api ${yarn.version} ${hadoop.deps.scope} @@ -1611,7 +1652,7 @@ - org.apache.hadoop + ${hadoop.group} hadoop-yarn-common ${yarn.version} ${hadoop.deps.scope} @@ -1659,7 +1700,7 @@ - org.apache.hadoop + ${hadoop.group} hadoop-yarn-server-tests ${yarn.version} tests @@ -1702,7 +1743,7 @@ * - org.apache.hadoop + ${hadoop.group} hadoop-yarn-server-resourcemanager @@ -1714,27 +1755,27 @@ fail, see also SPARK-33104. --> - org.apache.hadoop + ${hadoop.group} hadoop-yarn-server-resourcemanager ${yarn.version} test - org.apache.hadoop + ${hadoop.group} hadoop-yarn-server-web-proxy ${yarn.version} ${hadoop.deps.scope} - org.apache.hadoop + ${hadoop.group} hadoop-yarn-server-common - org.apache.hadoop + ${hadoop.group} hadoop-yarn-common - org.apache.hadoop + ${hadoop.group} hadoop-yarn-api @@ -1797,7 +1838,7 @@ - org.apache.hadoop + ${hadoop.group} hadoop-yarn-client ${yarn.version} ${hadoop.deps.scope} @@ -1888,13 +1929,23 @@ org.codehaus.jackson jackson-core-asl ${codehaus.jackson.version} - ${hadoop.deps.scope} + ${hive.deps.scope} org.codehaus.jackson jackson-mapper-asl ${codehaus.jackson.version} - ${hadoop.deps.scope} + ${hive.deps.scope} + + + org.codehaus.jackson + jackson-xc + ${codehaus.jackson.version} + + + org.codehaus.jackson + jackson-jaxrs + ${codehaus.jackson.version} ${hive.group} @@ -1912,7 +1963,7 @@ ${hive.group} - hive-jdbc + hops-jdbc ${hive.group} @@ -1968,7 +2019,7 @@ ${hive.group} - hive-jdbc + hops-jdbc ${hive.group} @@ -2023,11 +2074,11 @@ ant - org.apache.hadoop + ${hadoop.group} hadoop-common - org.apache.hadoop + ${hadoop.group} hadoop-auth @@ -2077,7 +2128,7 @@ - org.apache.hive + io.hops.hive hive-storage-api @@ -2137,11 +2188,6 @@ org.apache.avro avro-mapred - - - org.apache.calcite - calcite-core - org.apache.calcite calcite-avatica @@ -2178,6 +2224,10 @@ org.slf4j slf4j-log4j12 + + org.apache.logging.log4j + * + log4j log4j @@ -2190,6 +2240,10 @@ org.codehaus.groovy groovy-all + + com.sun.jersey + * + jline jline @@ -2240,7 +2294,7 @@ ${hive.group} - hive-jdbc + hops-jdbc ${hive.version} @@ -2287,10 +2341,26 @@ org.slf4j slf4j-api + + log4j + log4j + + + log4j + log4j + + + com.sun.jersey + * + org.slf4j slf4j-log4j12 + + org.apache.logging.log4j + * + log4j log4j @@ -2340,6 +2410,14 @@ org.slf4j slf4j-api + + log4j + log4j + + + org.apache.logging.log4j + log4j-slf4j-impl + org.slf4j slf4j-log4j12 @@ -2431,6 +2509,10 @@ tomcat jasper-runtime + + com.google + flatbuffers + @@ -2438,7 +2520,18 @@ ${hive.group} hive-service-rpc - 3.1.3 + ${hive.version} + + + * + * + + + + + ${hive.group} + hive-service + ${hive.version} * @@ -2473,7 +2566,7 @@ guava - org.apache.hadoop + ${hadoop.group} hadoop-yarn-server-resourcemanager @@ -2508,6 +2601,10 @@ org.codehaus.groovy groovy-all + + com.sun.jersey + * + @@ -2521,42 +2618,50 @@ - org.apache.hive + io.hops.hive hive-llap-common ${hive23.version} ${hive.llap.scope} - org.apache.hive + io.hops.hive hive-common - org.apache.hive + io.hops.hive hive-serde org.slf4j slf4j-api + + log4j + log4j + + + com.sun.jersey + * + - org.apache.hive + io.hops.hive hive-llap-client ${hive23.version} test - org.apache.hive + io.hops.hive hive-common - org.apache.hive + io.hops.hive hive-serde - org.apache.hive + io.hops.hive hive-llap-common @@ -2575,6 +2680,10 @@ org.slf4j slf4j-api + + log4j + log4j + @@ -2586,19 +2695,19 @@ ${orc.deps.scope} - org.apache.hadoop + ${hadoop.group} hadoop-common - org.apache.hadoop + ${hadoop.group} hadoop-hdfs - org.apache.hadoop + ${hadoop.group} hadoop-client-api - org.apache.hive + io.hops.hive hive-storage-api @@ -2616,11 +2725,11 @@ ${orc.deps.scope} - org.apache.hadoop + ${hadoop.group} hadoop-common - org.apache.hadoop + ${hadoop.group} hadoop-mapreduce-client-core @@ -2628,7 +2737,7 @@ orc-core - org.apache.hive + io.hops.hive hive-storage-api @@ -2642,6 +2751,12 @@ parquet-column ${parquet.version} ${parquet.deps.scope} + + + com.h2database + h2 + + org.apache.parquet @@ -2663,6 +2778,12 @@ ${parquet.version} ${parquet.test.deps.scope} tests + + + com.h2database + h2 + + org.apache.parquet @@ -2804,7 +2925,7 @@ 2.9.1 - org.apache.hive + io.hops.hive hive-storage-api ${hive.storage.version} ${hive.storage.scope} @@ -3909,4 +4030,16 @@ + + + HopsEE + Hops Release Repository + https://nexus.hops.works/repository/hops-artifacts + + + HopsEE + Hops Repo + https://nexus.hops.works/repository/hops-artifacts/ + + diff --git a/python/pyspark/pandas/tests/data_type_ops/test_udt_ops.py b/python/pyspark/pandas/tests/data_type_ops/test_udt_ops.py index 45f8cca56ee94..7b264582e044b 100644 --- a/python/pyspark/pandas/tests/data_type_ops/test_udt_ops.py +++ b/python/pyspark/pandas/tests/data_type_ops/test_udt_ops.py @@ -129,6 +129,26 @@ def test_from_to_pandas(self): self.assert_eq(pser, psser._to_pandas()) self.assert_eq(ps.from_pandas(pser), psser) + def test_with_first_null(self): + lst = [None, None, None, SparseVector(1, {0: 0.1})] + pser = pd.Series(lst) + psser = ps.Series(lst) + self.assert_eq(pser, psser._to_pandas()) + self.assert_eq(ps.from_pandas(pser), psser) + + lst2 = [SparseVector(1, {0: 0.1}), None, None, None] + pdf = pd.DataFrame({"a": lst, "b": lst2}) + psdf = ps.DataFrame({"a": lst, "b": lst2}) + self.assert_eq(pdf, psdf._to_pandas()) + self.assert_eq(ps.from_pandas(pdf), psdf) + + def test_with_all_null(self): + lst = [None, None, None, None] + pser = pd.Series(lst, dtype=object) + psser = ps.Series(lst, dtype=object) + self.assert_eq(pser, psser._to_pandas()) + self.assert_eq(ps.from_pandas(pser), psser) + def test_isnull(self): self.assert_eq(self.pser.isnull(), self.psser.isnull()) diff --git a/python/pyspark/pandas/typedef/typehints.py b/python/pyspark/pandas/typedef/typehints.py index 012eabf958eb8..5c7b3e01686ae 100644 --- a/python/pyspark/pandas/typedef/typehints.py +++ b/python/pyspark/pandas/typedef/typehints.py @@ -354,8 +354,9 @@ def infer_pd_series_spark_type( if dtype == np.dtype("object"): if len(pser) == 0 or pser.isnull().all(): return types.NullType() - elif hasattr(pser.iloc[0], "__UDT__"): - return pser.iloc[0].__UDT__ + notnull = pser[pser.notnull()] + if hasattr(notnull.iloc[0], "__UDT__"): + return notnull.iloc[0].__UDT__ else: return from_arrow_type(pa.Array.from_pandas(pser).type, prefer_timestamp_ntz) elif isinstance(dtype, CategoricalDtype): diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index afa979dab019e..c52fa568b7b31 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -576,7 +576,7 @@ def schema(self) -> StructType: except Exception as e: raise PySparkValueError( error_class="CANNOT_PARSE_DATATYPE", - message_parameters={"error": str(e)}, + message_parameters={"msg": str(e)}, ) return self._schema diff --git a/python/pyspark/sql/tests/connect/test_parity_udtf.py b/python/pyspark/sql/tests/connect/test_parity_udtf.py index 17df397d2f11b..2e37c6962caf0 100644 --- a/python/pyspark/sql/tests/connect/test_parity_udtf.py +++ b/python/pyspark/sql/tests/connect/test_parity_udtf.py @@ -54,7 +54,7 @@ class TestUDTF: def eval(self, a: int): yield a + 1, - with self.assertRaisesRegex(InvalidPlanInput, "Invalid.*type"): + with self.assertRaisesRegex(Exception, "Invalid.*type"): TestUDTF(lit(1)).collect() @unittest.skipIf( diff --git a/python/pyspark/sql/tests/streaming/test_streaming_foreach_batch.py b/python/pyspark/sql/tests/streaming/test_streaming_foreach_batch.py index 393101a096ea4..c10db1330b46c 100644 --- a/python/pyspark/sql/tests/streaming/test_streaming_foreach_batch.py +++ b/python/pyspark/sql/tests/streaming/test_streaming_foreach_batch.py @@ -25,17 +25,18 @@ def test_streaming_foreach_batch(self): q = None def collectBatch(batch_df, batch_id): - batch_df.createOrReplaceGlobalTempView("test_view") + batch_df.write.format("parquet").saveAsTable("test_table1") try: df = self.spark.readStream.format("text").load("python/test_support/sql/streaming") q = df.writeStream.foreachBatch(collectBatch).start() q.processAllAvailable() - collected = self.spark.sql("select * from global_temp.test_view").collect() + collected = self.spark.sql("select * from test_table1").collect() self.assertTrue(len(collected), 2) finally: if q: q.stop() + self.spark.sql("DROP TABLE IF EXISTS test_table1") def test_streaming_foreach_batch_tempview(self): q = None @@ -46,18 +47,19 @@ def collectBatch(batch_df, batch_id): # clone the session which is no longer same with the session used to start the # streaming query assert len(batch_df.sparkSession.sql("SELECT * FROM updates").collect()) == 2 - # Write to a global view verify on the repl/client side. - batch_df.createOrReplaceGlobalTempView("temp_view") + # Write a table to verify on the repl/client side. + batch_df.write.format("parquet").saveAsTable("test_table2") try: df = self.spark.readStream.format("text").load("python/test_support/sql/streaming") q = df.writeStream.foreachBatch(collectBatch).start() q.processAllAvailable() - collected = self.spark.sql("SELECT * FROM global_temp.temp_view").collect() + collected = self.spark.sql("SELECT * FROM test_table2").collect() self.assertTrue(len(collected[0]), 2) finally: if q: q.stop() + self.spark.sql("DROP TABLE IF EXISTS test_table2") def test_streaming_foreach_batch_propagates_python_errors(self): from pyspark.errors import StreamingQueryException diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index a2a8796957623..d4d6b6b086aeb 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -1411,7 +1411,7 @@ def _parse_datatype_json_value(json_value: Union[dict, str]) -> DataType: else: raise PySparkValueError( error_class="CANNOT_PARSE_DATATYPE", - message_parameters={"error": str(json_value)}, + message_parameters={"msg": str(json_value)}, ) else: tpe = json_value["type"] diff --git a/python/pyspark/version.py b/python/pyspark/version.py index e67736d7e0548..d3152f58da705 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__: str = "3.5.8.dev0" +__version__: str = "3.5.7.dev0" diff --git a/repl/pom.xml b/repl/pom.xml index 8f3ae8b4f2fc7..1efb8b8fbe1f1 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index f1feb2a61325f..7a97c4c1ff06c 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 115c6ae85ab96..a004dd12fedec 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index e57c69d1040f3..566e0baf8e23c 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 6e45fb3113ece..f7bfa6d6ee6e7 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../pom.xml @@ -40,13 +40,13 @@ - org.apache.hadoop + ${hadoop.group} hadoop-client-runtime ${hadoop.version} ${hadoop.deps.scope} - org.apache.hadoop + ${hadoop.group} hadoop-client-minicluster ${hadoop.version} test @@ -92,7 +92,7 @@ test - org.apache.hadoop + ${hadoop.group} hadoop-client-api ${hadoop.version} diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala index 552cc98311e8f..5c98785089ef5 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala @@ -616,35 +616,35 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { val mergeManager1DB = ShuffleTestAccessor.mergeManagerLevelDB(mergeManager1) ShuffleTestAccessor.recoveryFile(mergeManager1) should be (mergeMgrFile) - ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1).size() equals 0 - ShuffleTestAccessor.reloadAppShuffleInfo( - mergeManager1, mergeManager1DB).size() equals 0 + assert(ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1).size() equals 0) + assert(ShuffleTestAccessor.reloadAppShuffleInfo( + mergeManager1, mergeManager1DB).size() equals 0) mergeManager1.registerExecutor(app1Id.toString, mergedShuffleInfo1) var appShuffleInfo = ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1) - appShuffleInfo.size() equals 1 + assert(appShuffleInfo.size() equals 1) appShuffleInfo.get(app1Id.toString).getAppPathsInfo should be (appPathsInfo1) var appShuffleInfoAfterReload = ShuffleTestAccessor.reloadAppShuffleInfo(mergeManager1, mergeManager1DB) - appShuffleInfoAfterReload.size() equals 1 + assert(appShuffleInfoAfterReload.size() equals 1) appShuffleInfoAfterReload.get(app1Id.toString).getAppPathsInfo should be (appPathsInfo1) mergeManager1.registerExecutor(app2Attempt1Id.toString, mergedShuffleInfo2Attempt1) appShuffleInfo = ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1) - appShuffleInfo.size() equals 2 + assert(appShuffleInfo.size() equals 2) appShuffleInfo.get(app1Id.toString).getAppPathsInfo should be (appPathsInfo1) appShuffleInfo.get( app2Attempt1Id.toString).getAppPathsInfo should be (appPathsInfo2Attempt1) appShuffleInfoAfterReload = ShuffleTestAccessor.reloadAppShuffleInfo(mergeManager1, mergeManager1DB) - appShuffleInfoAfterReload.size() equals 2 + assert(appShuffleInfoAfterReload.size() equals 2) appShuffleInfoAfterReload.get(app1Id.toString).getAppPathsInfo should be (appPathsInfo1) appShuffleInfoAfterReload.get( app2Attempt1Id.toString).getAppPathsInfo should be (appPathsInfo2Attempt1) mergeManager1.registerExecutor(app3IdNoAttemptId.toString, mergedShuffleInfo3NoAttemptId) appShuffleInfo = ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1) - appShuffleInfo.size() equals 3 + assert(appShuffleInfo.size() equals 3) appShuffleInfo.get(app1Id.toString).getAppPathsInfo should be (appPathsInfo1) appShuffleInfo.get( app2Attempt1Id.toString).getAppPathsInfo should be (appPathsInfo2Attempt1) @@ -652,7 +652,7 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { app3IdNoAttemptId.toString).getAppPathsInfo should be (appPathsInfo3NoAttempt) appShuffleInfoAfterReload = ShuffleTestAccessor.reloadAppShuffleInfo(mergeManager1, mergeManager1DB) - appShuffleInfoAfterReload.size() equals 3 + assert(appShuffleInfoAfterReload.size() equals 3) appShuffleInfoAfterReload.get(app1Id.toString).getAppPathsInfo should be (appPathsInfo1) appShuffleInfoAfterReload.get( app2Attempt1Id.toString).getAppPathsInfo should be (appPathsInfo2Attempt1) @@ -661,7 +661,7 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { mergeManager1.registerExecutor(app2Attempt2Id.toString, mergedShuffleInfo2Attempt2) appShuffleInfo = ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1) - appShuffleInfo.size() equals 3 + assert(appShuffleInfo.size() equals 3) appShuffleInfo.get(app1Id.toString).getAppPathsInfo should be (appPathsInfo1) appShuffleInfo.get( app2Attempt2Id.toString).getAppPathsInfo should be (appPathsInfo2Attempt2) @@ -669,7 +669,7 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { app3IdNoAttemptId.toString).getAppPathsInfo should be (appPathsInfo3NoAttempt) appShuffleInfoAfterReload = ShuffleTestAccessor.reloadAppShuffleInfo(mergeManager1, mergeManager1DB) - appShuffleInfoAfterReload.size() equals 3 + assert(appShuffleInfoAfterReload.size() equals 3) appShuffleInfoAfterReload.get(app1Id.toString).getAppPathsInfo should be (appPathsInfo1) appShuffleInfoAfterReload.get( app2Attempt2Id.toString).getAppPathsInfo should be (appPathsInfo2Attempt2) @@ -678,14 +678,14 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { mergeManager1.applicationRemoved(app2Attempt2Id.toString, true) appShuffleInfo = ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1) - appShuffleInfo.size() equals 2 + assert(appShuffleInfo.size() equals 2) appShuffleInfo.get(app1Id.toString).getAppPathsInfo should be (appPathsInfo1) assert(!appShuffleInfo.containsKey(app2Attempt2Id.toString)) appShuffleInfo.get( app3IdNoAttemptId.toString).getAppPathsInfo should be (appPathsInfo3NoAttempt) appShuffleInfoAfterReload = ShuffleTestAccessor.reloadAppShuffleInfo(mergeManager1, mergeManager1DB) - appShuffleInfoAfterReload.size() equals 2 + assert(appShuffleInfoAfterReload.size() equals 2) appShuffleInfoAfterReload.get(app1Id.toString).getAppPathsInfo should be (appPathsInfo1) assert(!appShuffleInfoAfterReload.containsKey(app2Attempt2Id.toString)) appShuffleInfoAfterReload.get( @@ -725,9 +725,9 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { val mergeManager1DB = ShuffleTestAccessor.mergeManagerLevelDB(mergeManager1) ShuffleTestAccessor.recoveryFile(mergeManager1) should be (mergeMgrFile) - ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1).size() equals 0 - ShuffleTestAccessor.reloadAppShuffleInfo( - mergeManager1, mergeManager1DB).size() equals 0 + assert(ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1).size() equals 0) + assert(ShuffleTestAccessor.reloadAppShuffleInfo( + mergeManager1, mergeManager1DB).size() equals 0) mergeManager1.registerExecutor(app1Id.toString, mergedShuffleInfo1) mergeManager1.registerExecutor(app2Attempt1Id.toString, mergedShuffleInfo2Attempt1) @@ -737,7 +737,7 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { prepareAppShufflePartition(mergeManager1, partitionId2, 2, "4") var appShuffleInfo = ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1) - appShuffleInfo.size() equals 2 + assert(appShuffleInfo.size() equals 2) appShuffleInfo.get(app1Id.toString).getAppPathsInfo should be (appPathsInfo1) appShuffleInfo.get( app2Attempt1Id.toString).getAppPathsInfo should be (appPathsInfo2Attempt1) @@ -745,7 +745,7 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { assert(!appShuffleInfo.get(app2Attempt1Id.toString).getShuffles.get(2).isFinalized) var appShuffleInfoAfterReload = ShuffleTestAccessor.reloadAppShuffleInfo(mergeManager1, mergeManager1DB) - appShuffleInfoAfterReload.size() equals 2 + assert(appShuffleInfoAfterReload.size() equals 2) appShuffleInfoAfterReload.get(app1Id.toString).getAppPathsInfo should be (appPathsInfo1) appShuffleInfoAfterReload.get( app2Attempt1Id.toString).getAppPathsInfo should be (appPathsInfo2Attempt1) @@ -765,12 +765,12 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { mergeManager1.applicationRemoved(app1Id.toString, true) appShuffleInfo = ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1) - appShuffleInfo.size() equals 1 + assert(appShuffleInfo.size() equals 1) assert(!appShuffleInfo.containsKey(app1Id.toString)) assert(appShuffleInfo.get(app2Attempt1Id.toString).getShuffles.get(2).isFinalized) appShuffleInfoAfterReload = ShuffleTestAccessor.reloadAppShuffleInfo(mergeManager1, mergeManager1DB) - appShuffleInfoAfterReload.size() equals 1 + assert(appShuffleInfoAfterReload.size() equals 1) assert(!appShuffleInfoAfterReload.containsKey(app1Id.toString)) assert(appShuffleInfoAfterReload.get(app2Attempt1Id.toString).getShuffles.get(2).isFinalized) @@ -844,7 +844,7 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { prepareAppShufflePartition(mergeManager1, partitionId2, 2, "4") var appShuffleInfo = ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1) - appShuffleInfo.size() equals 2 + assert(appShuffleInfo.size() equals 2) appShuffleInfo.get(app1Id.toString).getAppPathsInfo should be (appPathsInfo1) appShuffleInfo.get( app2Id.toString).getAppPathsInfo should be (appPathsInfo2Attempt1) @@ -867,20 +867,20 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { mergeManager1.applicationRemoved(app1Id.toString, true) appShuffleInfo = ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1) - appShuffleInfo.size() equals 1 + assert(appShuffleInfo.size() equals 1) assert(!appShuffleInfo.containsKey(app1Id.toString)) assert(appShuffleInfo.get(app2Id.toString).getShuffles.get(2).isFinalized) // Clear the AppsShuffleInfo hashmap and reload the hashmap from DB appShuffleInfoAfterReload = ShuffleTestAccessor.reloadAppShuffleInfo(mergeManager1, mergeManager1DB) - appShuffleInfoAfterReload.size() equals 1 + assert(appShuffleInfoAfterReload.size() equals 1) assert(!appShuffleInfoAfterReload.containsKey(app1Id.toString)) assert(appShuffleInfoAfterReload.get(app2Id.toString).getShuffles.get(2).isFinalized) // Register application app1Id again and reload the DB again mergeManager1.registerExecutor(app1Id.toString, mergedShuffleInfo1) appShuffleInfo = ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1) - appShuffleInfo.size() equals 2 + assert(appShuffleInfo.size() equals 2) appShuffleInfo.get(app1Id.toString).getAppPathsInfo should be (appPathsInfo1) assert(appShuffleInfo.get(app1Id.toString).getShuffles.isEmpty) assert(appShuffleInfo.get(app2Id.toString).getShuffles.get(2).isFinalized) @@ -924,7 +924,7 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { prepareAppShufflePartition(mergeManager1, partitionId1, 2, "4") var appShuffleInfo = ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1) - appShuffleInfo.size() equals 1 + assert(appShuffleInfo.size() equals 1) appShuffleInfo.get( app1Id.toString).getAppPathsInfo should be (appPathsInfo1Attempt1) assert(!appShuffleInfo.get(app1Id.toString).getShuffles.get(2).isFinalized) @@ -938,7 +938,7 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { prepareAppShufflePartition(mergeManager1, partitionId2, 2, "4") appShuffleInfo = ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1) - appShuffleInfo.size() equals 1 + assert(appShuffleInfo.size() equals 1) appShuffleInfo.get( app1Id.toString).getAppPathsInfo should be (appPathsInfo1Attempt2) assert(!appShuffleInfo.get(app1Id.toString).getShuffles.get(2).isFinalized) @@ -973,7 +973,7 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { val mergeManager3 = s3.shuffleMergeManager.asInstanceOf[RemoteBlockPushResolver] val mergeManager3DB = ShuffleTestAccessor.mergeManagerLevelDB(mergeManager3) appShuffleInfo = ShuffleTestAccessor.getAppsShuffleInfo(mergeManager3) - appShuffleInfo.size() equals 1 + assert(appShuffleInfo.size() equals 1) appShuffleInfo.get( app1Id.toString).getAppPathsInfo should be (appPathsInfo1Attempt2) assert(appShuffleInfo.get(app1Id.toString).getShuffles.get(2).isFinalized) @@ -1014,7 +1014,7 @@ abstract class YarnShuffleServiceSuite extends SparkFunSuite with Matchers { mergeManager1.registerExecutor(app1Id.toString, mergedShuffleInfo1Attempt2) val appShuffleInfo = ShuffleTestAccessor.getAppsShuffleInfo(mergeManager1) - appShuffleInfo.size() equals 1 + assert(appShuffleInfo.size() equals 1) appShuffleInfo.get( app1Id.toString).getAppPathsInfo should be (appPathsInfo1Attempt2) diff --git a/sql/api/pom.xml b/sql/api/pom.xml index 49dee295ff196..4a21574462c3a 100644 --- a/sql/api/pom.xml +++ b/sql/api/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 7317a3f7ab94a..ff0992e7b21c7 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../pom.xml diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index a8f80cfd69a80..a7ebde57b863e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -723,6 +723,10 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor } else { colResolved.havingCondition } + // `cond` might contain unresolved aggregate functions so defer its resolution to + // `ResolveAggregateFunctions` rule if needed. + if (!cond.resolved) return colResolved + // Try resolving the condition of the filter as though it is in the aggregate clause val (extraAggExprs, Seq(resolvedHavingCond)) = ResolveAggregateFunctions.resolveExprsWithAggregate(Seq(cond), aggForResolving) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InferWindowGroupLimit.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InferWindowGroupLimit.scala index 04204c6a2e108..f2e99721e9261 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InferWindowGroupLimit.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InferWindowGroupLimit.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.optimizer -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, CurrentRow, DenseRank, EqualTo, Expression, GreaterThan, GreaterThanOrEqual, IntegerLiteral, LessThan, LessThanOrEqual, Literal, NamedExpression, PredicateHelper, Rank, RowFrame, RowNumber, SpecifiedWindowFrame, UnboundedPreceding, WindowExpression, WindowSpecDefinition} +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, CurrentRow, DenseRank, EqualTo, Expression, GreaterThan, GreaterThanOrEqual, IntegerLiteral, LessThan, LessThanOrEqual, Literal, NamedExpression, PredicateHelper, Rank, RowFrame, RowNumber, SizeBasedWindowFunction, SpecifiedWindowFrame, UnboundedPreceding, WindowExpression, WindowSpecDefinition} import org.apache.spark.sql.catalyst.plans.logical.{Filter, Limit, LocalRelation, LogicalPlan, Window, WindowGroupLimit} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.{FILTER, WINDOW} @@ -53,13 +53,14 @@ object InferWindowGroupLimit extends Rule[LogicalPlan] with PredicateHelper { } /** - * All window expressions should use the same expanding window, so that - * we can safely do the early stop. + * All window expressions should use the same expanding window and do not contains + * `SizeBasedWindowFunction`, so that we can safely do the early stop. */ private def isExpandingWindow( windowExpression: NamedExpression): Boolean = windowExpression match { - case Alias(WindowExpression(_, WindowSpecDefinition(_, _, - SpecifiedWindowFrame(RowFrame, UnboundedPreceding, CurrentRow))), _) => true + case Alias(WindowExpression(windowFunction, WindowSpecDefinition(_, _, + SpecifiedWindowFrame(RowFrame, UnboundedPreceding, CurrentRow))), _) + if !windowFunction.isInstanceOf[SizeBasedWindowFunction] => true case _ => false } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregates.scala index badf4065f5fb5..08423a6856404 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregates.scala @@ -54,7 +54,13 @@ object RemoveRedundantAggregates extends Rule[LogicalPlan] with AliasHelper { .map(_.toAttribute) )) - upperHasNoDuplicateSensitiveAgg && upperRefsOnlyDeterministicNonAgg + // If the lower aggregation is global, it is not redundant because a project with + // non-aggregate expressions is different with global aggregation in semantics. + // E.g., if the input relation is empty, a project might be optimized to an empty + // relation, while a global aggregation will return a single row. + lazy val lowerIsGlobalAgg = lower.groupingExpressions.isEmpty + + upperHasNoDuplicateSensitiveAgg && upperRefsOnlyDeterministicNonAgg && !lowerIsGlobalAgg } private def isDuplicateSensitive(ae: AggregateExpression): Boolean = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 8186d5fa00c3a..ca6938588ddb3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -4398,7 +4398,7 @@ object SQLConf { .internal() .doc("When true, exclude the references from the subquery expressions (in, exists, etc.) " + s"while removing redundant aliases.") - .version("4.0.0") + .version("3.5.1") .booleanConf .createWithDefault(true) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala index 4352d5bc9c6bb..52ed183d5d2c8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala @@ -545,61 +545,42 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { checkCast("0", false) } - protected def checkInvalidCastFromNumericType(to: DataType): Unit = { - cast(1.toByte, to).checkInputDataTypes() == - DataTypeMismatch( - errorSubClass = "CAST_WITH_FUNC_SUGGESTION", - messageParameters = Map( - "srcType" -> toSQLType(Literal(1.toByte).dataType), - "targetType" -> toSQLType(to), - "functionNames" -> "`DATE_FROM_UNIX_DATE`" - ) - ) - cast(1.toShort, to).checkInputDataTypes() == - DataTypeMismatch( - errorSubClass = "CAST_WITH_FUNC_SUGGESTION", - messageParameters = Map( - "srcType" -> toSQLType(Literal(1.toShort).dataType), - "targetType" -> toSQLType(to), - "functionNames" -> "`DATE_FROM_UNIX_DATE`" - ) - ) - cast(1, to).checkInputDataTypes() == - DataTypeMismatch( - errorSubClass = "CAST_WITH_FUNC_SUGGESTION", - messageParameters = Map( - "srcType" -> toSQLType(Literal(1).dataType), - "targetType" -> toSQLType(to), - "functionNames" -> "`DATE_FROM_UNIX_DATE`" - ) - ) - cast(1L, to).checkInputDataTypes() == - DataTypeMismatch( - errorSubClass = "CAST_WITH_FUNC_SUGGESTION", - messageParameters = Map( - "srcType" -> toSQLType(Literal(1L).dataType), - "targetType" -> toSQLType(to), - "functionNames" -> "`DATE_FROM_UNIX_DATE`" - ) - ) - cast(1.0.toFloat, to).checkInputDataTypes() == - DataTypeMismatch( - errorSubClass = "CAST_WITH_FUNC_SUGGESTION", - messageParameters = Map( - "srcType" -> toSQLType(Literal(1.0.toFloat).dataType), - "targetType" -> toSQLType(to), - "functionNames" -> "`DATE_FROM_UNIX_DATE`" - ) - ) - cast(1.0, to).checkInputDataTypes() == - DataTypeMismatch( - errorSubClass = "CAST_WITH_FUNC_SUGGESTION", - messageParameters = Map( - "srcType" -> toSQLType(Literal(1.0).dataType), - "targetType" -> toSQLType(to), - "functionNames" -> "`DATE_FROM_UNIX_DATE`" - ) - ) + protected def createCastMismatch( + srcType: DataType, + targetType: DataType, + errorSubClass: String, + extraParams: Map[String, String] = Map.empty): DataTypeMismatch = { + val baseParams = Map( + "srcType" -> toSQLType(srcType), + "targetType" -> toSQLType(targetType) + ) + DataTypeMismatch(errorSubClass, baseParams ++ extraParams) + } + + protected def checkInvalidCastFromNumericTypeToDateType(): Unit = { + val errorSubClass = if (evalMode == EvalMode.LEGACY) { + "CAST_WITHOUT_SUGGESTION" + } else { + "CAST_WITH_FUNC_SUGGESTION" + } + val funcParams = if (evalMode == EvalMode.LEGACY) { + Map.empty[String, String] + } else { + Map("functionNames" -> "`DATE_FROM_UNIX_DATE`") + } + Seq(1.toByte, 1.toShort, 1, 1L, 1.0.toFloat, 1.0).foreach { testValue => + val expectedError = + createCastMismatch(Literal(testValue).dataType, DateType, errorSubClass, funcParams) + assert(cast(testValue, DateType).checkInputDataTypes() == expectedError) + } + } + protected def checkInvalidCastFromNumericTypeToTimestampNTZType(): Unit = { + // All numeric types: `CAST_WITHOUT_SUGGESTION` + Seq(1.toByte, 1.toShort, 1, 1L, 1.0.toFloat, 1.0).foreach { testValue => + val expectedError = + createCastMismatch(Literal(testValue).dataType, TimestampNTZType, "CAST_WITHOUT_SUGGESTION") + assert(cast(testValue, TimestampNTZType).checkInputDataTypes() == expectedError) + } } test("SPARK-16729 type checking for casting to date type") { @@ -614,7 +595,7 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { ) ) ) - checkInvalidCastFromNumericType(DateType) + checkInvalidCastFromNumericTypeToDateType() } test("SPARK-20302 cast with same structure") { @@ -996,7 +977,7 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { test("disallow type conversions between Numeric types and Timestamp without time zone type") { import DataTypeTestUtils.numericTypes - checkInvalidCastFromNumericType(TimestampNTZType) + checkInvalidCastFromNumericTypeToTimestampNTZType() verifyCastFailure( cast(Literal(0L), TimestampNTZType), DataTypeMismatch( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastWithAnsiOnSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastWithAnsiOnSuite.scala index 5916e0501f8b6..3c554455426e8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastWithAnsiOnSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastWithAnsiOnSuite.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeConstants.MILLIS_PER_SECOND import org.apache.spark.sql.catalyst.util.DateTimeTestUtils import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{withDefaultTimeZone, UTC} import org.apache.spark.sql.errors.QueryErrorsBase +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -38,6 +39,33 @@ class CastWithAnsiOnSuite extends CastSuiteBase with QueryErrorsBase { override def evalMode: EvalMode.Value = EvalMode.ANSI + protected def checkInvalidCastFromNumericTypeToBinaryType(): Unit = { + def checkNumericTypeCast( + testValue: Any, + srcType: DataType, + to: DataType, + expectedErrorClass: String, + extraParams: Map[String, String] = Map.empty): Unit = { + val expectedError = createCastMismatch(srcType, to, expectedErrorClass, extraParams) + assert(cast(testValue, to).checkInputDataTypes() == expectedError) + } + + // Integer types: suggest config change + val configParams = Map( + "config" -> toSQLConf(SQLConf.ANSI_ENABLED.key), + "configVal" -> toSQLValue("false", StringType) + ) + checkNumericTypeCast(1.toByte, ByteType, BinaryType, "CAST_WITH_CONF_SUGGESTION", configParams) + checkNumericTypeCast( + 1.toShort, ShortType, BinaryType, "CAST_WITH_CONF_SUGGESTION", configParams) + checkNumericTypeCast(1, IntegerType, BinaryType, "CAST_WITH_CONF_SUGGESTION", configParams) + checkNumericTypeCast(1L, LongType, BinaryType, "CAST_WITH_CONF_SUGGESTION", configParams) + + // Floating types: no suggestion + checkNumericTypeCast(1.0.toFloat, FloatType, BinaryType, "CAST_WITHOUT_SUGGESTION") + checkNumericTypeCast(1.0, DoubleType, BinaryType, "CAST_WITHOUT_SUGGESTION") + } + private def isTryCast = evalMode == EvalMode.TRY private def testIntMaxAndMin(dt: DataType): Unit = { @@ -141,7 +169,7 @@ class CastWithAnsiOnSuite extends CastSuiteBase with QueryErrorsBase { test("ANSI mode: disallow type conversions between Numeric types and Date type") { import DataTypeTestUtils.numericTypes - checkInvalidCastFromNumericType(DateType) + checkInvalidCastFromNumericTypeToDateType() verifyCastFailure( cast(Literal(0L), DateType), DataTypeMismatch( @@ -167,7 +195,7 @@ class CastWithAnsiOnSuite extends CastSuiteBase with QueryErrorsBase { test("ANSI mode: disallow type conversions between Numeric types and Binary type") { import DataTypeTestUtils.numericTypes - checkInvalidCastFromNumericType(BinaryType) + checkInvalidCastFromNumericTypeToBinaryType() val binaryLiteral = Literal(new Array[Byte](1.toByte), BinaryType) numericTypes.foreach { numericType => assert(cast(binaryLiteral, numericType).checkInputDataTypes() == diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TryCastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TryCastSuite.scala index 9ead075663540..45b98c71ad705 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TryCastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TryCastSuite.scala @@ -53,6 +53,15 @@ class TryCastSuite extends CastWithAnsiOnSuite { checkEvaluation(cast(l, to), tryCastResult, InternalRow(l.value)) } + override protected def checkInvalidCastFromNumericTypeToBinaryType(): Unit = { + // All numeric types: `CAST_WITHOUT_SUGGESTION` + Seq(1.toByte, 1.toShort, 1, 1L, 1.0.toFloat, 1.0).foreach { testValue => + val expectedError = + createCastMismatch(Literal(testValue).dataType, BinaryType, "CAST_WITHOUT_SUGGESTION") + assert(cast(testValue, BinaryType).checkInputDataTypes() == expectedError) + } + } + test("print string") { assert(cast(Literal("1"), IntegerType).toString == "try_cast(1 as int)") assert(cast(Literal("1"), IntegerType).sql == "TRY_CAST('1' AS INT)") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala index cbab8894cb553..865e63405f4dc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala @@ -115,14 +115,14 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers with PlanTestB // Date is represented as Int in unsafeRow assert(DateTimeUtils.toJavaDate(unsafeRow.getInt(2)) === Date.valueOf("1970-01-01")) // Timestamp is represented as Long in unsafeRow - DateTimeUtils.toJavaTimestamp(unsafeRow.getLong(3)) should be - (Timestamp.valueOf("2015-05-08 08:10:25")) + DateTimeUtils.toJavaTimestamp(unsafeRow.getLong(3)) should be( + Timestamp.valueOf("2015-05-08 08:10:25")) unsafeRow.setInt(2, DateTimeUtils.fromJavaDate(Date.valueOf("2015-06-22"))) assert(DateTimeUtils.toJavaDate(unsafeRow.getInt(2)) === Date.valueOf("2015-06-22")) unsafeRow.setLong(3, DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf("2015-06-22 08:10:25"))) - DateTimeUtils.toJavaTimestamp(unsafeRow.getLong(3)) should be - (Timestamp.valueOf("2015-06-22 08:10:25")) + DateTimeUtils.toJavaTimestamp(unsafeRow.getLong(3)) should be( + Timestamp.valueOf("2015-06-22 08:10:25")) } testBothCodegenAndInterpreted( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferWindowGroupLimitSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferWindowGroupLimitSuite.scala index 5ffb45084184c..cfd2146d868c2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferWindowGroupLimitSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferWindowGroupLimitSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.expressions.{CurrentRow, DenseRank, Literal, NthValue, NTile, Rank, RowFrame, RowNumber, SpecifiedWindowFrame, UnboundedPreceding} +import org.apache.spark.sql.catalyst.expressions.{CurrentRow, DenseRank, Literal, NthValue, NTile, PercentRank, Rank, RowFrame, RowNumber, SpecifiedWindowFrame, UnboundedPreceding} import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor @@ -338,4 +338,20 @@ class InferWindowGroupLimitSuite extends PlanTest { WithoutOptimize.execute(correctAnswer1.analyze)) } } + + test("SPARK-46941: Can't Insert window group limit node for top-k computation if contains " + + "SizeBasedWindowFunction") { + val originalQuery = + testRelation + .select(a, b, c, + windowExpr(Rank(c :: Nil), + windowSpec(a :: Nil, c.desc :: Nil, windowFrame)).as("rank"), + windowExpr(PercentRank(c :: Nil), + windowSpec(a :: Nil, c.desc :: Nil, windowFrame)).as("percent_rank")) + .where(Symbol("rank") < 2) + + comparePlans( + Optimize.execute(originalQuery.analyze), + WithoutOptimize.execute(originalQuery.analyze)) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala index 2af3057c0b856..40b3d36d4bfc7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAggregatesSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.expressions.{Expression, PythonUDAF} +import org.apache.spark.sql.catalyst.expressions.{Expression, Literal, PythonUDAF} import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi, PlanTest} import org.apache.spark.sql.catalyst.plans.logical.{Distinct, LocalRelation, LogicalPlan} @@ -289,4 +289,23 @@ class RemoveRedundantAggregatesSuite extends PlanTest { val originalQuery = Distinct(x.groupBy($"a", $"b")($"a", TrueLiteral)).analyze comparePlans(Optimize.execute(originalQuery), originalQuery) } + + test("SPARK-53155: global lower aggregation should not be removed") { + object OptimizeNonRemovedRedundantAgg extends RuleExecutor[LogicalPlan] { + val batches = Batch("RemoveRedundantAggregates", FixedPoint(10), + PropagateEmptyRelation, + RemoveRedundantAggregates) :: Nil + } + + val query = relation + .groupBy()(Literal(1).as("col1"), Literal(2).as("col2"), Literal(3).as("col3")) + .groupBy($"col1")(max($"col1")) + .analyze + val expected = relation + .groupBy()(Literal(1).as("col1"), Literal(2).as("col2"), Literal(3).as("col3")) + .groupBy($"col1")(max($"col1")) + .analyze + val optimized = OptimizeNonRemovedRedundantAgg.execute(query) + comparePlans(optimized, expected) + } } diff --git a/sql/core/pom.xml b/sql/core/pom.xml index def8f5ddf98fb..5d678c9464593 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../pom.xml @@ -106,9 +106,17 @@ ${orc.classifier} - org.apache.hive + ${hive.group} hive-storage-api + + ${hive.group} + hive-common + + + ${hive.group} + hive-serde + org.apache.parquet parquet-column diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala index 367732dbb2059..ad42e8272718f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala @@ -166,6 +166,9 @@ class ShuffledRowRDD( } override def getPreferredLocations(partition: Partition): Seq[String] = { + if (conf.isRssEnable()) { + return Nil + } val tracker = SparkEnv.get.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] partition.asInstanceOf[ShuffledRowRDDPartition].spec match { case CoalescedPartitionSpec(startReducerIndex, endReducerIndex, _) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala index e517376bc5fc0..fe6307b5bbe86 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala @@ -530,7 +530,7 @@ case class ScalaAggregator[IN, BUF, OUT]( def eval(buffer: BUF): Any = { val row = outputSerializer(agg.finish(buffer)) - if (outputEncoder.isSerializedAsStruct) row else row.get(0, dataType) + if (outputEncoder.isSerializedAsStructForTopLevel) row else row.get(0, dataType) } private[this] lazy val bufferRow = new UnsafeRow(bufferEncoder.namedExpressions.length) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index f750a4503be16..e26e6d1b1ddc6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -217,35 +217,22 @@ case class CachedRDDBuilder( val cachedName = tableName.map(n => s"In-memory table $n") .getOrElse(StringUtils.abbreviate(cachedPlan.toString, 1024)) - def cachedColumnBuffers: RDD[CachedBatch] = { + def cachedColumnBuffers: RDD[CachedBatch] = synchronized { if (_cachedColumnBuffers == null) { - synchronized { - if (_cachedColumnBuffers == null) { - _cachedColumnBuffers = buildBuffers() - } - } + _cachedColumnBuffers = buildBuffers() } _cachedColumnBuffers } - def clearCache(blocking: Boolean = false): Unit = { + def clearCache(blocking: Boolean = false): Unit = synchronized { if (_cachedColumnBuffers != null) { - synchronized { - if (_cachedColumnBuffers != null) { - _cachedColumnBuffers.unpersist(blocking) - _cachedColumnBuffers = null - } - } + _cachedColumnBuffers.unpersist(blocking) + _cachedColumnBuffers = null } } - def isCachedColumnBuffersLoaded: Boolean = { - if (_cachedColumnBuffers != null) { - synchronized { - return _cachedColumnBuffers != null && isCachedRDDLoaded - } - } - false + def isCachedColumnBuffersLoaded: Boolean = synchronized { + _cachedColumnBuffers != null && isCachedRDDLoaded } private def isCachedRDDLoaded: Boolean = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DaysWritableV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DaysWritableV2.scala new file mode 100644 index 0000000000000..3133d7b12eb2f --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DaysWritableV2.scala @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import java.io.{DataInput, DataOutput, IOException} + +import org.apache.hadoop.hive.common.`type`.Date +import org.apache.hadoop.hive.serde2.io.DateWritableV2 +import org.apache.hadoop.io.WritableUtils + +import org.apache.spark.sql.catalyst.util.RebaseDateTime.{rebaseGregorianToJulianDays, rebaseJulianToGregorianDays} + + +/** + * The class accepts/returns days in Gregorian calendar and rebase them + * via conversion to local date in Julian calendar for dates before 1582-10-15 + * in read/write for backward compatibility with Spark 2.4 and earlier versions. + * + * @param gregorianDays The number of days since the epoch 1970-01-01 in + * Gregorian calendar. + * @param julianDays The number of days since the epoch 1970-01-01 in + * Julian calendar. + */ +class DaysWritableV2( + var gregorianDays: Int, + var julianDays: Int) + extends DateWritableV2 { + + def this() = this(0, 0) + def this(gregorianDays: Int) = + this(gregorianDays, rebaseGregorianToJulianDays(gregorianDays)) + def this(dateWritable: DateWritableV2) = { + this( + gregorianDays = dateWritable match { + case daysWritable: DaysWritableV2 => daysWritable.gregorianDays + case dateWritable: DateWritableV2 => + rebaseJulianToGregorianDays(dateWritable.getDays) + }, + julianDays = dateWritable.getDays) + } + + override def getDays: Int = julianDays + override def get: Date = { + Date.ofEpochMilli(DateWritableV2.daysToMillis(julianDays)) + } + + override def set(d: Int): Unit = { + gregorianDays = d + julianDays = rebaseGregorianToJulianDays(d) + } + + @throws[IOException] + override def write(out: DataOutput): Unit = { + WritableUtils.writeVInt(out, julianDays) + } + + @throws[IOException] + override def readFields(in: DataInput): Unit = { + julianDays = WritableUtils.readVInt(in) + gregorianDays = rebaseJulianToGregorianDays(julianDays) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala index 44d31131e9c6d..6bf8f6f9f5cae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala @@ -32,7 +32,6 @@ import org.apache.spark.sql.execution.streaming.FileStreamSink import org.apache.spark.sql.types.StructType import org.apache.spark.util.HadoopFSUtils - /** * A [[FileIndex]] that generates the list of files to process by recursively listing all the * files present in `paths`. @@ -143,7 +142,6 @@ class InMemoryFileIndex( } object InMemoryFileIndex extends Logging { - private[sql] def bulkListLeafFiles( paths: Seq[Path], hadoopConf: Configuration, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CacheTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CacheTableExec.scala index 8c14b5e370736..691629e64956a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CacheTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CacheTableExec.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.execution.datasources.v2 import java.util.Locale +import scala.util.control.NonFatal + import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.LocalTempView @@ -26,8 +28,10 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper -import org.apache.spark.sql.execution.command.CreateViewCommand +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.execution.command.{CreateViewCommand, DropTempViewCommand} import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils trait BaseCacheTableExec extends LeafV2CommandExec { def relationName: String @@ -60,7 +64,16 @@ trait BaseCacheTableExec extends LeafV2CommandExec { if (!isLazy) { // Performs eager caching. - dataFrameForCachedPlan.count() + try { + dataFrameForCachedPlan.count() + } catch { + case NonFatal(e) => + // If the query fails, we should remove the cached table. + Utils.tryLogNonFatalError { + session.sharedState.cacheManager.uncacheQuery(session, planToCache, cascade = false) + } + throw e + } } Seq.empty @@ -113,6 +126,18 @@ case class CacheTableAsSelectExec( override lazy val dataFrameForCachedPlan: DataFrame = { session.table(tempViewName) } + + override def run(): Seq[InternalRow] = { + try { + super.run() + } catch { + case NonFatal(e) => + Utils.tryLogNonFatalError { + DropTempViewCommand(Identifier.of(Array.empty, tempViewName)).run(session) + } + throw e + } + } } case class UncacheTableExec( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala index 67e77a97865df..36872f232e7ed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala @@ -60,6 +60,14 @@ class DataSourceRDD( private var currentIter: Option[Iterator[Object]] = None private var currentIndex: Int = 0 + private val partitionMetricCallback = new PartitionMetricCallback(customMetrics) + + // In case of early stopping before consuming the entire iterator, + // we need to do one more metric update at the end of the task. + context.addTaskCompletionListener[Unit] { _ => + partitionMetricCallback.execute() + } + override def hasNext: Boolean = currentIter.exists(_.hasNext) || advanceToNextIter() override def next(): Object = { @@ -86,13 +94,10 @@ class DataSourceRDD( new PartitionIterator[InternalRow](rowReader, customMetrics)) (iter, rowReader) } - context.addTaskCompletionListener[Unit] { _ => - // In case of early stopping before consuming the entire iterator, - // we need to do one more metric update at the end of the task. - CustomMetrics.updateMetrics(reader.currentMetricsValues, customMetrics) - iter.forceUpdateMetrics() - reader.close() - } + + // Once we advance to the next partition, update the metric callback for early finish + partitionMetricCallback.advancePartition(iter, reader) + currentIter = Some(iter) hasNext } @@ -107,6 +112,27 @@ class DataSourceRDD( } } +private class PartitionMetricCallback + (customMetrics: Map[String, SQLMetric]) { + private var iter: MetricsIterator[_] = null + private var reader: PartitionReader[_] = null + + def advancePartition(iter: MetricsIterator[_], reader: PartitionReader[_]): Unit = { + execute() + + this.iter = iter + this.reader = reader + } + + def execute(): Unit = { + if (iter != null && reader != null) { + CustomMetrics.updateMetrics(reader.currentMetricsValues, customMetrics) + iter.forceUpdateMetrics() + reader.close() + } + } +} + private class PartitionIterator[T]( reader: PartitionReader[T], customMetrics: Map[String, SQLMetric]) extends Iterator[T] { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala index 5774df95ac070..fcebdceb849ef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala @@ -84,6 +84,8 @@ case class ParquetPartitionReaderFactory( private val datetimeRebaseModeInRead = options.datetimeRebaseModeInRead private val int96RebaseModeInRead = options.int96RebaseModeInRead + private val parquetReaderCallback = new ParquetReaderCallback() + private def getFooter(file: PartitionedFile): ParquetMetadata = { val conf = broadcastedConf.value.value if (aggregation.isDefined || enableVectorizedReader) { @@ -304,7 +306,8 @@ case class ParquetPartitionReaderFactory( reader, readDataSchema) val iter = new RecordReaderIterator(readerWithRowIndexes) // SPARK-23457 Register a task completion listener before `initialization`. - taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) + parquetReaderCallback.advanceFile(iter) + taskContext.foreach(parquetReaderCallback.initIfNotAlready) readerWithRowIndexes } @@ -332,8 +335,39 @@ case class ParquetPartitionReaderFactory( capacity) val iter = new RecordReaderIterator(vectorizedReader) // SPARK-23457 Register a task completion listener before `initialization`. - taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) + parquetReaderCallback.advanceFile(iter) + taskContext.foreach(parquetReaderCallback.initIfNotAlready) logDebug(s"Appending $partitionSchema $partitionValues") vectorizedReader } } + +/** + * A callback class to handle the cleanup of Parquet readers. + * + * This class is used to ensure that the Parquet readers are closed properly when the task + * completes, and it also allows for the initialization of the reader callback only once per task. + */ +private class ParquetReaderCallback extends Serializable { + private var init: Boolean = false + private var iter: RecordReaderIterator[_] = null + + def initIfNotAlready(taskContext: TaskContext): Unit = { + if (!init) { + taskContext.addTaskCompletionListener[Unit](_ => closeCurrent()) + init = true + } + } + + def advanceFile(iter: RecordReaderIterator[_]): Unit = { + closeCurrent() + + this.iter = iter + } + + def closeCurrent(): Unit = { + if (iter != null) { + iter.close() + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala index 974f6f9e50c2e..97ca74aee30c0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala @@ -65,14 +65,35 @@ case class ShuffledHashJoinExec( case _ => super.outputOrdering } + private def validCondForIgnoreDupKey(cond: Expression): Boolean = { + // to ignore duplicate keys on the build side, the join condition must + // have the following properties: + // 1) a subtree that is a semantic match to a build-side key, and/or + // 2) outside any subtree that is a semantic match to a build-side key, + // all attributes should be from the stream-side. + val buildKeysSet = ExpressionSet(buildKeys) + val streamedOutputAttrs = AttributeSet(streamedOutput) + + def validCond(cond: Expression): Boolean = { + cond match { + // don't bother traversing any subtree that has a semantic match to a build key + case e: Expression if buildKeysSet.contains(e) => true + // all attributes (outside any subtree that matches a build key) should be + // from the stream side + case a: Attribute if !streamedOutputAttrs.contains(a) => false + case e: Expression => + e.children.forall(validCond(_)) + case _ => true + } + } + + validCond(cond) + } + // Exposed for testing @transient lazy val ignoreDuplicatedKey = joinType match { case LeftExistence(_) => - // For building hash relation, ignore duplicated rows with same join keys if: - // 1. Join condition is empty, or - // 2. Join condition only references streamed attributes and build join keys. - val streamedOutputAndBuildKeys = AttributeSet(streamedOutput ++ buildKeys) - condition.forall(_.references.subsetOf(streamedOutputAndBuildKeys)) + condition.forall(validCondForIgnoreDupKey(_)) case _ => false } diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/grouping_set.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/grouping_set.sql.out index bb453923ce954..e9a31fa780db8 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/grouping_set.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/grouping_set.sql.out @@ -86,7 +86,7 @@ FROM (VALUES ('x', 'a', 10), ('y', 'b', 20) ) AS t (c1, c2, c3) GROUP BY GROUPING SETS ( ( c1 ), ( c2 ) ) HAVING GROUPING__ID > 1 -- !query analysis -Filter (grouping__id#xL > cast(1 as bigint)) +Filter (GROUPING__ID#xL > cast(1 as bigint)) +- Aggregate [c1#x, c2#x, spark_grouping_id#xL], [c1#x, c2#x, sum(c3#x) AS sum(c3)#xL, spark_grouping_id#xL AS grouping__id#xL] +- Expand [[c1#x, c2#x, c3#x, c1#x, null, 1], [c1#x, c2#x, c3#x, null, c2#x, 2]], [c1#x, c2#x, c3#x, c1#x, c2#x, spark_grouping_id#xL] +- Project [c1#x, c2#x, c3#x, c1#x AS c1#x, c2#x AS c2#x] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 9815cb816c994..e54947266951b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -26,7 +26,7 @@ import scala.concurrent.duration._ import org.apache.commons.io.FileUtils -import org.apache.spark.CleanerListener +import org.apache.spark.{CleanerListener, SparkException} import org.apache.spark.executor.DataReadMethod._ import org.apache.spark.executor.DataReadMethod.DataReadMethod import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent, SparkListenerJobStart} @@ -1729,4 +1729,13 @@ class CachedTableSuite extends QueryTest with SQLTestUtils } } + + test("SPARK-52684: Atomicity of cache table on error") { + withTempView("SPARK_52684") { + intercept[SparkException] { + spark.sql("CACHE TABLE SPARK_52684 AS SELECT raise_error('SPARK-52684') AS c1") + } + assert(!spark.catalog.tableExists("SPARK_52684")) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 5a8681aed973a..6f3090d89088b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -2282,6 +2282,13 @@ class DataFrameAggregateSuite extends QueryTest checkAnswer(df, Row(1.001d, 1, 1) :: Row(6.002d, 1, 1) :: Nil) } } + + test("SPARK-53155: global lower aggregation should not be removed") { + val df = emptyTestData + .groupBy().agg(lit(1).as("col1"), lit(2).as("col2"), lit(3).as("col3")) + .groupBy($"col1").agg(max("col1")) + checkAnswer(df, Seq(Row(1, 1))) + } } case class B(c: Option[Double]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala index 47a311c71d55d..a1d5d57933864 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala @@ -1637,4 +1637,31 @@ class DataFrameWindowFunctionsSuite extends QueryTest } } } + + test("SPARK-46941: Can't insert window group limit node for top-k computation if contains " + + "SizeBasedWindowFunction") { + val df = Seq( + (1, "Dave", 1, 2020), + (2, "Mark", 2, 2020), + (3, "Amy", 3, 2020), + (4, "Dave", 1, 2021), + (5, "Mark", 2, 2021), + (6, "Amy", 3, 2021), + (7, "John", 4, 2021)).toDF("id", "name", "score", "year") + + val window = Window.partitionBy($"year").orderBy($"score".desc) + + Seq(-1, 100).foreach { threshold => + withSQLConf(SQLConf.WINDOW_GROUP_LIMIT_THRESHOLD.key -> threshold.toString) { + val df2 = df + .withColumn("rank", rank().over(window)) + .withColumn("percent_rank", percent_rank().over(window)) + .sort($"year") + checkAnswer(df2.filter("rank=2"), Seq( + Row(2, "Mark", 2, 2020, 2, 0.5), + Row(6, "Amy", 3, 2021, 2, 0.3333333333333333) + )) + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index 4d256154c8574..44c8cb92fc3fd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -1556,30 +1556,58 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan spark.range(10).map(i => (i.toString, i + 1)).toDF("c1", "c2").write.saveAsTable("t1") spark.range(10).map(i => ((i % 5).toString, i % 3)).toDF("c1", "c2").write.saveAsTable("t2") + spark.range(10).map(i => (i, i + 1)).toDF("c1", "c2").write.saveAsTable("t1a") + spark.range(10).map(i => (i % 5, i % 3)).toDF("c1", "c2").write.saveAsTable("t2a") + + val semiExpected1 = Seq(Row("0"), Row("1"), Row("2"), Row("3"), Row("4")) + val antiExpected1 = Seq(Row("5"), Row("6"), Row("7"), Row("8"), Row("9")) + val semiExpected2 = Seq(Row(0)) + val antiExpected2 = Seq.tabulate(9) { x => Row(x + 1) } + val semiJoinQueries = Seq( // No join condition, ignore duplicated key. (s"SELECT /*+ SHUFFLE_HASH(t2) */ t1.c1 FROM t1 LEFT SEMI JOIN t2 ON t1.c1 = t2.c1", - true), + true, semiExpected1, antiExpected1), // Have join condition on build join key only, ignore duplicated key. (s""" |SELECT /*+ SHUFFLE_HASH(t2) */ t1.c1 FROM t1 LEFT SEMI JOIN t2 |ON t1.c1 = t2.c1 AND CAST(t1.c2 * 2 AS STRING) != t2.c1 """.stripMargin, - true), + true, semiExpected1, antiExpected1), // Have join condition on other build attribute beside join key, do not ignore // duplicated key. (s""" |SELECT /*+ SHUFFLE_HASH(t2) */ t1.c1 FROM t1 LEFT SEMI JOIN t2 |ON t1.c1 = t2.c1 AND t1.c2 * 100 != t2.c2 """.stripMargin, - false) + false, semiExpected1, antiExpected1), + // SPARK-52873: Have a join condition that references attributes from the build-side + // join key, but those attributes are contained by a different expression than that + // used as the build-side join key (that is, CAST((t2.c2+10000)/1000 AS INT) is not + // the same as t2.c2). In this case, ignoreDuplicatedKey should be false + ( + s""" + |SELECT /*+ SHUFFLE_HASH(t2a) */ t1a.c1 FROM t1a LEFT SEMI JOIN t2a + |ON CAST((t1a.c2+10000)/1000 AS INT) = CAST((t2a.c2+10000)/1000 AS INT) + |AND t2a.c2 >= t1a.c2 + 1 + |""".stripMargin, + false, semiExpected2, antiExpected2), + // SPARK-52873: Have a join condition that contains the same expression as the + // build-side join key,and does not violate any other rules for the join condition. + // In this case, ignoreDuplicatedKey should be true + ( + s""" + |SELECT /*+ SHUFFLE_HASH(t2a) */ t1a.c1 FROM t1a LEFT SEMI JOIN t2a + |ON t1a.c1 * 10000 = t2a.c1 * 1000 AND t2a.c1 * 1000 >= t1a.c1 + |""".stripMargin, + true, semiExpected2, antiExpected2) ) semiJoinQueries.foreach { - case (query, ignoreDuplicatedKey) => + case (query, ignoreDuplicatedKey, semiExpected, antiExpected) => val semiJoinDF = sql(query) val antiJoinDF = sql(query.replaceAll("SEMI", "ANTI")) - checkAnswer(semiJoinDF, Seq(Row("0"), Row("1"), Row("2"), Row("3"), Row("4"))) - checkAnswer(antiJoinDF, Seq(Row("5"), Row("6"), Row("7"), Row("8"), Row("9"))) + checkAnswer(semiJoinDF, semiExpected) + checkAnswer(antiJoinDF, antiExpected) Seq(semiJoinDF, antiJoinDF).foreach { df => assert(collect(df.queryExecution.executedPlan) { case j: ShuffledHashJoinExec if j.ignoreDuplicatedKey == ignoreDuplicatedKey => true diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 793a0da6a8622..3cf2bfd17ab12 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -4724,6 +4724,22 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark Row(Array(0), Array(0)), Row(Array(1), Array(1)), Row(Array(2), Array(2))) checkAnswer(df, expectedAnswer) } + + test("SPARK-53094: Fix cube-related data quality problem") { + val df = sql( + """SELECT product, region, sum(amount) AS s + |FROM VALUES + | ('a', 'east', 100), + | ('b', 'east', 200), + | ('a', 'west', 150), + | ('b', 'west', 250), + | ('a', 'east', 120) AS t(product, region, amount) + |GROUP BY product, region WITH CUBE + |HAVING count(product) > 2 + |ORDER BY s DESC""".stripMargin) + + checkAnswer(df, Seq(Row(null, null, 820), Row(null, "east", 420), Row("a", null, 370))) + } } case class Foo(bar: Option[String]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala index 9ac61f0cee5fc..54403ea99c813 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala @@ -554,6 +554,30 @@ class FileIndexSuite extends SharedSparkSession { assert(FileIndexOptions.isValidOption("modifiedafter")) assert(FileIndexOptions.isValidOption("pathglobfilter")) } + + test("SPARK-52339: Correctly compare root paths") { + withTempDir { dir => + val file1 = new File(dir, "text1.txt") + stringToFile(file1, "text1") + val file2 = new File(dir, "text2.txt") + stringToFile(file2, "text2") + val path1 = new Path(file1.getCanonicalPath) + val path2 = new Path(file2.getCanonicalPath) + + val schema = StructType(Seq(StructField("a", StringType, false))) + + // Verify that the order of paths doesn't matter + val fileIndex1a = new InMemoryFileIndex(spark, Seq(path1, path2), Map.empty, Some(schema)) + val fileIndex1b = new InMemoryFileIndex(spark, Seq(path2, path1), Map.empty, Some(schema)) + assert(fileIndex1a == fileIndex1b) + + // Verify that a different number of paths does matter + val fileIndex2a = new InMemoryFileIndex(spark, Seq(path1, path1), Map.empty, Some(schema)) + val fileIndex2b = new InMemoryFileIndex(spark, Seq(path1, path1, path1), + Map.empty, Some(schema)) + assert(fileIndex2a != fileIndex2b) + } + } } object DeletionRaceFileSystem { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala index c69088589cc23..168f4f2452709 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala @@ -319,7 +319,7 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { }, AssertOnQuery { q => eventually(timeout(streamingTimeout)) { - q.lastProgress.sink.numOutputRows == 0L + assert(q.lastProgress.sink.numOutputRows == 0L) true } } diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 9cf2c20f6761a..176ad85f71237 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../pom.xml @@ -77,7 +77,7 @@ ${hive.group} - hive-jdbc + hops-jdbc ${hive.group} diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 3d85c41481dfe..a3ebd9f98c96a 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../../pom.xml @@ -110,15 +110,30 @@ ${hive.shims.scope} - org.apache.hive + ${hive.group} hive-llap-common ${hive.llap.scope} - org.apache.hive + ${hive.group} hive-llap-client ${hive.llap.scope} + + ${hive.group} + hops-jdbc + ${hive.jdbc.scope} + + + ${hive.group} + hive-service-rpc + ${hive.service.scope} + + + ${hive.group} + hive-service + ${hive.service.scope} + org.apache.avro @@ -158,7 +173,7 @@ datanucleus-core - org.apache.hadoop + ${hadoop.group} hadoop-client-runtime ${hadoop.deps.scope} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index 707532f2bad38..955ffd6b3b5b9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -23,7 +23,7 @@ import java.time.Duration import scala.collection.JavaConverters._ import org.apache.hadoop.{io => hadoopIo} -import org.apache.hadoop.hive.common.`type`.{HiveChar, HiveDecimal, HiveIntervalDayTime, HiveIntervalYearMonth, HiveVarchar} +import org.apache.hadoop.hive.common.`type`.{HiveChar, HiveDecimal, HiveIntervalDayTime, HiveIntervalYearMonth, HiveVarchar, Timestamp} import org.apache.hadoop.hive.serde2.{io => hiveIo} import org.apache.hadoop.hive.serde2.objectinspector.{StructField => HiveStructField, _} import org.apache.hadoop.hive.serde2.objectinspector.primitive._ @@ -33,7 +33,7 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.execution.datasources.DaysWritable +import org.apache.spark.sql.execution.datasources.DaysWritableV2 import org.apache.spark.sql.types import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -91,8 +91,8 @@ import org.apache.spark.unsafe.types.UTF8String * org.apache.hadoop.hive.serde2.io.ShortWritable * org.apache.hadoop.hive.serde2.io.ByteWritable * org.apache.hadoop.io.BytesWritable - * org.apache.hadoop.hive.serde2.io.DateWritable - * org.apache.hadoop.hive.serde2.io.TimestampWritable + * org.apache.hadoop.hive.serde2.io.DateWritableV2 + * org.apache.hadoop.hive.serde2.io.TimestampWritableV2 * org.apache.hadoop.hive.serde2.io.HiveDecimalWritable * Complex Type * List: Object[] / java.util.List @@ -189,8 +189,8 @@ private[hive] trait HiveInspectors { case c: Class[_] if c == classOf[hiveIo.HiveDecimalWritable] => DecimalType.SYSTEM_DEFAULT case c: Class[_] if c == classOf[hiveIo.ByteWritable] => ByteType case c: Class[_] if c == classOf[hiveIo.ShortWritable] => ShortType - case c: Class[_] if c == classOf[hiveIo.DateWritable] => DateType - case c: Class[_] if c == classOf[hiveIo.TimestampWritable] => TimestampType + case c: Class[_] if c == classOf[hiveIo.DateWritableV2] => DateType + case c: Class[_] if c == classOf[hiveIo.TimestampWritableV2] => TimestampType case c: Class[_] if c == classOf[hadoopIo.Text] => StringType case c: Class[_] if c == classOf[hadoopIo.IntWritable] => IntegerType case c: Class[_] if c == classOf[hadoopIo.LongWritable] => LongType @@ -478,7 +478,7 @@ private[hive] trait HiveInspectors { _ => constant case poi: WritableConstantTimestampObjectInspector => val t = poi.getWritableConstantValue - val constant = DateTimeUtils.fromJavaTimestamp(t.getTimestamp) + val constant = DateTimeUtils.fromJavaTimestamp(t.getTimestamp.toSqlTimestamp) _ => constant case poi: WritableConstantIntObjectInspector => val constant = poi.getWritableConstantValue.get() @@ -496,7 +496,8 @@ private[hive] trait HiveInspectors { val constant = poi.getWritableConstantValue.get() _ => constant case poi: WritableConstantShortObjectInspector => - val constant = poi.getWritableConstantValue.get() + val constant = DateTimeUtils.fromJavaDate(new java.sql.Date(poi.getWritableConstantValue + .get())) _ => constant case poi: WritableConstantByteObjectInspector => val constant = poi.getWritableConstantValue.get() @@ -507,7 +508,8 @@ private[hive] trait HiveInspectors { System.arraycopy(writable.getBytes, 0, constant, 0, constant.length) _ => constant case poi: WritableConstantDateObjectInspector => - val constant = DateTimeUtils.fromJavaDate(poi.getWritableConstantValue.get()) + val constant = DateTimeUtils.fromJavaDate(new java.sql.Date(poi.getWritableConstantValue + .get().toEpochMilli)) _ => constant case mi: StandardConstantMapObjectInspector => val keyUnwrapper = unwrapperFor(mi.getMapKeyObjectInspector) @@ -631,7 +633,7 @@ private[hive] trait HiveInspectors { case x: DateObjectInspector if x.preferWritable() => data: Any => { if (data != null) { - new DaysWritable(x.getPrimitiveWritableObject(data)).gregorianDays + new DaysWritableV2(x.getPrimitiveWritableObject(data).getDays).gregorianDays } else { null } @@ -639,7 +641,8 @@ private[hive] trait HiveInspectors { case x: DateObjectInspector => data: Any => { if (data != null) { - DateTimeUtils.fromJavaDate(x.getPrimitiveJavaObject(data)) + DateTimeUtils.fromJavaDate(new java.sql.Date(x.getPrimitiveJavaObject(data) + .toEpochMilli)) } else { null } @@ -647,7 +650,8 @@ private[hive] trait HiveInspectors { case x: TimestampObjectInspector if x.preferWritable() => data: Any => { if (data != null) { - DateTimeUtils.fromJavaTimestamp(x.getPrimitiveWritableObject(data).getTimestamp) + DateTimeUtils.fromJavaTimestamp(x.getPrimitiveWritableObject(data).getTimestamp + .toSqlTimestamp) } else { null } @@ -655,7 +659,7 @@ private[hive] trait HiveInspectors { case ti: TimestampObjectInspector => data: Any => { if (data != null) { - DateTimeUtils.fromJavaTimestamp(ti.getPrimitiveJavaObject(data)) + DateTimeUtils.fromJavaTimestamp(ti.getPrimitiveJavaObject(data).toSqlTimestamp) } else { null } @@ -1079,18 +1083,19 @@ private[hive] trait HiveInspectors { new hadoopIo.BytesWritable(value.asInstanceOf[Array[Byte]]) } - private def getDateWritable(value: Any): DaysWritable = + private def getDateWritable(value: Any): DaysWritableV2 = if (value == null) { null } else { - new DaysWritable(value.asInstanceOf[Int]) + new DaysWritableV2(value.asInstanceOf[Int]) } - private def getTimestampWritable(value: Any): hiveIo.TimestampWritable = + private def getTimestampWritable(value: Any): hiveIo.TimestampWritableV2 = if (value == null) { null } else { - new hiveIo.TimestampWritable(DateTimeUtils.toJavaTimestamp(value.asInstanceOf[Long])) + val ts = DateTimeUtils.toJavaTimestamp(value.asInstanceOf[Long]) + new hiveIo.TimestampWritableV2(Timestamp.ofEpochMilli(ts.getTime)) } private def getHiveIntervalDayTimeWritable(value: Any): hiveIo.HiveIntervalDayTimeWritable = diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index cd1d236dd36c9..e3c93cbb692b3 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -522,10 +522,12 @@ private[hive] object HadoopTableReader extends HiveInspectors with Logging { row.update(ordinal, HiveShim.toCatalystDecimal(oi, value)) case oi: TimestampObjectInspector => (value: Any, row: InternalRow, ordinal: Int) => - row.setLong(ordinal, DateTimeUtils.fromJavaTimestamp(oi.getPrimitiveJavaObject(value))) + row.setLong(ordinal, DateTimeUtils.fromJavaTimestamp(oi.getPrimitiveJavaObject(value) + .toSqlTimestamp)) case oi: DateObjectInspector => (value: Any, row: InternalRow, ordinal: Int) => - row.setInt(ordinal, DateTimeUtils.fromJavaDate(oi.getPrimitiveJavaObject(value))) + row.setInt(ordinal, DateTimeUtils.fromJavaDate(new java.sql.Date(oi + .getPrimitiveJavaObject(value).toEpochMilli))) case oi: BinaryObjectInspector => (value: Any, row: InternalRow, ordinal: Int) => row.update(ordinal, oi.getPrimitiveJavaObject(value)) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala index 7dbabcae484f1..226c722721e0d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive.client -import java.io.PrintStream +import org.apache.hadoop.hive.common.io.SessionStream import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog._ @@ -59,9 +59,9 @@ private[hive] trait HiveClient { */ def runSqlHive(sql: String): Seq[String] - def setOut(stream: PrintStream): Unit - def setInfo(stream: PrintStream): Unit - def setError(stream: PrintStream): Unit + def setOut(stream: SessionStream): Unit + def setInfo(stream: SessionStream): Unit + def setError(stream: SessionStream): Unit /** Returns the names of all tables in the given database. */ def listTables(dbName: String): Seq[String] diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index e51658355b10d..d6a28c1f81c41 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.hive.client -import java.io.PrintStream import java.lang.{Iterable => JIterable} import java.lang.reflect.InvocationTargetException import java.nio.charset.StandardCharsets.UTF_8 @@ -31,6 +30,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.common.StatsSetupConst +import org.apache.hadoop.hive.common.io.SessionStream import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hadoop.hive.metastore.{IMetaStoreClient, TableType => HiveTableType} @@ -183,8 +183,8 @@ private[hive] class HiveClientImpl( // got changed. We reset it to clientLoader.ClassLoader here. state.getConf.setClassLoader(clientLoader.classLoader) shim.setCurrentSessionState(state) - state.out = new PrintStream(outputBuffer, true, UTF_8.name()) - state.err = new PrintStream(outputBuffer, true, UTF_8.name()) + state.out = new SessionStream(outputBuffer, true, UTF_8.name()) + state.err = new SessionStream(outputBuffer, true, UTF_8.name()) state } @@ -312,15 +312,15 @@ private[hive] class HiveClientImpl( ret } - def setOut(stream: PrintStream): Unit = withHiveState { + def setOut(stream: SessionStream): Unit = withHiveState { state.out = stream } - def setInfo(stream: PrintStream): Unit = withHiveState { + def setInfo(stream: SessionStream): Unit = withHiveState { state.info = stream } - def setError(stream: PrintStream): Unit = withHiveState { + def setError(stream: SessionStream): Unit = withHiveState { state.err = stream } @@ -866,14 +866,16 @@ private[hive] class HiveClientImpl( def closeDriver(driver: Driver): Unit = { // Since HIVE-18238(Hive 3.0.0), the Driver.close function's return type changed // and the CommandProcessorFactory.clean function removed. - driver.getClass.getMethod("close").invoke(driver) - if (version != hive.v3_0 && version != hive.v3_1) { - CommandProcessorFactory.clean(conf) - } + // Fabio: Comment this to avoid compilation issue with Hive3 + // if (version != hive.v3_0 && version != hive.v3_1) { + // CommandProcessorFactory.clean(conf) + // } } // Hive query needs to start SessionState. SessionState.start(state) + state.out = new SessionStream(outputBuffer, true, UTF_8.name()) + state.err = new SessionStream(outputBuffer, true, UTF_8.name()) logDebug(s"Running hiveql '$cmd'") if (cmd.toLowerCase(Locale.ROOT).startsWith("set")) { logDebug(s"Changing config: $cmd") } try { @@ -1029,6 +1031,8 @@ private[hive] class HiveClientImpl( others.foreach { table => val t = table.getTableName logDebug(s"Deleting table $t") + shim.dropTable(client, "default", t) + /* try { shim.getIndexes(client, "default", t, 255).foreach { index => shim.dropIndex(client, "default", t, index.getIndexName) @@ -1041,6 +1045,7 @@ private[hive] class HiveClientImpl( // HIVE-18448 Hive 3.0 remove index APIs shim.dropTable(client, "default", t) } + */ } shim.getAllDatabases(client).filterNot(_ == "default").foreach { db => logDebug(s"Dropping Database: $db") @@ -1350,7 +1355,7 @@ private[hive] object HiveClientImpl extends Logging { new HiveConf(conf, classOf[HiveConf]) } try { - Hive.getWithoutRegisterFns(hiveConf) + Hive.getWithFastCheck(hiveConf, false) } catch { // SPARK-37069: not all Hive versions have the above method (e.g., Hive 2.3.9 has it but // 2.3.8 don't), therefore here we fallback when encountering the exception. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 01c3fc724822c..6f797d36c5f79 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -30,7 +30,7 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.IMetaStoreClient import org.apache.hadoop.hive.metastore.TableType -import org.apache.hadoop.hive.metastore.api.{Database, EnvironmentContext, Function => HiveFunction, FunctionType, Index, MetaException, PrincipalType, ResourceType, ResourceUri} +import org.apache.hadoop.hive.metastore.api.{Database, EnvironmentContext, Function => HiveFunction, FunctionType, MetaException, PrincipalType, ResourceType, ResourceUri} import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.io.AcidUtils import org.apache.hadoop.hive.ql.metadata.{Hive, HiveException, Partition, Table} @@ -243,7 +243,7 @@ private[client] sealed abstract class Shim { def getMSC(hive: Hive): IMetaStoreClient - def getIndexes(hive: Hive, dbName: String, tableName: String, max: Short): Seq[Index] + // def getIndexes(hive: Hive, dbName: String, tableName: String, max: Short): Seq[Index] protected def findMethod(klass: Class[_], name: String, args: Class[_]*): Method = { klass.getMethod(name, args: _*) @@ -688,6 +688,7 @@ private[client] class Shim_v0_12 extends Shim with Logging { hive.renamePartition(table, oldPartSpec, newPart) } + /* override def getIndexes( hive: Hive, dbName: String, @@ -696,6 +697,7 @@ private[client] class Shim_v0_12 extends Shim with Logging { recordHiveCall() hive.getIndexes(dbName, tableName, max).asScala.toSeq } + */ } private[client] class Shim_v0_13 extends Shim_v0_12 { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala index 18090b53e3c10..f5cc72860ff7c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala @@ -89,23 +89,28 @@ private[hive] object IsolatedClientLoader extends Logging { } def hiveVersion(version: String): HiveVersion = { - VersionUtils.majorMinorPatchVersion(version).flatMap { - case (12, _, _) | (0, 12, _) => Some(hive.v12) - case (13, _, _) | (0, 13, _) => Some(hive.v13) - case (14, _, _) | (0, 14, _) => Some(hive.v14) - case (1, 0, _) => Some(hive.v1_0) - case (1, 1, _) => Some(hive.v1_1) - case (1, 2, _) => Some(hive.v1_2) - case (2, 0, _) => Some(hive.v2_0) - case (2, 1, _) => Some(hive.v2_1) - case (2, 2, _) => Some(hive.v2_2) - case (2, 3, _) => Some(hive.v2_3) - case (3, 0, _) => Some(hive.v3_0) - case (3, 1, _) => Some(hive.v3_1) - case _ => None - }.getOrElse { - throw QueryExecutionErrors.unsupportedHiveMetastoreVersionError( - version, HiveUtils.HIVE_METASTORE_VERSION.key) + def extractMajorMinorVersion(version: String): String = { + val parts = version.split("\\.") + if (parts.length >= 2) parts(0) + "." + parts(1) else parts(0) + } + + val majorMinorVersion = extractMajorMinorVersion(version) + majorMinorVersion match { + case "0.12" => hive.v12 + case "0.13" => hive.v13 + case "0.14" => hive.v14 + case "1.0" => hive.v1_0 + case "1.1" => hive.v1_1 + case "1.2" => hive.v1_2 + case "2.0" => hive.v2_0 + case "2.1" => hive.v2_1 + case "2.2" => hive.v2_2 + case "2.3" => hive.v2_3 + case "3.0" => hive.v3_0 + case "3.1" => hive.v3_1 + case _ => throw new UnsupportedOperationException(s"Unsupported " + + s"Hive Metastore version ($version). Please set " + + s"${HiveUtils.HIVE_METASTORE_VERSION.key} with a valid version.") } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala index 29734c4de3441..9ee3089977eba 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala @@ -86,7 +86,7 @@ class HiveFileFormat(fileSinkConf: FileSinkDesc) // Add table properties from storage handler to hadoopConf, so any custom storage // handler settings can be set to hadoopConf HiveTableUtil.configureJobPropertiesForStorageHandler(tableDesc, conf, false) - Utilities.copyTableJobPropertiesToConf(tableDesc, conf) + Utilities.copyTableJobPropertiesToConf(tableDesc, new JobConf(conf)) // Avoid referencing the outer object. val fileSinkConfSer = fileSinkConf diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/UDAQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/UDAQuerySuite.scala index 0bd6b1403d39c..31d0452c70617 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/UDAQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/UDAQuerySuite.scala @@ -60,6 +60,22 @@ object LongProductSumAgg extends Aggregator[(jlLong, jlLong), Long, jlLong] { def outputEncoder: Encoder[jlLong] = Encoders.LONG } +final case class Reduce[T: Encoder](r: (T, T) => T)(implicit i: Encoder[Option[T]]) + extends Aggregator[T, Option[T], T] { + def zero: Option[T] = None + def reduce(b: Option[T], a: T): Option[T] = Some(b.fold(a)(r(_, a))) + def merge(b1: Option[T], b2: Option[T]): Option[T] = + (b1, b2) match { + case (Some(a), Some(b)) => Some(r(a, b)) + case (Some(a), None) => Some(a) + case (None, Some(b)) => Some(b) + case (None, None) => None + } + def finish(reduction: Option[T]): T = reduction.get + def bufferEncoder: Encoder[Option[T]] = implicitly + def outputEncoder: Encoder[T] = implicitly +} + @SQLUserDefinedType(udt = classOf[CountSerDeUDT]) case class CountSerDeSQL(nSer: Int, nDeSer: Int, sum: Int) @@ -180,6 +196,9 @@ abstract class UDAQuerySuite extends QueryTest with SQLTestUtils with TestHiveSi val data4 = Seq[Boolean](true, false, true).toDF("boolvalues") data4.write.saveAsTable("agg4") + val data5 = Seq[(Int, (Int, Int))]((1, (2, 3))).toDF("key", "value") + data5.write.saveAsTable("agg5") + val emptyDF = spark.createDataFrame( sparkContext.emptyRDD[Row], StructType(StructField("key", StringType) :: StructField("value", IntegerType) :: Nil)) @@ -190,6 +209,9 @@ abstract class UDAQuerySuite extends QueryTest with SQLTestUtils with TestHiveSi spark.udf.register("mydoubleavg", udaf(MyDoubleAvgAgg)) spark.udf.register("longProductSum", udaf(LongProductSumAgg)) spark.udf.register("arraysum", udaf(ArrayDataAgg)) + spark.udf.register("reduceOptionPair", udaf(Reduce[Option[(Int, Int)]]( + (opt1, opt2) => + opt1.zip(opt2).map { case ((a1, b1), (a2, b2)) => (a1 + a2, b1 + b2) }.headOption))) } override def afterAll(): Unit = { @@ -371,6 +393,12 @@ abstract class UDAQuerySuite extends QueryTest with SQLTestUtils with TestHiveSi Row(Seq(12.0, 15.0, 18.0)) :: Nil) } + test("SPARK-52023: Returning Option[Product] from udaf") { + checkAnswer( + spark.sql("SELECT reduceOptionPair(value) FROM agg5 GROUP BY key"), + Row(Row(2, 3)) :: Nil) + } + test("verify aggregator ser/de behavior") { val data = sparkContext.parallelize((1 to 100).toSeq, 3).toDF("value1") val agg = udaf(CountSerDeAgg) diff --git a/streaming/pom.xml b/streaming/pom.xml index a8e82cb10d377..92d716fa5c09d 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index b864ede1149ec..f2e171b2dfe97 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.5.8-SNAPSHOT + 3.5.5 ../pom.xml