Skip to content

Commit 491add8

Browse files
committed
[SPARK-56364][BUILD][TESTS] Generate Scala-based test JARs dynamically instead of storing pre-built binaries
### What changes were proposed in this pull request? This PR is a part of SPARK-56352 for Scala-based test JARs, replacing pre-built test JAR files containing Scala classes with dynamic compilation at test time, removing 6 binary JAR files and 1 binary from the repository. Changes: - Add `TestUtils.createJarWithScalaSources()` in `SparkTestUtils.scala` that compiles Scala source files via `scala.tools.nsc.Main` and packages the resulting classes into a JAR, with support for excluding specific classes by prefix. - Update test suites to use dynamically generated JARs instead of pre-built ones. - Refactor `StubClassLoaderSuite` to use a self-contained dummy class instead of the pre-built `udf_noA.jar` that contained spark-connect classes, eliminating the cross-module dependency noted in the original TODO comment. - Extract `StubClassDummyUdfPacker` from `StubClassDummyUdf.scala` into a separate file for use by `UDFClassLoadingE2ESuite`. - Remove deleted JAR/binary entries from `dev/test-jars.txt`. JARs/binaries removed: - `core/src/test/resources/TestHelloV2_2.13.jar` - `core/src/test/resources/TestHelloV3_2.13.jar` - `sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar` - `sql/connect/client/jvm/src/test/resources/udf2.13.jar` - `sql/connect/client/jvm/src/test/resources/udf2.13` (serialized binary) - `sql/core/src/test/resources/artifact-tests/udf_noA.jar` - `sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar` ### Why are the changes needed? As noted in the PR discussion (#50378): > the ultimate goal is to refactor the tests to automatically build the jars instead of using pre-built ones This PR achieves that goal for all Scala-based test JARs. By generating JARs dynamically at test time, no binary artifacts need to be stored in the source tree, and the release-time workaround becomes unnecessary for these files. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? All affected test suites pass: - ClassLoaderIsolationSuite (core) - StubClassLoaderSuite, ArtifactManagerSuite (sql/core) - HiveSparkSubmitSuite (sql/hive) - ReplE2ESuite (sql/connect) - UDFClassLoadingE2ESuite (sql/connect) ### Was this patch authored or co-authored using generative AI tooling? Kiro CLI / Opus 4.6 Closes #55218 from sarutak/remove-test-jars-c. Authored-by: Kousuke Saruta <sarutak@amazon.co.jp> Signed-off-by: Kousuke Saruta <sarutak@apache.org>
1 parent af9c8b3 commit 491add8

File tree

19 files changed

+234
-126
lines changed

19 files changed

+234
-126
lines changed

common/utils/src/main/scala/org/apache/spark/util/SparkTestUtils.scala

Lines changed: 50 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -151,6 +151,56 @@ private[spark] trait SparkTestUtils {
151151
jarFile.toURI.toURL
152152
}
153153

154+
/**
155+
* Compile Scala source files and package the resulting class files
156+
* into a JAR.
157+
*
158+
* @param sourceFiles Scala source files to compile
159+
* @param jarFile the JAR file to create
160+
* @param classpathUrls additional classpath URLs needed for compilation
161+
* @param excludeClassPrefixes class name prefixes to exclude from the
162+
* JAR (e.g., Seq("A") excludes A.class, A$.class)
163+
*/
164+
def createJarWithScalaSources(
165+
sourceFiles: Seq[File],
166+
jarFile: File,
167+
classpathUrls: Seq[URL] = Seq.empty,
168+
excludeClassPrefixes: Seq[String] = Seq.empty): URL = {
169+
val classDir = Files.createTempDirectory("spark-test-scala-classes").toFile
170+
val compilerClass = SparkClassUtils.classForName[AnyRef]("scala.tools.nsc.Main")
171+
val processMethod = compilerClass.getMethod("process", classOf[Array[String]])
172+
173+
val cpStr = classpathUrls.map(_.getFile).mkString(File.pathSeparator)
174+
val args = Array("-classpath", cpStr, "-d", classDir.getAbsolutePath) ++
175+
sourceFiles.map(_.getAbsolutePath)
176+
177+
val success = processMethod.invoke(null, args).asInstanceOf[Boolean]
178+
assert(success, s"Scala compilation failed for: ${sourceFiles.map(_.getName).mkString(", ")}")
179+
180+
try {
181+
val classFiles = listFilesRecursively(classDir).filter { f =>
182+
f.getName.endsWith(".class") && !excludeClassPrefixes.exists(p =>
183+
f.getName == s"$p.class" || f.getName.startsWith(s"$p$$"))
184+
}
185+
186+
jarFile.getParentFile.mkdirs()
187+
val jarStream = new JarOutputStream(new FileOutputStream(jarFile))
188+
try {
189+
for (classFile <- classFiles) {
190+
val entryName = classDir.toPath.relativize(classFile.toPath).toString.replace('\\', '/')
191+
jarStream.putNextEntry(new JarEntry(entryName))
192+
val in = new FileInputStream(classFile)
193+
try { in.transferTo(jarStream) } finally { in.close() }
194+
}
195+
} finally {
196+
jarStream.close()
197+
}
198+
} finally {
199+
SparkFileUtils.deleteRecursively(classDir)
200+
}
201+
202+
jarFile.toURI.toURL
203+
}
154204

