-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-43744][CONNECT] Fix class loading problem caused by stub user classes not found on the server classpath #42069
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
Closed
Changes from 3 commits
Commits
Show all changes
5 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
85 changes: 85 additions & 0 deletions
85
...ctor/connect/client/jvm/src/test/scala/org/apache/spark/sql/UDFClassLoadingE2ESuite.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,85 @@ | ||
| /* | ||
| * 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 | ||
|
|
||
| import java.util.Arrays | ||
|
|
||
| import org.apache.spark.sql.connect.client.SparkResult | ||
| import org.apache.spark.sql.connect.client.util.RemoteSparkSession | ||
| import org.apache.spark.sql.types.{StringType, StructField, StructType} | ||
|
|
||
| class UDFClassLoadingE2ESuite extends RemoteSparkSession { | ||
|
|
||
| test("load udf with default stub class loader") { | ||
| val rows = spark.range(10).filter(n => n % 2 == 0).collectAsList() | ||
| assert(rows == Arrays.asList[Long](0, 2, 4, 6, 8)) | ||
| } | ||
|
|
||
| test("update class loader after stubbing: new session") { | ||
| // Session1 uses Stub SparkResult class | ||
| val session1 = spark.newSession() | ||
zhenlineo marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| addClientTestArtifactInServerClasspath(session1) | ||
| val ds = session1.range(10).filter(n => n % 2 == 0) | ||
|
|
||
| val rows = ds.collectAsList() | ||
| assert(rows == Arrays.asList[Long](0, 2, 4, 6, 8)) | ||
|
|
||
| // Session2 uses the real SparkResult class | ||
| val session2 = spark.newSession() | ||
| addClientTestArtifactInServerClasspath(session2) | ||
| addClientTestArtifactInServerClasspath(session2, testJar = false) | ||
| val rows2 = session2 | ||
| .range(10) | ||
| .filter(n => { | ||
| // Try to use spark result | ||
| new SparkResult[Int](null, null, null) | ||
zhenlineo marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| n > 5 | ||
| }) | ||
| .collectAsList() | ||
| assert(rows2 == Arrays.asList[Long](6, 7, 8, 9)) | ||
| } | ||
|
|
||
| test("update class loader after stubbing: same session") { | ||
| val session = spark.newSession() | ||
| addClientTestArtifactInServerClasspath(session) | ||
| val ds = session.range(10).filter(n => n % 2 == 0) | ||
|
|
||
| // load SparkResult as a stubbed class | ||
| val rows = ds.collectAsList() | ||
| assert(rows == Arrays.asList[Long](0, 2, 4, 6, 8)) | ||
|
|
||
| // Upload SparkResult and then SparkResult can be used in the udf | ||
| addClientTestArtifactInServerClasspath(session, testJar = false) | ||
| val rows2 = session.range(10).filter(n => { | ||
| // Try to use spark result | ||
| new SparkResult[Int](null, null, null) | ||
| n > 5 | ||
| }).collectAsList() | ||
| assert(rows2 == Arrays.asList[Long](6, 7, 8, 9)) | ||
| } | ||
|
|
||
| // This dummy method generates a lambda in the test class with SparkResult in its signature. | ||
| // This will cause class loading issue on the server side as the client jar is | ||
| // not in the server classpath. | ||
| def dummyMethod(): Unit = { | ||
| val df = spark.sql("select val from (values ('Hello'), ('World')) as t(val)") | ||
| df.withResult { result => | ||
| val schema = result.schema | ||
| assert(schema == StructType(StructField("val", StringType, nullable = false) :: Nil)) | ||
| } | ||
| } | ||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
55 changes: 55 additions & 0 deletions
55
connector/connect/server/src/test/resources/StubClassDummyUdf.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,55 @@ | ||
| /* | ||
| * 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.connect.artifact | ||
|
|
||
| // To generate a jar from the source file: | ||
| // `scalac StubClassDummyUdf.scala -d udf.jar` | ||
| // To remove class A from the jar: | ||
| // `jar -xvf udf.jar` -> delete A.class and A$.class | ||
| // `jar -cvf udf_noA.jar org/` | ||
| class StubClassDummyUdf { | ||
| val udf: Int => Int = (x: Int) => x + 1 | ||
| val dummy = (x: Int) => A(x) | ||
| } | ||
|
|
||
| case class A(x: Int) { def get: Int = x + 5 } | ||
|
|
||
| // The code to generate the udf file | ||
| object StubClassDummyUdf { | ||
| import java.io.{BufferedOutputStream, File, FileOutputStream} | ||
| import org.apache.spark.sql.connect.common.UdfPacket | ||
| import org.apache.spark.util.Utils | ||
|
|
||
| def packDummyUdf(): String = { | ||
| val byteArray = | ||
| Utils.serialize[UdfPacket]( | ||
| new UdfPacket( | ||
| new StubClassDummyUdf().udf, | ||
| Seq.empty, | ||
| null | ||
| ) | ||
| ) | ||
| val file = new File("src/test/resources/udf") | ||
| val target = new BufferedOutputStream(new FileOutputStream(file)) | ||
| try { | ||
| target.write(byteArray) | ||
| file.getAbsolutePath | ||
| } finally { | ||
| target.close | ||
| } | ||
| } | ||
| } |
Binary file not shown.
Binary file not shown.
147 changes: 147 additions & 0 deletions
147
...ct/server/src/test/scala/org/apache/spark/sql/connect/artifact/StubClassLoaderSuite.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,147 @@ | ||
| /* | ||
| * 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.connect.artifact | ||
|
|
||
| import java.io.File | ||
| import java.nio.file.{Files, Paths} | ||
|
|
||
| import org.apache.spark.SparkFunSuite | ||
| import org.apache.spark.sql.connect.common.UdfPacket | ||
| import org.apache.spark.util.{ChildFirstURLClassLoader, StubClassLoader, Utils} | ||
|
|
||
| class StubClassLoaderSuite extends SparkFunSuite { | ||
|
|
||
| private val udfByteArray: Array[Byte] = Files.readAllBytes(Paths.get("src/test/resources/udf")) | ||
| private val udfNoAJar = new File("src/test/resources/udf_noA.jar").toURI.toURL | ||
|
|
||
| test("find class with stub class") { | ||
| val cl = new RecordedStubClassLoader(getClass().getClassLoader(), _ => true) | ||
| val cls = cl.findClass("my.name.HelloWorld") | ||
| assert(cls.getName === "my.name.HelloWorld") | ||
| assert(cl.lastStubbed === "my.name.HelloWorld") | ||
| } | ||
|
|
||
| test("class for name with stub class") { | ||
| val cl = new RecordedStubClassLoader(getClass().getClassLoader(), _ => true) | ||
| // scalastyle:off classforname | ||
| val cls = Class.forName("my.name.HelloWorld", false, cl) | ||
| // scalastyle:on classforname | ||
| assert(cls.getName === "my.name.HelloWorld") | ||
| assert(cl.lastStubbed === "my.name.HelloWorld") | ||
| } | ||
|
|
||
| test("filter class to stub") { | ||
| val list = "my.name" :: Nil | ||
| val cl = StubClassLoader(getClass().getClassLoader(), list) | ||
| val cls = cl.findClass("my.name.HelloWorld") | ||
| assert(cls.getName === "my.name.HelloWorld") | ||
|
|
||
| intercept[ClassNotFoundException] { | ||
| cl.findClass("name.my.GoodDay") | ||
| } | ||
| } | ||
|
|
||
| test("load udf") { | ||
| // See src/test/resources/StubClassDummyUdf for how the udf and jar is created. | ||
| val sysClassLoader = getClass.getClassLoader() | ||
| val stubClassLoader = new RecordedStubClassLoader(null, _ => true) | ||
|
|
||
| // Install artifact without class A. | ||
| val sessionClassLoader = new ChildFirstURLClassLoader( | ||
| Array(udfNoAJar), | ||
| stubClassLoader, | ||
| sysClassLoader | ||
| ) | ||
| // Load udf with A used in the same class. | ||
| deserializeUdf(sessionClassLoader) | ||
| // Class A should be stubbed. | ||
| assert(stubClassLoader.lastStubbed === "org.apache.spark.sql.connect.artifact.A") | ||
| } | ||
|
|
||
| test("unload stub class") { | ||
| // See src/test/resources/StubClassDummyUdf for how the udf and jar is created. | ||
| val sysClassLoader = getClass.getClassLoader() | ||
| val stubClassLoader = new RecordedStubClassLoader(null, _ => true) | ||
|
|
||
| val cl1 = new ChildFirstURLClassLoader( | ||
| Array.empty, | ||
| stubClassLoader, | ||
| sysClassLoader) | ||
|
|
||
| // Failed to load dummy udf | ||
| intercept[Exception]{ | ||
| deserializeUdf(cl1) | ||
| } | ||
| // Successfully stubbed the missing class. | ||
| assert(stubClassLoader.lastStubbed === | ||
| "org.apache.spark.sql.connect.artifact.StubClassDummyUdf") | ||
|
|
||
| // Creating a new class loader will unpack the udf correctly. | ||
| val cl2 = new ChildFirstURLClassLoader( | ||
| Array(udfNoAJar), | ||
| stubClassLoader, // even with the same stub class loader. | ||
| sysClassLoader | ||
| ) | ||
| // Should be able to load after the artifact is added | ||
| deserializeUdf(cl2) | ||
| } | ||
|
|
||
| test("throw no such method if trying to access methods on stub class") { | ||
| // See src/test/resources/StubClassDummyUdf for how the udf and jar is created. | ||
| val sysClassLoader = getClass.getClassLoader() | ||
| val stubClassLoader = new RecordedStubClassLoader(null, _ => true) | ||
|
|
||
| val sessionClassLoader = new ChildFirstURLClassLoader( | ||
| Array.empty, | ||
| stubClassLoader, | ||
| sysClassLoader) | ||
|
|
||
| // Failed to load dummy udf | ||
| val exception = intercept[Exception]{ | ||
| deserializeUdf(sessionClassLoader) | ||
| } | ||
| // Successfully stubbed the missing class. | ||
| assert(stubClassLoader.lastStubbed === | ||
| "org.apache.spark.sql.connect.artifact.StubClassDummyUdf") | ||
| // But failed to find the method on the stub class. | ||
| val cause = exception.getCause | ||
| assert(cause.isInstanceOf[NoSuchMethodException]) | ||
| assert( | ||
| cause.getMessage.contains("org.apache.spark.sql.connect.artifact.StubClassDummyUdf"), | ||
| cause.getMessage | ||
| ) | ||
| } | ||
|
|
||
| private def deserializeUdf(sessionClassLoader: ClassLoader): UdfPacket = { | ||
| Utils.deserialize[UdfPacket]( | ||
| udfByteArray, | ||
| sessionClassLoader | ||
| ) | ||
| } | ||
| } | ||
|
|
||
| class RecordedStubClassLoader(parent: ClassLoader, shouldStub: String => Boolean) | ||
| extends StubClassLoader(parent, shouldStub) { | ||
| var lastStubbed: String = _ | ||
|
|
||
| override def findClass(name: String): Class[_] = { | ||
| if (shouldStub(name)) { | ||
| lastStubbed = name | ||
| } | ||
| super.findClass(name) | ||
| } | ||
| } |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.