Skip to content
Merged
2 changes: 1 addition & 1 deletion .github/workflows/velox_backend_x86.yml
Original file line number Diff line number Diff line change
Expand Up @@ -1432,7 +1432,7 @@ jobs:
export PATH=$JAVA_HOME/bin:$PATH
java -version
$MVN_CMD clean test -Pspark-4.1 -Pscala-2.13 -Pjava-17 -Pbackends-velox \
-Pspark-ut -DargLine="-Dspark.test.home=/opt/shims/spark41/spark_home/" \
-Pspark-ut -Pdelta -DargLine="-Dspark.test.home=/opt/shims/spark41/spark_home/" \
-DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.spark.tags.SlowHiveTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.EnhancedFeaturesTest,org.apache.gluten.tags.SkipTest
- name: Upload test report
if: always()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
*/
package org.apache.spark.sql.delta

import org.apache.spark.internal.{LoggingShims, MDC}
import org.apache.spark.internal.{Logging, MDC => SparkMDC}
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.delta.GlutenDeltaParquetFileFormat._
Expand Down Expand Up @@ -62,7 +62,7 @@ case class GlutenDeltaParquetFileFormat(
tablePath: Option[String] = None,
isCDCRead: Boolean = false)
extends GlutenParquetFileFormat
with LoggingShims {
with Logging {
// Validate either we have all arguments for DV enabled read or none of them.
if (hasTablePath) {
SparkSession.getActiveSession.map { session =>
Expand Down Expand Up @@ -528,7 +528,7 @@ case class GlutenDeltaParquetFileFormat(
case AlwaysTrue() => Some(AlwaysTrue())
case AlwaysFalse() => Some(AlwaysFalse())
case _ =>
logError(log"Failed to translate filter ${MDC(DeltaLogKeys.FILTER, filter)}")
logError(log"Failed to translate filter ${SparkMDC.of(DeltaLogKeys.FILTER, filter)}")
None
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import org.apache.gluten.execution.datasource.GlutenFormatFactory
import org.apache.gluten.extension.columnar.transition.{Convention, Transitions}

import org.apache.spark._
import org.apache.spark.internal.{LoggingShims, MDC}
import org.apache.spark.internal.{Logging, MDC => SparkMDC}
import org.apache.spark.internal.io.{FileCommitProtocol, SparkHadoopWriterUtils}
import org.apache.spark.shuffle.FetchFailedException
import org.apache.spark.sql.SparkSession
Expand Down Expand Up @@ -63,7 +63,7 @@ import java.util.{Date, UUID}
* values to data files. Specifically L123-126, L132, and L140 where it adds option
* WRITE_PARTITION_COLUMNS
*/
object GlutenDeltaFileFormatWriter extends LoggingShims {
object GlutenDeltaFileFormatWriter extends Logging {

/**
* A variable used in tests to check whether the output ordering of the query matches the
Expand Down Expand Up @@ -343,20 +343,20 @@ object GlutenDeltaFileFormatWriter extends LoggingShims {
val ret = f
val commitMsgs = ret.map(_.commitMsg)

logInfo(log"Start to commit write Job ${MDC(DeltaLogKeys.JOB_ID, description.uuid)}.")
logInfo(log"Start to commit write Job ${SparkMDC.of(DeltaLogKeys.JOB_ID, description.uuid)}.")
val (_, duration) = Utils.timeTakenMs { committer.commitJob(job, commitMsgs) }
logInfo(log"Write Job ${MDC(DeltaLogKeys.JOB_ID, description.uuid)} committed. " +
log"Elapsed time: ${MDC(DeltaLogKeys.DURATION, duration)} ms.")
logInfo(log"Write Job ${SparkMDC.of(DeltaLogKeys.JOB_ID, description.uuid)} committed. " +
log"Elapsed time: ${SparkMDC.of(DeltaLogKeys.DURATION, duration)} ms.")

processStats(description.statsTrackers, ret.map(_.summary.stats), duration)
logInfo(log"Finished processing stats for write job " +
log"${MDC(DeltaLogKeys.JOB_ID, description.uuid)}.")
log"${SparkMDC.of(DeltaLogKeys.JOB_ID, description.uuid)}.")

// return a set of all the partition paths that were updated during this job
ret.map(_.summary.updatedPartitions).reduceOption(_ ++ _).getOrElse(Set.empty)
} catch {
case cause: Throwable =>
logError(log"Aborting job ${MDC(DeltaLogKeys.JOB_ID, description.uuid)}", cause)
logError(log"Aborting job ${SparkMDC.of(DeltaLogKeys.JOB_ID, description.uuid)}", cause)
committer.abortJob(job)
throw cause
}
Expand Down Expand Up @@ -490,7 +490,7 @@ object GlutenDeltaFileFormatWriter extends LoggingShims {
})(catchBlock = {
// If there is an error, abort the task
dataWriter.abort()
logError(log"Job ${MDC(DeltaLogKeys.JOB_ID, jobId)} aborted.")
logError(log"Job ${SparkMDC.of(DeltaLogKeys.JOB_ID, jobId)} aborted.")
}, finallyBlock = {
dataWriter.close()
})
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,19 @@
package org.apache.spark.sql.delta

object DeltaInsertIntoTableSuiteShims {
val INSERT_INTO_TMP_VIEW_ERROR_MSG = "[EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE]"
private val isSpark41 = org.apache.spark.SPARK_VERSION.startsWith("4.1")

// Spark 4.0.1 reports non-constant defaults with NOT_CONSTANT.
val INVALID_COLUMN_DEFAULT_VALUE_ERROR_MSG = "INVALID_DEFAULT_VALUE.NOT_CONSTANT"
val INSERT_INTO_TMP_VIEW_ERROR_MSG =
if (isSpark41) {
"[TABLE_OR_VIEW_NOT_FOUND]"
} else {
"[EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE]"
}

val INVALID_COLUMN_DEFAULT_VALUE_ERROR_MSG =
if (isSpark41) {
"INVALID_DEFAULT_VALUE.UNRESOLVED_EXPRESSION"
} else {
"INVALID_DEFAULT_VALUE.NOT_CONSTANT"
}
}
4 changes: 2 additions & 2 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -1351,8 +1351,8 @@
<sparkshim.artifactId>spark-sql-columnar-shims-spark41</sparkshim.artifactId>
<spark.version>4.1.1</spark.version>
<iceberg.version>1.10.0</iceberg.version>
<delta.package.name>delta-spark</delta.package.name>
<delta.version>4.0.0</delta.version>
<delta.package.name>delta-spark_4.1</delta.package.name>
<delta.version>4.1.0</delta.version>
<delta.binary.version>40</delta.binary.version>
<hudi.version>1.1.0</hudi.version>
<fasterxml.version>2.18.2</fasterxml.version>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.sql.execution.streaming
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it seems this class is introduced in this patch, is this a must?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Rechecked this and agreed it should not live in the Spark 4.1 shim jar. I moved the compatibility wrapper into backends-velox/src-delta40-spark41/test, wired that test source root only under the Spark 4.1 profile, and deleted the production shim source. Verified locally with ./build/mvn -ntp -pl backends-velox -am clean test-compile -Pspark-4.1 -Pscala-2.13 -Pjava-17 -Pbackends-velox -Pspark-ut -Pdelta -DskipTests -Dmaven.source.skip.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Update after the later discussion: I moved this back into the Spark 4.1 shim layer and removed the backends-velox/src-delta40-spark41 test source root. This follows the same pattern as ShuffleUtils: expose the old Spark API package from the Spark 4.1 shim while delegating to vanilla Spark 4.1 execution.streaming.runtime.MemoryStream. The class is only an API compatibility bridge for shared sources across Spark versions.


import org.apache.spark.sql.{Encoder, SQLContext}
import org.apache.spark.sql.execution.streaming.runtime.{MemoryStream => RuntimeMemoryStream}

object MemoryStream {
def apply[A: Encoder](implicit sqlContext: SQLContext): RuntimeMemoryStream[A] = {
RuntimeMemoryStream[A](implicitly[Encoder[A]], sqlContext.sparkSession)
}

def apply[A: Encoder](
numPartitions: Int)(
implicit sqlContext: SQLContext): RuntimeMemoryStream[A] = {
RuntimeMemoryStream[A](numPartitions)(implicitly[Encoder[A]], sqlContext.sparkSession)
}
}
Loading