155205
private def listFilesRecursively(dir: File): Seq[File] = {
156206
val children = dir.listFiles()

core/pom.xml

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -39,6 +39,11 @@
3939
<groupId>org.scala-lang.modules</groupId>
4040
<artifactId>scala-parallel-collections_${scala.binary.version}</artifactId>
4141
</dependency>
42+
<dependency>
43+
<groupId>org.scala-lang</groupId>
44+
<artifactId>scala-compiler</artifactId>
45+
<scope>test</scope>
46+
</dependency>
4247
<dependency>
4348
<groupId>org.apache.avro</groupId>
4449
<artifactId>avro</artifactId>
-4.02 KB
Binary file not shown.
-3.74 KB
Binary file not shown.

core/src/test/scala/org/apache/spark/executor/ClassLoaderIsolationSuite.scala

Lines changed: 19 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -17,37 +17,38 @@
1717

1818
package org.apache.spark.executor
1919

20-
import java.io.File
20+
import java.io.{File, PrintWriter}
2121
import java.net.URL
2222

23-
import scala.util.Properties
24-
2523
import org.apache.spark.{JobArtifactSet, JobArtifactState, LocalSparkContext, SparkConf, SparkContext, SparkFunSuite, TestUtils}
2624
import org.apache.spark.util.{MutableURLClassLoader, Utils}
2725

2826

2927
class ClassLoaderIsolationSuite extends SparkFunSuite with LocalSparkContext {
3028

31-
private val scalaVersion = Properties.versionNumberString
32-
.split("\\.")
33-
.take(2)
34-
.mkString(".")
35-
3629
private val jarURL1 = TestUtils.createJarWithClasses(Seq("ClassLoaderIsolation_Dummy"))
3730
private lazy val jar1 = jarURL1.toString
3831

39-
// package com.example
40-
// object Hello { def test(): Int = 2 }
41-
// case class Hello(x: Int, y: Int)
42-
private val jarURL2 = Thread.currentThread().getContextClassLoader
43-
.getResource(s"TestHelloV2_$scalaVersion.jar")
32+
// Dynamically compile Scala source to generate JARs containing
33+
// `object Hello { def test(): Int = N }` in package com.example.
34+
private def createHelloJar(returnValue: Int): URL = {
35+
val source =
36+
s"""package com.example
37+
|object Hello { def test(): Int = $returnValue }
38+
|""".stripMargin
39+
val srcFile = File.createTempFile("Hello", ".scala", Utils.createTempDir())
40+
val pw = new PrintWriter(srcFile)
41+
try { pw.write(source) } finally { pw.close() }
42+
val jarFile = File.createTempFile(s"TestHelloV$returnValue", ".jar", Utils.createTempDir())
43+
val cp = System.getProperty("java.class.path")
44+
.split(File.pathSeparator).map(p => new File(p).toURI.toURL).toSeq
45+
TestUtils.createJarWithScalaSources(Seq(srcFile), jarFile, cp)
46+
}
47+
48+
private val jarURL2 = createHelloJar(2)
4449
private lazy val jar2 = jarURL2.toString
4550

46-
// package com.example
47-
// object Hello { def test(): Int = 3 }
48-
// case class Hello(x: String)
49-
private val jarURL3 = Thread.currentThread().getContextClassLoader
50-
.getResource(s"TestHelloV3_$scalaVersion.jar")
51+
private val jarURL3 = createHelloJar(3)
5152
private lazy val jar3 = jarURL3.toString
5253

5354
test("Executor classloader isolation with JobArtifactSet") {
@@ -122,7 +123,6 @@ class ClassLoaderIsolationSuite extends SparkFunSuite with LocalSparkContext {
122123

123124
test("SPARK-51537 Executor isolation session classloader inherits from " +
124125
"default session classloader") {
125-
assume(jarURL2 != null)
126126
sc = new SparkContext(new SparkConf()
127127
.setAppName("test")
128128
.setMaster("local")

dev/test-jars.txt

Lines changed: 0 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -1,10 +1,4 @@
1-
core/src/test/resources/TestHelloV2_2.13.jar
2-
core/src/test/resources/TestHelloV3_2.13.jar
31
data/artifact-tests/junitLargeJar.jar
42
data/artifact-tests/smallJar.jar
5-
sql/connect/client/jvm/src/test/resources/TestHelloV2_2.13.jar
6-
sql/connect/client/jvm/src/test/resources/udf2.13.jar
73
sql/connect/common/src/test/resources/artifact-tests/junitLargeJar.jar
84
sql/connect/common/src/test/resources/artifact-tests/smallJar.jar
9-
sql/core/src/test/resources/artifact-tests/udf_noA.jar
10-
sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.13.jar

sql/connect/client/jvm/src/test/resources/StubClassDummyUdf.scala

Lines changed: 7 additions & 32 deletions
Original file line numberDiff line numberDiff line change
@@ -16,41 +16,16 @@
1616
*/
1717
package org.apache.spark.sql.connect.client
1818

19-
// To generate a jar from the source file:
20-
// `scalac StubClassDummyUdf.scala -d udf.jar`
21-
// To remove class A from the jar:
22-
// `jar -xvf udf.jar` -> delete A.class and A$.class
23-
// `jar -cvf udf_noA.jar org/`
19+
/**
20+
* Dummy UDF class used for testing dynamic class loading and stub class behavior.
21+
*
22+
* At test time, this file is compiled by UDFClassLoadingE2ESuite via
23+
* createJarWithScalaSources() to generate a JAR and a serialized UdfPacket binary.
24+
* StubClassDummyUdfPacker (in a separate file) handles the serialization.
25+
*/
2426
class StubClassDummyUdf {
2527
val udf: Int => Int = (x: Int) => x + 1
2628
val dummy = (x: Int) => A(x)
2729
}
2830

2931
case class A(x: Int) { def get: Int = x + 5 }
30-
31-
// The code to generate the udf file
32-
object StubClassDummyUdf {
33-
import java.io.{BufferedOutputStream, File, FileOutputStream}
34-
import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.PrimitiveIntEncoder
35-
import org.apache.spark.sql.connect.common.UdfPacket
36-
import org.apache.spark.util.Utils
37-
38-
def packDummyUdf(): String = {
39-
val byteArray =
40-
Utils.serialize[UdfPacket](
41-
new UdfPacket(
42-
new StubClassDummyUdf().udf,
43-
Seq(PrimitiveIntEncoder),
44-
PrimitiveIntEncoder
45-
)
46-
)
47-
val file = new File("src/test/resources/udf")
48-
val target = new BufferedOutputStream(new FileOutputStream(file))
49-
try {
50-
target.write(byteArray)
51-
file.getAbsolutePath
52-
} finally {
53-
target.close
54-
}
55-
}
56-
}
Lines changed: 41 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,41 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
package org.apache.spark.sql.connect.client
18+
19+
import java.io.{ByteArrayOutputStream, File, FileOutputStream, ObjectOutputStream}
20+
21+
import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.PrimitiveIntEncoder
22+
import org.apache.spark.sql.connect.common.UdfPacket
23+
24+
/**
25+
* Generates the serialized UdfPacket binary used by UDFClassLoadingE2ESuite.
26+
* Run at build time via TestJarTask.buildJarAndRun.
27+
*/
28+
object StubClassDummyUdfPacker {
29+
def main(args: Array[String]): Unit = {
30+
val packet = new UdfPacket(
31+
new StubClassDummyUdf().udf,
32+
Seq(PrimitiveIntEncoder),
33+
PrimitiveIntEncoder)
34+
val bos = new ByteArrayOutputStream()
35+
val oos = new ObjectOutputStream(bos)
36+
oos.writeObject(packet)
37+
oos.flush()
38+
val fos = new FileOutputStream(new File(args(0)))
39+
try { fos.write(bos.toByteArray) } finally { fos.close() }
40+
}
41+
}
-4.02 KB
Binary file not shown.
-1.59 KB
Binary file not shown.

0 commit comments

Comments
 (0)