Skip to content

Commit 2da838b

Browse files
LuciferYangdongjoon-hyun
authored andcommitted
[SPARK-52612][INFRA] Add an env NO_PROVIDED_SPARK_JARS to control collection behavior of sbt/package for spark-avro.jar and spark-protobuf.jar
### What changes were proposed in this pull request? This pr introduces an environment variable named `NO_PROVIDED_SPARK_JARS`, which controls the behavior of the `sbt/package` command so that it only collects `spark-avro.jar` and `spark-protobuf.jar` into the `assembly/target/scala-2.13/jars` directory during documentation generation. ### Why are the changes needed? 1. To ensure that, by default, the `sbt/package` command does not collect jars with a `provided` scope, such as `spark-avro.jar` and `spark-protobuf.jar`, into the `assembly/target/scala-2.13/jars` directory, maintaining consistency with Maven's behavior. 2. To ensure that, during documentation generation, the `sbt/package` command collects the necessary jars into the `assembly/target/scala-2.13/jars` directory to ensure that no dependencies are missing for the documentation generation task. 3. To avoid the following error when executing benchmark tasks using GitHub Actions: ``` 25/06/28 07:03:45 ERROR SparkContext: Failed to add file:///home/runner/work/spark/spark/assembly/target/scala-2.13/jars/spark-avro_2.13-4.1.0-SNAPSHOT.jar to Spark environment java.lang.IllegalArgumentException: requirement failed: File spark-avro_2.13-4.1.0-SNAPSHOT.jar was already registered with a different path (old path = /home/runner/work/spark/spark/connector/avro/target/scala-2.13/spark-avro_2.13-4.1.0-SNAPSHOT.jar, new path = /home/runner/work/spark/spark/assembly/target/scala-2.13/jars/spark-avro_2.13-4.1.0-SNAPSHOT.jar ... ``` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - Passed GitHub Actions. - Manually confirmed that benchmark tasks are not affected and that the ERROR log described above no longer appears during benchmark task execution. ### Was this patch authored or co-authored using generative AI tooling? No Closes #51321 from LuciferYang/SPARK-52612. Authored-by: yangjie01 <[email protected]> Signed-off-by: Dongjoon Hyun <[email protected]> (cherry picked from commit 591e1c3) Signed-off-by: Dongjoon Hyun <[email protected]>
1 parent 37a4f1c commit 2da838b

File tree

2 files changed

+9
-2
lines changed

2 files changed

+9
-2
lines changed

docs/_plugins/build_api_docs.rb

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -45,7 +45,7 @@ def build_spark_if_necessary
4545

4646
print_header "Building Spark."
4747
cd(SPARK_PROJECT_ROOT)
48-
command = "build/sbt -Phive -Pkinesis-asl clean package"
48+
command = "NO_PROVIDED_SPARK_JARS=0 build/sbt -Phive -Pkinesis-asl clean package"
4949
puts "Running '#{command}'; this may take a few minutes..."
5050
system(command) || raise("Failed to build Spark")
5151
$spark_package_is_built = true

project/SparkBuild.scala

Lines changed: 8 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1517,6 +1517,9 @@ object CopyDependencies {
15171517
val fid = (LocalProject("connect") / assembly).value
15181518
val fidClient = (LocalProject("connect-client-jvm") / assembly).value
15191519
val fidProtobuf = (LocalProject("protobuf") / assembly).value
1520+
val noProvidedSparkJars: Boolean = sys.env.getOrElse("NO_PROVIDED_SPARK_JARS", "1") == "1" ||
1521+
sys.env.getOrElse("NO_PROVIDED_SPARK_JARS", "true")
1522+
.toLowerCase(Locale.getDefault()) == "true"
15201523

15211524
(Compile / dependencyClasspath).value.map(_.data)
15221525
.filter { jar => jar.isFile() }
@@ -1532,12 +1535,16 @@ object CopyDependencies {
15321535
// Don't copy the spark connect common JAR as it is shaded in the spark connect.
15331536
} else if (jar.getName.contains("connect-client-jvm")) {
15341537
// Do not place Spark Connect client jars as it is not built-in.
1538+
} else if (noProvidedSparkJars && jar.getName.contains("spark-avro")) {
1539+
// Do not place Spark Avro jars as it is not built-in.
15351540
} else if (jar.getName.contains("spark-connect") &&
15361541
!SbtPomKeys.profiles.value.contains("noshade-connect")) {
15371542
Files.copy(fid.toPath, destJar.toPath)
15381543
} else if (jar.getName.contains("spark-protobuf") &&
15391544
!SbtPomKeys.profiles.value.contains("noshade-protobuf")) {
1540-
Files.copy(fidProtobuf.toPath, destJar.toPath)
1545+
if (!noProvidedSparkJars) {
1546+
Files.copy(fidProtobuf.toPath, destJar.toPath)
1547+
}
15411548
} else {
15421549
Files.copy(jar.toPath(), destJar.toPath())
15431550
}

0 commit comments

Comments
 (0)