From 33d896138b87b2069d4452a78305db7289f3a596 Mon Sep 17 00:00:00 2001 From: Zhen Li Date: Thu, 15 Jun 2023 11:07:28 -0700 Subject: [PATCH 1/5] Reproduce the error with classNotFound with SparkResult --- .../apache/spark/sql/SparkSessionE2ESuite.scala | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionE2ESuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionE2ESuite.scala index 5afafaaa6b92..e8041394dba9 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionE2ESuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionE2ESuite.scala @@ -25,6 +25,7 @@ import org.scalatest.concurrent.Eventually._ import org.apache.spark.SparkException import org.apache.spark.sql.connect.client.util.RemoteSparkSession +import org.apache.spark.sql.types.{StringType, StructField, StructType} import org.apache.spark.util.ThreadUtils /** @@ -247,4 +248,19 @@ class SparkSessionE2ESuite extends RemoteSparkSession { } assert(e.getMessage contains "OPERATION_CANCELED") } + + test("spark result schema") { + val df = spark.sql("select val from (values ('Hello'), ('World')) as t(val)") +// val result = df.collectResult() +// try { +// val schema = result.schema +// assert(schema == StructType(StructField("val", StringType, nullable = false) :: Nil)) +// } finally { +// result.close() +// } + df.withResult { result => + val schema = result.schema + assert(schema == StructType(StructField("val", StringType, nullable = false) :: Nil)) + } + } } From 98ec68c62d83b586fc8b4d83ec0650842082390b Mon Sep 17 00:00:00 2001 From: Zhen Li Date: Thu, 6 Jul 2023 08:56:03 -0700 Subject: [PATCH 2/5] Fix executor class loader --- connector/connect/client/jvm/pom.xml | 7 + .../spark/sql/SparkSessionE2ESuite.scala | 16 --- .../spark/sql/UDFClassLoadingE2ESuite.scala | 61 +++++++++ .../client/util/IntegrationTestUtils.scala | 2 +- .../client/util/RemoteSparkSession.scala | 17 +++ .../SparkConnectArtifactManager.scala | 9 +- .../test/resources/StubClassDummyUdf.scala | 55 ++++++++ .../connect/server/src/test/resources/udf | Bin 0 -> 973 bytes .../connect/server/src/test/resources/udf.jar | Bin 0 -> 5350 bytes .../server/src/test/resources/udf_noA.jar | Bin 0 -> 4396 bytes .../artifact/StubClassLoaderSuite.scala | 122 ++++++++++++++++++ .../spark/util/ChildFirstURLClassLoader.java | 9 ++ .../org/apache/spark/executor/Executor.scala | 44 +++++-- .../spark/internal/config/package.scala | 14 ++ .../apache/spark/util/StubClassLoader.scala | 79 ++++++++++++ 15 files changed, 404 insertions(+), 31 deletions(-) create mode 100644 connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UDFClassLoadingE2ESuite.scala create mode 100644 connector/connect/server/src/test/resources/StubClassDummyUdf.scala create mode 100644 connector/connect/server/src/test/resources/udf create mode 100644 connector/connect/server/src/test/resources/udf.jar create mode 100644 connector/connect/server/src/test/resources/udf_noA.jar create mode 100644 connector/connect/server/src/test/scala/org/apache/spark/sql/connect/artifact/StubClassLoaderSuite.scala create mode 100644 core/src/main/scala/org/apache/spark/util/StubClassLoader.scala diff --git a/connector/connect/client/jvm/pom.xml b/connector/connect/client/jvm/pom.xml index 6605496a1654..48d63f85a9d1 100644 --- a/connector/connect/client/jvm/pom.xml +++ b/connector/connect/client/jvm/pom.xml @@ -221,6 +221,13 @@ org.apache.maven.plugins maven-jar-plugin + + prepare-client-jar + compile + + jar + + prepare-test-jar test-compile diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionE2ESuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionE2ESuite.scala index e8041394dba9..5afafaaa6b92 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionE2ESuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionE2ESuite.scala @@ -25,7 +25,6 @@ import org.scalatest.concurrent.Eventually._ import org.apache.spark.SparkException import org.apache.spark.sql.connect.client.util.RemoteSparkSession -import org.apache.spark.sql.types.{StringType, StructField, StructType} import org.apache.spark.util.ThreadUtils /** @@ -248,19 +247,4 @@ class SparkSessionE2ESuite extends RemoteSparkSession { } assert(e.getMessage contains "OPERATION_CANCELED") } - - test("spark result schema") { - val df = spark.sql("select val from (values ('Hello'), ('World')) as t(val)") -// val result = df.collectResult() -// try { -// val schema = result.schema -// assert(schema == StructType(StructField("val", StringType, nullable = false) :: Nil)) -// } finally { -// result.close() -// } - df.withResult { result => - val schema = result.schema - assert(schema == StructType(StructField("val", StringType, nullable = false) :: Nil)) - } - } } diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UDFClassLoadingE2ESuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UDFClassLoadingE2ESuite.scala new file mode 100644 index 000000000000..cf586db7c421 --- /dev/null +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UDFClassLoadingE2ESuite.scala @@ -0,0 +1,61 @@ +/* + * 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") { + 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, 8)) + } + + // 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)) + } + } +} diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/util/IntegrationTestUtils.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/util/IntegrationTestUtils.scala index 0eaca7577b92..01d1c1204382 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/util/IntegrationTestUtils.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/util/IntegrationTestUtils.scala @@ -30,7 +30,7 @@ object IntegrationTestUtils { // System properties used for testing and debugging private val DEBUG_SC_JVM_CLIENT = "spark.debug.sc.jvm.client" // Enable this flag to print all client debug log + server logs to the console - private[connect] val isDebug = System.getProperty(DEBUG_SC_JVM_CLIENT, "false").toBoolean + private[connect] val isDebug = System.getProperty(DEBUG_SC_JVM_CLIENT, "true").toBoolean private[sql] lazy val scalaVersion = { versionNumberString.split('.') match { diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/util/RemoteSparkSession.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/util/RemoteSparkSession.scala index 594d3c369fe6..67fac0285f36 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/util/RemoteSparkSession.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/util/RemoteSparkSession.scala @@ -211,6 +211,23 @@ trait RemoteSparkSession extends ConnectFunSuite with BeforeAndAfterAll { throw error } } + + addClientTestArtifactInServerClasspath(spark) + } + + // For UDF maven E2E tests, the server needs the client test code to find the UDFs defined in + // tests. + private[sql] def addClientTestArtifactInServerClasspath( + session: SparkSession, + testJar: Boolean = true): Unit = { + tryFindJar( + "connector/connect/client/jvm", + // SBT passes the client & test jars to the server process automatically. + // So we skip building or finding this jar for SBT. + "sbt-tests-do-not-need-this-jar", + "spark-connect-client-jvm", + test = testJar + ).foreach(clientTestJar => session.addArtifact(clientTestJar.getCanonicalPath)) } override def afterAll(): Unit = { diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/artifact/SparkConnectArtifactManager.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/artifact/SparkConnectArtifactManager.scala index d8f290639c2f..a64c33415894 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/artifact/SparkConnectArtifactManager.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/artifact/SparkConnectArtifactManager.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.connect.artifact import java.io.File -import java.net.{URI, URL, URLClassLoader} +import java.net.{URI, URL} import java.nio.file.{Files, Path, Paths, StandardCopyOption} import java.util.concurrent.CopyOnWriteArrayList import javax.ws.rs.core.UriBuilder @@ -31,12 +31,13 @@ import org.apache.hadoop.fs.{LocalFileSystem, Path => FSPath} import org.apache.spark.{JobArtifactSet, JobArtifactState, SparkContext, SparkEnv} import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.CONNECT_SCALA_UDF_STUB_CLASSES import org.apache.spark.sql.SparkSession import org.apache.spark.sql.connect.artifact.util.ArtifactUtils import org.apache.spark.sql.connect.config.Connect.CONNECT_COPY_FROM_LOCAL_TO_FS_ALLOW_DEST_LOCAL import org.apache.spark.sql.connect.service.SessionHolder import org.apache.spark.storage.{CacheId, StorageLevel} -import org.apache.spark.util.Utils +import org.apache.spark.util.{ChildFirstURLClassLoader, StubClassLoader, Utils} /** * The Artifact Manager for the [[SparkConnectService]]. @@ -161,7 +162,9 @@ class SparkConnectArtifactManager(sessionHolder: SessionHolder) extends Logging */ def classloader: ClassLoader = { val urls = getSparkConnectAddedJars :+ classDir.toUri.toURL - new URLClassLoader(urls.toArray, Utils.getContextOrSparkClassLoader) + val stubClassLoader = + StubClassLoader(null, SparkEnv.get.conf.get(CONNECT_SCALA_UDF_STUB_CLASSES)) + new ChildFirstURLClassLoader(urls.toArray, stubClassLoader, Utils.getContextOrSparkClassLoader) } /** diff --git a/connector/connect/server/src/test/resources/StubClassDummyUdf.scala b/connector/connect/server/src/test/resources/StubClassDummyUdf.scala new file mode 100644 index 000000000000..e836c6010850 --- /dev/null +++ b/connector/connect/server/src/test/resources/StubClassDummyUdf.scala @@ -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 } + +object StubClassDummyUdf { + // The code to generate the udf file + 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 + } + } +} diff --git a/connector/connect/server/src/test/resources/udf b/connector/connect/server/src/test/resources/udf new file mode 100644 index 0000000000000000000000000000000000000000..55a3264a017fdccbed4cf40334bc6563395b890e GIT binary patch literal 973 zcmaiz&5P4O7{}k;zqEQ#3Ak@p-$uv9L`ABEplvcr; z7f&9%`w#fHh!_6|51zd``JzRu=8%x#$uqzAnJ1rrkTcGbTgGM~sXLr7J$VN;y$AxAnj7D-fK7kof2|A%$%%t2pB>mVikU*J{(kuLjG~v@yXbM~^k?h4Diq=rHf}$RZGC+_Ehm<##^dQs5 zp&KnJ-FF>wJ|aD#a`}Ygu}L2b zoes%G200d%rw2Ck)Z4>#v-N6`(Aj+n&dV;XsH!DD*OfLPkda1i$G~_aUkxll>wUcb z>Kc7b_7oJKlDA|tCfENeN<^liByW`G%E_wznuV27R}8n80sQXJ7uq3c6o5RfNuDqA z+JBxCLe3Vp4X94MIs6Y_UVhqg1KA;@(vg`Z_x{;5uqb>!7$#VZD(DA=QVZWazfF71 zb0jzF64}CIDNf77-lLBmHdWjW6pNz3d0`jwTrQUHr*B4s2rG;=-Kq;uTMYUK29jO? z+GrntP^9Bvw+wZwy7|)GuBiO~(pEbZN)-qr$-`JU4B8Gt+501F=8I;iQyXPQ-eTuJ W7Oew6)M@bk6&wCcE~qNtdHxHjbxnEz literal 0 HcmV?d00001 diff --git a/connector/connect/server/src/test/resources/udf.jar b/connector/connect/server/src/test/resources/udf.jar new file mode 100644 index 0000000000000000000000000000000000000000..47a81b4bf8d564162543950157f0572162a5d14f GIT binary patch literal 5350 zcmbtY2TT)ew=M;wLZNJsy*IQB%P0l04B4X$LH6Fe6;Kcq1e7Jn6i`+bS+e((p)A>3 z*?Y<;pdeT8{gdyzfAZga$$gSHIVW%4bKacfJUQpl(v$9W5C})3m^h$ zs~9SYs_UpiwUu<#RaMZ2V%n;|dI5mtRUkl5lLS;8kxNVT`wj5-(ff~)kiT!ee4U^+ z-Zpm54p2XD8(%l5pN~7#&dbx&!OkCQ#UTDT@tv(^rXm2V%!ak%p z)H)2=mBnwAuOE{ftDPJVibXTQR?&(dB>n1-`u+BU*YPSnEDi@MFR%!Ly7obXeu5YrYZWew;_d4e## zH1+Td7T#Lz6b$R+7A`RuYo$#)@I@VUTu0u8TY}>~ zQ9-1>BJXT%>_}WST#!UX_mzBfpC?0Ws!XO8|M#@CxCalPDCk=AvMp14S(@vZkN2c7 z*xe3tdgiCCd0TmorLTeL7(yFW`=JbJ5g@xTlVm3a1rK(GC(V%$R#VaviL1}`erNUI z;h{|XVMuNC$c{yLB*m8X#yE*#nX87RzEsSz@e>mNth%Y>NCKc`&uhTs`w37~7O=GT z_*nr{5R+St|2J%HfF=F3XCKmdS7?+ZOzo26_$_++8e671o+D$k2CqI2N#zs>C3_00 zE8(t=whpG=Y9PB~->ArSskTEhyvV@oEvtkD7?U0PH4+8`~S7bpsUCmBA zvH5KbpSlp?e)_L+#sM3Ow%%jD+^3`-z)j zUu+_mrU-v(O?@_27dLues5o4KI|%JuwjdQ7398r7?|JD1>xV;XVf==b3YAo6fw-wj zCn-FYfr z28Cr*n!qi$oAkRl?;2uO^@}Wx`z^3Y(?fEteub{fx;;h|MV^Q2NQ12)AKc&(L4n}tO7H+Iv89nqXTVR99S8(IzleU6U zC^(7}BP}#UY%|fX9|+ww0!=PCZNnB)$%DX47j$Mk=2R@m!uzm`SG%{rZ&t@r2`rE9 z%d~;ayN1K+AH&P}KWzjiY?vr<<$&CxSrdp^(&G_ZAt&3!%5yM+N$+Bq{KGyJ$fk7m zx;3mewDN1XqRk=44>lZhiQC^M=f*v_!i@>z$dNl+$b18K8@dGuW4C&kw(28A6vb>k zdyf+lFrWjbdcOIRir+OqS~L&kSDsX%SqM2eBb2X?TIJ=bIdbS!5NP*|T(u^7re>=M zgz-Ez=z0seI^fg*1PbD>v%OoC3B_V5=9Ldnh;tVEA*tt;rp4>pJZbG$EfF{CAlvvK ze1qAXoxqtmZ%E6fH;H5RhdPjOL6ZEv8#Udx9K2*9ZLNQJeQZP><3S7nB$54Jye|5W zd0pv$cs%Q_2YPmjPQj!?q_5n^zO6V=(}CHYuw9(9-TIC&;uaWAk&wX4`MI|S##gh# zARFdiJBzp4Io53T$Kop4YqGG?qgk7Mlb32)n>9!idJ3}e9EXIm{pR)Ez(H9PhL)>S zt6!vm$^$o`O|xm7Af_PTO}81Q;9O|J)zD7GVstE*3Z0?qVAT$lfP}Z5pjNA?i-gQ% zQ=lxl0Eh^|{i1h<$x|P3mae0`aiO)9RV8p?jp{>VK|>?903hM4YGXpFP|3t4wgh`o zSEcBoLCrXTHO$xwNe=pktK8KvMaN*#+SCiAJx^!2YJ3ojh7qvR_$X-=pl8Pqce}?s zYvD{(#9oGLy#`h7Nm4F%^ryq}ga*9i?9*kYpO&=U;rSGIJ@J$5MGQw2fe(r$dX`sx zElulv&Kc6xEL#(ptH=*YZ-Hb#d;oT<{Y=o}O}@CHoXi zAEeuNrNQ?5Y~xM3xbq|oMvY_70hciSh_^Qc(I~g|lC&{;Mr;j%7s)v#9rlj%cB8YJ zSKDKVrWZb^(*1DWYiXvPgfQwy?MwAF^^zSKI6%YB#m{aFt=XCmiXu5XaKTeK9(-db zq6C3!Gf-){fL!xDE1j=6?3z3lmwRnZy7JVaV@g6ny?d(!CaJ>J6ey+{d(qb(!Ko9M zzfu4~aD`;D>#!*tIq6d>WmbJw<`*J^ICpzxN7&RVg4E8`5`5T+L=A*z(c!mvZ#k>jeA-+ z8~gbkhGlhIRb}$v?DJ)7!}ICNNXJ9A8N2w~wIJnEXUR?!l+nCnkJTB;HSMsNkDsIX5m$Mg|c^E&$Z8n zgppd(bs_UH#WY5YeO%N7b#3HGAPlC|=E!1bMpT;3Ev0Wv?#i{LOd~F(KenCb^3ztO zi1E}rPH^6*9Xy8vRuTohyY()APk8q|)w5-?=5K+?ik+!G#EtKu)X0bjv(Cm<602xz zn>;}(uc)-}wI7@1xIUw5oa?o)5Hs6#XGsJgnB1)Fvc}QS)m5Z`y|ARD1`pzwzo0E) zIL=cGQk5L5nBiIYSuXCzD)^FfHT<6tU9u{HufS!!^^E;;!t1z=Bva=2jG${ zLcW;Vn=y!FseVXwMeI{s1`S)AT(V5-yp8mFINcP-LiyPzMpv}Bez42+V7Tz>~Vwzn1l&Up`XZaCi>@K07_h_iyjgX zXzx~~Rey&M)%X^~%TEhH%vM_sZ(O2-LviMMd+schl$I^Jz6bIj$P7Y2WddwOHYS2; zDP;Hs*1%Zbo{d+^T-s{_B#kGAUqqNa8 zK}nHZ*F47+^anABQQ-1=u|`Q2t>_dXyT*RO(Q$L-3syEE=lCvhi7Kg!2P~Hdztr*t zzO`SLwLVGynsYzllK$|KB==dreeyZOG1V`^OqGUAm4$CEgL~#1!OXAcR& zKDQE9TIc!v@HqQ^uWs*cWsurXcgVQ51!zanI#YfsP3oLK<|1@k<{T3H>}@Oj2PEqG z^u4W$T7Rks0i)%%I;J}(THwnYeEcmD6P<{FqexBa(b1b<{aOax56I9)pZM6in6xWLhlGg0l@vjuurXTS8*k3#?<#_pu*SY3(y1i-(l#Qyo9#`CX9&B#rcMu`@QCM_rcJvJbIeK$w6*KA%-Sr}4MV=a(y z;4?$vfI-|lRT{D)P78g}S;qHEU^#0SC!&5}3<5adQOGOY3 zf}=F&wGXr0uB94zv>d0vB+mIH?fE1Xg7ojoNgu~I*++ptj<>FWIX?~g`MZHUz~#Bi zDLAm_?5Jl6-lniZZc9ZXob+f^^E_P!_1^TEkL{>}KHR?$kpvcvUXeh}hD*oh(hP%u zGAQvlQ-2W);~Rs#iLG}=4yZ}{YVsmEH;OmmDB-fnSO~b5y54@&m+I5CkKRd(oaS%* z!&{4^@M_*vgy9L?tmd0<*)^rIDs<=bP&!7TU$BfaUuJ$dC=jJH!kA)d9bdR2Mnn_60 z4`li)8{Gv5zo(7tf2iod5<%bYL0Nwkoj1mO_$`(r>;d@7pr1nXp!e|yZ99vLX}Ch2 zoF|W#C_c?*qi#jb0vvWC6}{Yhxt!JZsw$(61|zYxaAGU%$U2267rGZd#--yRLS3>^Ms$GIgFV;49=h!^!)V zIJaRL11XCl5l2k*kda|>pLzq9i{C`M0h&$XvNT1yWgXO^!TjT0#3w9=Mz%}FNup8f z5F4;@W9E?0X;@pAdjW1rH!?EXsLmI9Bd9j8v@fmTepPvlADbL{MDOLykMJgfibbTk z`HiWrf@OjqOiBu&OkK_qi^Q?_Hh(JHGnOt0Dygm!Ic3E6?*qIkuso!91*o;wDCesSPE6XvBD81oE`ok-G7id|^-IGc1Gy z^~R@zR{vqc6Zc99`-2O;r!hV6j^22mK2%*C6T=z2hSgw5OM=JVJQ}2LI&?=^Cb8yU zmB5L2>2#aB)8!Coj|KHF2_`Q?{<}FnX)vTpga7~o$)DyV0MY{fQriBmoc&SS{x|-U z?)L9d{=TREQQ7{2#&7fgrMdlks=qVMpT+Mla9#VSss6PF{(I=Z*Xa*c{e?crKMwsb Z;?mP3{{0=m?}`!uoCyE`XPQ6We*y2fk+uK; literal 0 HcmV?d00001 diff --git a/connector/connect/server/src/test/resources/udf_noA.jar b/connector/connect/server/src/test/resources/udf_noA.jar new file mode 100644 index 0000000000000000000000000000000000000000..0d6fddee91cb0233818b456335dfc01a9ff70bb2 GIT binary patch literal 4396 zcma)<2{@Gd7stn-8Nwj4ug%ChMllIvtjWGBLdIH(Way%Vq9JP(T2v~!A~kj**~K(j zvgURzSH>1HBqjfOtDE7f{%4+fo_C)4o%5dG=RN25evgGQBNGn@3Wb8UCcm=-(HRSf z31q5oqpe_Qwoma(3kbvrvM`1~ffeBI%PfBC%tKr8!`W2Z%y6H+wGGmA-@GZQ)x=l< z+0JgPAWJ5_EVM$s+C4bxk2ErsH#BVze89vC(x3l}DW&thAR`$q;i$#YMEHIfa_l*nX2EfY*m)Ou0-xVsKim%~lzvFfxl9=u^#1 zzl#wG2`%6ZtCMa~SqY0Nov-E`CJ+ON4x~ALsDpAkAD_RGZoD`s226T$i6+>Uts0v+&x0l426(h)Rftwr(Z&*bv5 zn(YcmkJ;)6$$Z6I#(>R`k={OLU(xLdzr!JHOBIFgR93uYmRt#XygWc=fZW#En$9=mKolYtlIorWL)b>;Qes^$9@4$aI@uf z__R6Bp*ZzdL0h@^I8{tFG6tQ$FvwCFZkY*4e5unu)oI8?Xf zJ`(9l`^`^%uJf5aMY@N&%KL`AsuL~N*l{L9aZVVj2K4Z;y7bpO=3Z!v$6(pUf*OKp zXFw4LqfT7SSE`6)kMaNZ@J2);KSShp!q?ZsE#Mb)rGDp1NBQ5Al4t!%$)wHsdE7b} znV5a@mf(Ic(TW<1I-~FDmE!5O6Lj2PwlSPBNkY)8C|OAmUer+5bm`;D_2z^!mm*XcDO0EtdHctGfffa~?mVR)ZHmVzJ;57R!uBCPswuFs#K8Ox_>UOY0fDZ%$8mVP zUZ9Upu&ui%(hc}kW#{t;w*k-7I`AYnXKK1$bF5A@%Re3nyKU!)PSG)Xca-4790w5> z!*2N}?)DlTVr|&A>UP-l-bD&=j0&L|EKteF7(vviHTLmteEB>HKNI@)-ItY@P1T@5 zp&(P2=O`GW|Mry~`rhb2gXLAsm2+Gv6;?#Jw}W^5@ICk9mf(2w^G<=kGzMAS$gbhXw6Ed@-!Bz2X1yYbip zq^W$qNKjMid3F00dmrCWxrxzwnTYxwau2OuRr93J`s>cuvS1FPo!Jt7G~PN!SenBX z;uYD0cKIl!9@_i-gyt1}S2A&%~L+sFUeGLxJPHf+98 zJoGSugwMA$+#ee?mYYSe#3Ejo#&;wXqh%oa=GQf1T7CJd(^PL9-Z__^o^blic};T{ zgy;zO31>$$$F8PSJ~y2ZuZwt7V;#K#;noT!DvT$(^zkE%Q_$YQ{uDQ5MK*FnWXb?L zxtNoONy%`aWlqEg4(Ci?w&Au7a}$>9NOctv=z`ch@;2ISsT?W$a#C8Z!5-bTDB7sVXz`jy>H#OMf8T@qWHD( zl10NVSek@%_|^;1vWGj0V_ZVHCn~n?cYlJFTrrq}v=8wi{Pt=G>OLx_GTF&r+E=?f z$~^)U9E%m!^;&)1=Ov1#w0}6tl>I4SR3ZUfkvs7q@MR`(hQMU<(S_Uh_8rd~3vsZ4 zB(|C|u|}5xf+dN$3iT34817~KWYYWG&QQtrK@PUPAbU2ve#D6t*%4B1WZ882B#MNF zm7;dq6lxZ7Ed^)y_Fz6bvLjk;7V4&Y(Cq;=)Fy+(St0PzkKUN#L&Zt zxS*?A;b=hzb<0OQtT9r~6|b&ZkWzhf0`*-FwsyR|>T4Ft<;`4xGwE>x@9{)tw<|?~ zTKyL)g}IB2gi@^#kNdY}AI?}BQctp1!Sf%|WZIRBNKAW^&<+zq@oxdcpnBD8N?w(= z*VGk4`S>)BbI%r23+7JsNK>{)WIEqtJu6+C3KOl}fpsMCtDh3`=$yV-C1GiMO+-Q_ z!I8VSJ_I^!<7?D)a`K8UxaX<+Oj&-Y+BpIStkZ|e%l5F6pHN&qzZmlD3xV@i?FY?X z0;7C4taq$<9=9~Kcj|)DBSqA;!UJgM$x6!xiGwzQqn5eOb|j}$82k6^CM3;!iKe@jO8 zs~SD)wl6oODT4ko6qWPOvN{#nq9p?N$hI-#x=EJ7iqp2xo?)*^)LinG#qnIyBb_s<&!CPC?GfeY(8Qh1l;9-F0d2`l=&=})Bxd2OiRu$) z7bcnY22cz=et9U3Gp)K%T>fho7gT9@;a`zh+-G<8`@MlNS>>s1~*FH@iF;EDyK;1{eK?!#7RC5%A9GK(yte-5dJpE||yx8AI-`bS~+ zHs#nt`@AtzczVt1(EGE17@yYLk9i0frZ&3Lk2HX`UNhho@xzsF zlma5Kf_^^wtVO7yU49#c*3tz=A^MR37=_ksgmwvd{CM5}>Sn;qux1Mk>jne*eFFES z*VOCxDM}-^*+lWHTLapA&7>LEMfg{VuirGiG6l5wnjP58eNzG(6&Ijm*UXLf;ORE< zEsbxgb}bbk1ihRCbnTj@{w&1*l@1^P*Q}gYted8@CI{E2MX!DUjk9Ld--X&Vz0Gvf wT1*MZZ_@jrfapsJ&{6cA2sFu>dHmlof1;`^jG4Cp;W%iYK3Z8|zP88z0CLwo^#A|> literal 0 HcmV?d00001 diff --git a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/artifact/StubClassLoaderSuite.scala b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/artifact/StubClassLoaderSuite.scala new file mode 100644 index 000000000000..34b759316174 --- /dev/null +++ b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/artifact/StubClassLoaderSuite.scala @@ -0,0 +1,122 @@ +/* + * 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 scala.collection.mutable + +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")) + + 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") { + var list: mutable.Seq[String] = mutable.ArrayBuffer[String]() + val cl = StubClassLoader(getClass().getClassLoader(), () => list) + list :+= "my.name" + var cls = cl.findClass("my.name.HelloWorld") + assert(cls.getName === "my.name.HelloWorld") + + intercept[ClassNotFoundException] { + cl.findClass("name.my.GoodDay") + } + + list :+= "name.my" + cls = cl.findClass("name.my.GoodDay") + assert(cls.getName === "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(new File("src/test/resources/udf_noA.jar").toURI.toURL), + stubClassLoader, + sysClassLoader + ) + // Load udf with A used in the same class. + Utils.deserialize[UdfPacket]( + udfByteArray, + sessionClassLoader + ) + // Class A should be stubbed. + assert(stubClassLoader.lastStubbed === "org.apache.spark.sql.connect.artifact.A") + } + + 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]{Utils.deserialize[UdfPacket]( + udfByteArray, + sessionClassLoader + )} + // Succesfully 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 + ) + } +} + +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) + } +} diff --git a/core/src/main/java/org/apache/spark/util/ChildFirstURLClassLoader.java b/core/src/main/java/org/apache/spark/util/ChildFirstURLClassLoader.java index 57d96756c8be..de088c10d0b9 100644 --- a/core/src/main/java/org/apache/spark/util/ChildFirstURLClassLoader.java +++ b/core/src/main/java/org/apache/spark/util/ChildFirstURLClassLoader.java @@ -40,6 +40,15 @@ public ChildFirstURLClassLoader(URL[] urls, ClassLoader parent) { this.parent = new ParentClassLoader(parent); } + /** + * Specify the realParent if there is a need to load in the order of + * `realParent -> urls (child) -> parent`. + */ + public ChildFirstURLClassLoader(URL[] urls, ClassLoader parent, ClassLoader realParent) { + super(urls, realParent); + this.parent = new ParentClassLoader(parent); + } + @Override public Class loadClass(String name, boolean resolve) throws ClassNotFoundException { try { diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index b30569dc9641..56fe017c158b 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -513,12 +513,7 @@ private[spark] class Executor( override def run(): Unit = { // Classloader isolation - val isolatedSession = taskDescription.artifacts.state match { - case Some(jobArtifactState) => isolatedSessionCache.get( - jobArtifactState.uuid, - () => newSessionState(jobArtifactState)) - case _ => defaultSessionState - } + val isolatedSession = getIsolatedSession(taskDescription) setMDCForTask(taskName, mdcProperties) threadId = Thread.currentThread.getId @@ -543,11 +538,16 @@ private[spark] class Executor( // requires access to properties contained within (e.g. for access control). Executor.taskDeserializationProps.set(taskDescription.properties) - updateDependencies( + val updated = updateDependencies( taskDescription.artifacts.files, taskDescription.artifacts.jars, taskDescription.artifacts.archives, isolatedSession) + if (updated) { + // reset the thread class loader + val newIsolatedSession = getIsolatedSession(taskDescription) + Thread.currentThread.setContextClassLoader(newIsolatedSession.replClassLoader) + } task = ser.deserialize[Task[Any]]( taskDescription.serializedTask, Thread.currentThread.getContextClassLoader) task.localProperties = taskDescription.properties @@ -860,6 +860,14 @@ private[spark] class Executor( } } + private def getIsolatedSession( + taskDescription: TaskDescription) = { + taskDescription.artifacts.state match { + case Some(jobArtifactState) => + isolatedSessionCache.get(jobArtifactState.uuid, () => newSessionState(jobArtifactState)) + case _ => defaultSessionState + } + } private def setMDCForTask(taskName: String, mdc: Seq[(String, String)]): Unit = { try { mdc.foreach { case (key, value) => MDC.put(key, value) } @@ -1014,9 +1022,15 @@ private[spark] class Executor( logInfo(s"Starting executor with user classpath (userClassPathFirst = $userClassPathFirst): " + urls.mkString("'", ",", "'")) if (userClassPathFirst) { - new ChildFirstURLClassLoader(urls, systemLoader) + // user -> (sys -> stub) + val stubClassLoader = + StubClassLoader(systemLoader, conf.get(CONNECT_SCALA_UDF_STUB_CLASSES)) + new ChildFirstURLClassLoader(urls, stubClassLoader) } else { - new MutableURLClassLoader(urls, systemLoader) + // sys -> user -> stub + val stubClassLoader = + StubClassLoader(null, conf.get(CONNECT_SCALA_UDF_STUB_CLASSES)) + new ChildFirstURLClassLoader(urls, stubClassLoader, systemLoader) } } @@ -1047,7 +1061,8 @@ private[spark] class Executor( newArchives: immutable.Map[String, Long], state: IsolatedSessionState, testStartLatch: Option[CountDownLatch] = None, - testEndLatch: Option[CountDownLatch] = None): Unit = { + testEndLatch: Option[CountDownLatch] = None): Boolean = { + var updated = false; lazy val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) updateDependenciesLock.lockInterruptibly() try { @@ -1102,12 +1117,19 @@ private[spark] class Executor( val url = new File(root, localName).toURI.toURL if (!state.urlClassLoader.getURLs().contains(url)) { logInfo(s"Adding $url to class loader") - state.urlClassLoader.addURL(url) + // TODO: make use of the session cache for the class loader. + // Currently we invalidate all when adding a new url to always clear the stubbed + // classes in the current repl class loader. + // This is not always needed if the newly added jar does not contains any stubbed + // classes. + isolatedSessionCache.invalidateAll() + updated = true } } } // For testing, so we can simulate a slow file download: testEndLatch.foreach(_.await()) + updated } finally { updateDependenciesLock.unlock() } diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 04eba8bddeb6..77b20084378d 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -2547,4 +2547,18 @@ package object config { .version("3.5.0") .booleanConf .createWithDefault(false) + + private[spark] val CONNECT_SCALA_UDF_STUB_CLASSES = + ConfigBuilder("spark.connect.scalaUdf.stubClasses") + .internal() + .doc(""" + |Comma separated list of binary names of classes/packages that should be stub during the + |Scala UDF serdeser and execution if not found on the server classpath. + |An empty list effectively disables stubbing for all missing classes. + |By default the server stubs classes from the Scala client package. + |""".stripMargin) + .version("3.5.0") + .stringConf + .toSequence + .createWithDefault("org.apache.spark.sql.connect.client" :: Nil) } diff --git a/core/src/main/scala/org/apache/spark/util/StubClassLoader.scala b/core/src/main/scala/org/apache/spark/util/StubClassLoader.scala new file mode 100644 index 000000000000..a0bc753f4887 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/StubClassLoader.scala @@ -0,0 +1,79 @@ +/* + * 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.util + +import org.apache.xbean.asm9.{ClassWriter, Opcodes} + +/** + * [[ClassLoader]] that replaces missing classes with stubs, if the cannot be found. It will only + * do this for classes that are marked for stubbing. + * + * While this is generally not a good idea. In this particular case this is used to load lambda's + * whose capturing class contains unknown (and unneeded) classes. The lambda itself does not need + * the class and therefor is safe to replace by a stub. + */ +class StubClassLoader(parent: ClassLoader, shouldStub: String => Boolean) + extends ClassLoader(parent) { + override def findClass(name: String): Class[_] = { + if (!shouldStub(name)) { + throw new ClassNotFoundException(name) + } + val bytes = StubClassLoader.generateStub(name) + defineClass(name, bytes, 0, bytes.length) + } +} + +object StubClassLoader { + def apply(parent: ClassLoader, binaryName: Seq[String]): StubClassLoader = { + new StubClassLoader(parent, name => binaryName.exists(p => name.startsWith(p))) + } + + def generateStub(binaryName: String): Array[Byte] = { + // Convert binary names to internal names. + val name = binaryName.replace('.', '/') + val classWriter = new ClassWriter(0) + classWriter.visit( + 49, + Opcodes.ACC_PUBLIC + Opcodes.ACC_SUPER, + name, + null, + "java/lang/Object", + null) + classWriter.visitSource(name + ".java", null) + + // Generate constructor. + val ctorWriter = classWriter.visitMethod( + Opcodes.ACC_PUBLIC, + "", + "()V", + null, + null) + ctorWriter.visitVarInsn(Opcodes.ALOAD, 0) + ctorWriter.visitMethodInsn( + Opcodes.INVOKESPECIAL, + "java/lang/Object", + "", + "()V", + false) + + ctorWriter.visitInsn(Opcodes.RETURN) + ctorWriter.visitMaxs(1, 1) + ctorWriter.visitEnd() + classWriter.visitEnd() + classWriter.toByteArray + } +} From 79f716d81cb86733e28675acac7dd00549161a88 Mon Sep 17 00:00:00 2001 From: Zhen Li Date: Wed, 19 Jul 2023 16:12:53 -0700 Subject: [PATCH 3/5] Fix --- connector/connect/client/jvm/pom.xml | 2 +- .../spark/sql/UDFClassLoadingE2ESuite.scala | 28 ++++- .../client/util/RemoteSparkSession.scala | 14 +-- .../test/resources/StubClassDummyUdf.scala | 2 +- .../connect/server/src/test/resources/udf.jar | Bin 5350 -> 0 bytes .../artifact/StubClassLoaderSuite.scala | 65 +++++++---- .../spark/util/ChildFirstURLClassLoader.java | 8 +- .../org/apache/spark/executor/Executor.scala | 108 ++++++++++++------ .../apache/spark/util/StubClassLoader.scala | 1 + 9 files changed, 149 insertions(+), 79 deletions(-) delete mode 100644 connector/connect/server/src/test/resources/udf.jar diff --git a/connector/connect/client/jvm/pom.xml b/connector/connect/client/jvm/pom.xml index 48d63f85a9d1..29146d41396d 100644 --- a/connector/connect/client/jvm/pom.xml +++ b/connector/connect/client/jvm/pom.xml @@ -222,7 +222,7 @@ maven-jar-plugin - prepare-client-jar + default-jar compile jar diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UDFClassLoadingE2ESuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UDFClassLoadingE2ESuite.scala index cf586db7c421..9c84cd792444 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UDFClassLoadingE2ESuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UDFClassLoadingE2ESuite.scala @@ -29,7 +29,31 @@ class UDFClassLoadingE2ESuite extends RemoteSparkSession { assert(rows == Arrays.asList[Long](0, 2, 4, 6, 8)) } - test("update class loader after stubbing") { + test("update class loader after stubbing: new session") { + // Session1 uses Stub SparkResult class + val session1 = spark.newSession() + 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) + 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) @@ -45,7 +69,7 @@ class UDFClassLoadingE2ESuite extends RemoteSparkSession { new SparkResult[Int](null, null, null) n > 5 }).collectAsList() - assert(rows2 == Arrays.asList[Long](6, 8)) + assert(rows2 == Arrays.asList[Long](6, 7, 8, 9)) } // This dummy method generates a lambda in the test class with SparkResult in its signature. diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/util/RemoteSparkSession.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/util/RemoteSparkSession.scala index 67fac0285f36..869eabf55420 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/util/RemoteSparkSession.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/util/RemoteSparkSession.scala @@ -99,19 +99,7 @@ object SparkConnectServerUtils { .map(clientTestJar => Seq("--jars", clientTestJar.getCanonicalPath)) .getOrElse(Seq.empty) - // For UDF maven E2E tests, the server needs the client code to find the UDFs defined in tests. - val connectClientTestJar = tryFindJar( - "connector/connect/client/jvm", - // SBT passes the client & test jars to the server process automatically. - // So we skip building or finding this jar for SBT. - "sbt-tests-do-not-need-this-jar", - "spark-connect-client-jvm", - test = true) - .map(clientTestJar => Seq(clientTestJar.getCanonicalPath)) - .getOrElse(Seq.empty) - - val allJars = catalystTestJar ++ connectClientTestJar - val jarsConfigs = Seq("--jars", allJars.mkString(",")) + val jarsConfigs = Seq("--jars", catalystTestJar.mkString(",")) // Use InMemoryTableCatalog for V2 writer tests val writerV2Configs = Seq( diff --git a/connector/connect/server/src/test/resources/StubClassDummyUdf.scala b/connector/connect/server/src/test/resources/StubClassDummyUdf.scala index e836c6010850..16953a482d11 100644 --- a/connector/connect/server/src/test/resources/StubClassDummyUdf.scala +++ b/connector/connect/server/src/test/resources/StubClassDummyUdf.scala @@ -28,8 +28,8 @@ class StubClassDummyUdf { case class A(x: Int) { def get: Int = x + 5 } +// The code to generate the udf file object StubClassDummyUdf { - // The code to generate the udf file import java.io.{BufferedOutputStream, File, FileOutputStream} import org.apache.spark.sql.connect.common.UdfPacket import org.apache.spark.util.Utils diff --git a/connector/connect/server/src/test/resources/udf.jar b/connector/connect/server/src/test/resources/udf.jar deleted file mode 100644 index 47a81b4bf8d564162543950157f0572162a5d14f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5350 zcmbtY2TT)ew=M;wLZNJsy*IQB%P0l04B4X$LH6Fe6;Kcq1e7Jn6i`+bS+e((p)A>3 z*?Y<;pdeT8{gdyzfAZga$$gSHIVW%4bKacfJUQpl(v$9W5C})3m^h$ zs~9SYs_UpiwUu<#RaMZ2V%n;|dI5mtRUkl5lLS;8kxNVT`wj5-(ff~)kiT!ee4U^+ z-Zpm54p2XD8(%l5pN~7#&dbx&!OkCQ#UTDT@tv(^rXm2V%!ak%p z)H)2=mBnwAuOE{ftDPJVibXTQR?&(dB>n1-`u+BU*YPSnEDi@MFR%!Ly7obXeu5YrYZWew;_d4e## zH1+Td7T#Lz6b$R+7A`RuYo$#)@I@VUTu0u8TY}>~ zQ9-1>BJXT%>_}WST#!UX_mzBfpC?0Ws!XO8|M#@CxCalPDCk=AvMp14S(@vZkN2c7 z*xe3tdgiCCd0TmorLTeL7(yFW`=JbJ5g@xTlVm3a1rK(GC(V%$R#VaviL1}`erNUI z;h{|XVMuNC$c{yLB*m8X#yE*#nX87RzEsSz@e>mNth%Y>NCKc`&uhTs`w37~7O=GT z_*nr{5R+St|2J%HfF=F3XCKmdS7?+ZOzo26_$_++8e671o+D$k2CqI2N#zs>C3_00 zE8(t=whpG=Y9PB~->ArSskTEhyvV@oEvtkD7?U0PH4+8`~S7bpsUCmBA zvH5KbpSlp?e)_L+#sM3Ow%%jD+^3`-z)j zUu+_mrU-v(O?@_27dLues5o4KI|%JuwjdQ7398r7?|JD1>xV;XVf==b3YAo6fw-wj zCn-FYfr z28Cr*n!qi$oAkRl?;2uO^@}Wx`z^3Y(?fEteub{fx;;h|MV^Q2NQ12)AKc&(L4n}tO7H+Iv89nqXTVR99S8(IzleU6U zC^(7}BP}#UY%|fX9|+ww0!=PCZNnB)$%DX47j$Mk=2R@m!uzm`SG%{rZ&t@r2`rE9 z%d~;ayN1K+AH&P}KWzjiY?vr<<$&CxSrdp^(&G_ZAt&3!%5yM+N$+Bq{KGyJ$fk7m zx;3mewDN1XqRk=44>lZhiQC^M=f*v_!i@>z$dNl+$b18K8@dGuW4C&kw(28A6vb>k zdyf+lFrWjbdcOIRir+OqS~L&kSDsX%SqM2eBb2X?TIJ=bIdbS!5NP*|T(u^7re>=M zgz-Ez=z0seI^fg*1PbD>v%OoC3B_V5=9Ldnh;tVEA*tt;rp4>pJZbG$EfF{CAlvvK ze1qAXoxqtmZ%E6fH;H5RhdPjOL6ZEv8#Udx9K2*9ZLNQJeQZP><3S7nB$54Jye|5W zd0pv$cs%Q_2YPmjPQj!?q_5n^zO6V=(}CHYuw9(9-TIC&;uaWAk&wX4`MI|S##gh# zARFdiJBzp4Io53T$Kop4YqGG?qgk7Mlb32)n>9!idJ3}e9EXIm{pR)Ez(H9PhL)>S zt6!vm$^$o`O|xm7Af_PTO}81Q;9O|J)zD7GVstE*3Z0?qVAT$lfP}Z5pjNA?i-gQ% zQ=lxl0Eh^|{i1h<$x|P3mae0`aiO)9RV8p?jp{>VK|>?903hM4YGXpFP|3t4wgh`o zSEcBoLCrXTHO$xwNe=pktK8KvMaN*#+SCiAJx^!2YJ3ojh7qvR_$X-=pl8Pqce}?s zYvD{(#9oGLy#`h7Nm4F%^ryq}ga*9i?9*kYpO&=U;rSGIJ@J$5MGQw2fe(r$dX`sx zElulv&Kc6xEL#(ptH=*YZ-Hb#d;oT<{Y=o}O}@CHoXi zAEeuNrNQ?5Y~xM3xbq|oMvY_70hciSh_^Qc(I~g|lC&{;Mr;j%7s)v#9rlj%cB8YJ zSKDKVrWZb^(*1DWYiXvPgfQwy?MwAF^^zSKI6%YB#m{aFt=XCmiXu5XaKTeK9(-db zq6C3!Gf-){fL!xDE1j=6?3z3lmwRnZy7JVaV@g6ny?d(!CaJ>J6ey+{d(qb(!Ko9M zzfu4~aD`;D>#!*tIq6d>WmbJw<`*J^ICpzxN7&RVg4E8`5`5T+L=A*z(c!mvZ#k>jeA-+ z8~gbkhGlhIRb}$v?DJ)7!}ICNNXJ9A8N2w~wIJnEXUR?!l+nCnkJTB;HSMsNkDsIX5m$Mg|c^E&$Z8n zgppd(bs_UH#WY5YeO%N7b#3HGAPlC|=E!1bMpT;3Ev0Wv?#i{LOd~F(KenCb^3ztO zi1E}rPH^6*9Xy8vRuTohyY()APk8q|)w5-?=5K+?ik+!G#EtKu)X0bjv(Cm<602xz zn>;}(uc)-}wI7@1xIUw5oa?o)5Hs6#XGsJgnB1)Fvc}QS)m5Z`y|ARD1`pzwzo0E) zIL=cGQk5L5nBiIYSuXCzD)^FfHT<6tU9u{HufS!!^^E;;!t1z=Bva=2jG${ zLcW;Vn=y!FseVXwMeI{s1`S)AT(V5-yp8mFINcP-LiyPzMpv}Bez42+V7Tz>~Vwzn1l&Up`XZaCi>@K07_h_iyjgX zXzx~~Rey&M)%X^~%TEhH%vM_sZ(O2-LviMMd+schl$I^Jz6bIj$P7Y2WddwOHYS2; zDP;Hs*1%Zbo{d+^T-s{_B#kGAUqqNa8 zK}nHZ*F47+^anABQQ-1=u|`Q2t>_dXyT*RO(Q$L-3syEE=lCvhi7Kg!2P~Hdztr*t zzO`SLwLVGynsYzllK$|KB==dreeyZOG1V`^OqGUAm4$CEgL~#1!OXAcR& zKDQE9TIc!v@HqQ^uWs*cWsurXcgVQ51!zanI#YfsP3oLK<|1@k<{T3H>}@Oj2PEqG z^u4W$T7Rks0i)%%I;J}(THwnYeEcmD6P<{FqexBa(b1b<{aOax56I9)pZM6in6xWLhlGg0l@vjuurXTS8*k3#?<#_pu*SY3(y1i-(l#Qyo9#`CX9&B#rcMu`@QCM_rcJvJbIeK$w6*KA%-Sr}4MV=a(y z;4?$vfI-|lRT{D)P78g}S;qHEU^#0SC!&5}3<5adQOGOY3 zf}=F&wGXr0uB94zv>d0vB+mIH?fE1Xg7ojoNgu~I*++ptj<>FWIX?~g`MZHUz~#Bi zDLAm_?5Jl6-lniZZc9ZXob+f^^E_P!_1^TEkL{>}KHR?$kpvcvUXeh}hD*oh(hP%u zGAQvlQ-2W);~Rs#iLG}=4yZ}{YVsmEH;OmmDB-fnSO~b5y54@&m+I5CkKRd(oaS%* z!&{4^@M_*vgy9L?tmd0<*)^rIDs<=bP&!7TU$BfaUuJ$dC=jJH!kA)d9bdR2Mnn_60 z4`li)8{Gv5zo(7tf2iod5<%bYL0Nwkoj1mO_$`(r>;d@7pr1nXp!e|yZ99vLX}Ch2 zoF|W#C_c?*qi#jb0vvWC6}{Yhxt!JZsw$(61|zYxaAGU%$U2267rGZd#--yRLS3>^Ms$GIgFV;49=h!^!)V zIJaRL11XCl5l2k*kda|>pLzq9i{C`M0h&$XvNT1yWgXO^!TjT0#3w9=Mz%}FNup8f z5F4;@W9E?0X;@pAdjW1rH!?EXsLmI9Bd9j8v@fmTepPvlADbL{MDOLykMJgfibbTk z`HiWrf@OjqOiBu&OkK_qi^Q?_Hh(JHGnOt0Dygm!Ic3E6?*qIkuso!91*o;wDCesSPE6XvBD81oE`ok-G7id|^-IGc1Gy z^~R@zR{vqc6Zc99`-2O;r!hV6j^22mK2%*C6T=z2hSgw5OM=JVJQ}2LI&?=^Cb8yU zmB5L2>2#aB)8!Coj|KHF2_`Q?{<}FnX)vTpga7~o$)DyV0MY{fQriBmoc&SS{x|-U z?)L9d{=TREQQ7{2#&7fgrMdlks=qVMpT+Mla9#VSss6PF{(I=Z*Xa*c{e?crKMwsb Z;?mP3{{0=m?}`!uoCyE`XPQ6We*y2fk+uK; diff --git a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/artifact/StubClassLoaderSuite.scala b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/artifact/StubClassLoaderSuite.scala index 34b759316174..74b3e3c4306e 100644 --- a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/artifact/StubClassLoaderSuite.scala +++ b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/artifact/StubClassLoaderSuite.scala @@ -19,8 +19,6 @@ package org.apache.spark.sql.connect.artifact import java.io.File import java.nio.file.{Files, Paths} -import scala.collection.mutable - import org.apache.spark.SparkFunSuite import org.apache.spark.sql.connect.common.UdfPacket import org.apache.spark.util.{ChildFirstURLClassLoader, StubClassLoader, Utils} @@ -28,6 +26,7 @@ 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) @@ -46,19 +45,14 @@ class StubClassLoaderSuite extends SparkFunSuite { } test("filter class to stub") { - var list: mutable.Seq[String] = mutable.ArrayBuffer[String]() - val cl = StubClassLoader(getClass().getClassLoader(), () => list) - list :+= "my.name" - var cls = cl.findClass("my.name.HelloWorld") + 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") } - - list :+= "name.my" - cls = cl.findClass("name.my.GoodDay") - assert(cls.getName === "name.my.GoodDay") } test("load udf") { @@ -68,19 +62,44 @@ class StubClassLoaderSuite extends SparkFunSuite { // Install artifact without class A. val sessionClassLoader = new ChildFirstURLClassLoader( - Array(new File("src/test/resources/udf_noA.jar").toURI.toURL), + Array(udfNoAJar), stubClassLoader, sysClassLoader ) // Load udf with A used in the same class. - Utils.deserialize[UdfPacket]( - udfByteArray, - sessionClassLoader - ) + 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() @@ -92,11 +111,10 @@ class StubClassLoaderSuite extends SparkFunSuite { sysClassLoader) // Failed to load dummy udf - val exception = intercept[Exception]{Utils.deserialize[UdfPacket]( - udfByteArray, - sessionClassLoader - )} - // Succesfully stubbed the missing class. + 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. @@ -107,6 +125,13 @@ class StubClassLoaderSuite extends SparkFunSuite { cause.getMessage ) } + + private def deserializeUdf(sessionClassLoader: ClassLoader): UdfPacket = { + Utils.deserialize[UdfPacket]( + udfByteArray, + sessionClassLoader + ) + } } class RecordedStubClassLoader(parent: ClassLoader, shouldStub: String => Boolean) diff --git a/core/src/main/java/org/apache/spark/util/ChildFirstURLClassLoader.java b/core/src/main/java/org/apache/spark/util/ChildFirstURLClassLoader.java index de088c10d0b9..6b6e5cf18ab3 100644 --- a/core/src/main/java/org/apache/spark/util/ChildFirstURLClassLoader.java +++ b/core/src/main/java/org/apache/spark/util/ChildFirstURLClassLoader.java @@ -41,11 +41,11 @@ public ChildFirstURLClassLoader(URL[] urls, ClassLoader parent) { } /** - * Specify the realParent if there is a need to load in the order of - * `realParent -> urls (child) -> parent`. + * Specify the grandparent if there is a need to load in the order of + * `grandparent -> urls (child) -> parent`. */ - public ChildFirstURLClassLoader(URL[] urls, ClassLoader parent, ClassLoader realParent) { - super(urls, realParent); + public ChildFirstURLClassLoader(URL[] urls, ClassLoader parent, ClassLoader grandparent) { + super(urls, grandparent); this.parent = new ParentClassLoader(parent); } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 56fe017c158b..da5840de2844 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -56,11 +56,12 @@ import org.apache.spark.util._ private[spark] class IsolatedSessionState( val sessionUUID: String, - val urlClassLoader: MutableURLClassLoader, + var urlClassLoader: MutableURLClassLoader, var replClassLoader: ClassLoader, val currentFiles: HashMap[String, Long], val currentJars: HashMap[String, Long], - val currentArchives: HashMap[String, Long]) + val currentArchives: HashMap[String, Long], + val replClassDirUri: Option[String]) /** * Spark executor, backed by a threadpool to run tasks. @@ -173,14 +174,20 @@ private[spark] class Executor( val currentFiles = new HashMap[String, Long] val currentJars = new HashMap[String, Long] val currentArchives = new HashMap[String, Long] - val urlClassLoader = createClassLoader(currentJars) + val urlClassLoader = createClassLoader(currentJars, !isDefaultState(jobArtifactState.uuid)) val replClassLoader = addReplClassLoaderIfNeeded( - urlClassLoader, jobArtifactState.replClassDirUri) + urlClassLoader, jobArtifactState.replClassDirUri, jobArtifactState.uuid) new IsolatedSessionState( jobArtifactState.uuid, urlClassLoader, replClassLoader, - currentFiles, currentJars, currentArchives) + currentFiles, + currentJars, + currentArchives, + jobArtifactState.replClassDirUri + ) } + private def isDefaultState(name: String) = name == "default" + // Classloader isolation // The default isolation group val defaultSessionState = newSessionState(JobArtifactState("default", None)) @@ -513,7 +520,11 @@ private[spark] class Executor( override def run(): Unit = { // Classloader isolation - val isolatedSession = getIsolatedSession(taskDescription) + val isolatedSession = taskDescription.artifacts.state match { + case Some(jobArtifactState) => + isolatedSessionCache.get(jobArtifactState.uuid, () => newSessionState(jobArtifactState)) + case _ => defaultSessionState + } setMDCForTask(taskName, mdcProperties) threadId = Thread.currentThread.getId @@ -538,16 +549,13 @@ private[spark] class Executor( // requires access to properties contained within (e.g. for access control). Executor.taskDeserializationProps.set(taskDescription.properties) - val updated = updateDependencies( + updateDependencies( taskDescription.artifacts.files, taskDescription.artifacts.jars, taskDescription.artifacts.archives, isolatedSession) - if (updated) { - // reset the thread class loader - val newIsolatedSession = getIsolatedSession(taskDescription) - Thread.currentThread.setContextClassLoader(newIsolatedSession.replClassLoader) - } + // Always reset the thread class loader + Thread.currentThread.setContextClassLoader(isolatedSession.replClassLoader) task = ser.deserialize[Task[Any]]( taskDescription.serializedTask, Thread.currentThread.getContextClassLoader) task.localProperties = taskDescription.properties @@ -860,14 +868,6 @@ private[spark] class Executor( } } - private def getIsolatedSession( - taskDescription: TaskDescription) = { - taskDescription.artifacts.state match { - case Some(jobArtifactState) => - isolatedSessionCache.get(jobArtifactState.uuid, () => newSessionState(jobArtifactState)) - case _ => defaultSessionState - } - } private def setMDCForTask(taskName: String, mdc: Seq[(String, String)]): Unit = { try { mdc.foreach { case (key, value) => MDC.put(key, value) } @@ -1007,7 +1007,9 @@ private[spark] class Executor( * Create a ClassLoader for use in tasks, adding any JARs specified by the user or any classes * created by the interpreter to the search path */ - private def createClassLoader(currentJars: HashMap[String, Long]): MutableURLClassLoader = { + private def createClassLoader( + currentJars: HashMap[String, Long], + useStub: Boolean): MutableURLClassLoader = { // Bootstrap the list of jars with the user class path. val now = System.currentTimeMillis() userClassPath.foreach { url => @@ -1019,17 +1021,42 @@ private[spark] class Executor( val urls = userClassPath.toArray ++ currentJars.keySet.map { uri => new File(uri.split("/").last).toURI.toURL } - logInfo(s"Starting executor with user classpath (userClassPathFirst = $userClassPathFirst): " + - urls.mkString("'", ",", "'")) + createClassLoader(urls, useStub) + } + + private def createClassLoader(urls: Array[URL], useStub: Boolean): MutableURLClassLoader = { + logInfo( + s"Starting executor with user classpath (userClassPathFirst = $userClassPathFirst): " + + urls.mkString("'", ",", "'") + ) + + if (useStub && conf.get(CONNECT_SCALA_UDF_STUB_CLASSES).nonEmpty) { + createClassLoaderWithStub(urls, conf.get(CONNECT_SCALA_UDF_STUB_CLASSES)) + } else { + createClassLoader(urls) + } + } + + private def createClassLoader(urls: Array[URL]): MutableURLClassLoader = { + if (userClassPathFirst) { + new ChildFirstURLClassLoader(urls, systemLoader) + } else { + new MutableURLClassLoader(urls, systemLoader) + } + } + + private def createClassLoaderWithStub( + urls: Array[URL], + binaryName: Seq[String]): MutableURLClassLoader = { if (userClassPathFirst) { // user -> (sys -> stub) val stubClassLoader = - StubClassLoader(systemLoader, conf.get(CONNECT_SCALA_UDF_STUB_CLASSES)) + StubClassLoader(systemLoader, binaryName) new ChildFirstURLClassLoader(urls, stubClassLoader) } else { // sys -> user -> stub val stubClassLoader = - StubClassLoader(null, conf.get(CONNECT_SCALA_UDF_STUB_CLASSES)) + StubClassLoader(null, binaryName) new ChildFirstURLClassLoader(urls, stubClassLoader, systemLoader) } } @@ -1040,14 +1067,17 @@ private[spark] class Executor( */ private def addReplClassLoaderIfNeeded( parent: ClassLoader, - sessionClassUri: Option[String]): ClassLoader = { + sessionClassUri: Option[String], + sessionUUID: String): ClassLoader = { val classUri = sessionClassUri.getOrElse(conf.get("spark.repl.class.uri", null)) - if (classUri != null) { + val classLoader = if (classUri != null) { logInfo("Using REPL class URI: " + classUri) new ExecutorClassLoader(conf, env, classUri, parent, userClassPathFirst) } else { parent } + logInfo(s"Created or updated repl class loader for $sessionUUID.") + classLoader } /** @@ -1061,7 +1091,7 @@ private[spark] class Executor( newArchives: immutable.Map[String, Long], state: IsolatedSessionState, testStartLatch: Option[CountDownLatch] = None, - testEndLatch: Option[CountDownLatch] = None): Boolean = { + testEndLatch: Option[CountDownLatch] = None): Unit = { var updated = false; lazy val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) updateDependenciesLock.lockInterruptibly() @@ -1071,7 +1101,7 @@ private[spark] class Executor( // If the session ID was specified from SparkSession, it's from a Spark Connect client. // Specify a dedicated directory for Spark Connect client. - lazy val root = if (state.sessionUUID != "default") { + lazy val root = if (!isDefaultState(state.sessionUUID)) { val newDest = new File(SparkFiles.getRootDirectory(), state.sessionUUID) newDest.mkdir() newDest @@ -1116,20 +1146,22 @@ private[spark] class Executor( // Add it to our class loader val url = new File(root, localName).toURI.toURL if (!state.urlClassLoader.getURLs().contains(url)) { - logInfo(s"Adding $url to class loader") - // TODO: make use of the session cache for the class loader. - // Currently we invalidate all when adding a new url to always clear the stubbed - // classes in the current repl class loader. - // This is not always needed if the newly added jar does not contains any stubbed - // classes. - isolatedSessionCache.invalidateAll() - updated = true + logInfo(s"Adding $url to class loader ${state.sessionUUID}") + state.urlClassLoader.addURL(url) + if (!isDefaultState(state.sessionUUID)) { + updated = true + } } } } + if (updated) { + // TODO: only update the class loader if the stub class should be unloaded. + state.urlClassLoader = createClassLoader(state.urlClassLoader.getURLs, useStub = true) + state.replClassLoader = + addReplClassLoaderIfNeeded(state.urlClassLoader, state.replClassDirUri, state.sessionUUID) + } // For testing, so we can simulate a slow file download: testEndLatch.foreach(_.await()) - updated } finally { updateDependenciesLock.unlock() } diff --git a/core/src/main/scala/org/apache/spark/util/StubClassLoader.scala b/core/src/main/scala/org/apache/spark/util/StubClassLoader.scala index a0bc753f4887..40f5cadeb0d8 100644 --- a/core/src/main/scala/org/apache/spark/util/StubClassLoader.scala +++ b/core/src/main/scala/org/apache/spark/util/StubClassLoader.scala @@ -37,6 +37,7 @@ class StubClassLoader(parent: ClassLoader, shouldStub: String => Boolean) } } +// TODO: provide helpful error message for method not found exception. object StubClassLoader { def apply(parent: ClassLoader, binaryName: Seq[String]): StubClassLoader = { new StubClassLoader(parent, name => binaryName.exists(p => name.startsWith(p))) From 31adf156203ea3659372a8d8cfa4f56d4d50a100 Mon Sep 17 00:00:00 2001 From: Zhen Li Date: Tue, 25 Jul 2023 08:30:45 -0700 Subject: [PATCH 4/5] Fix after review --- connector/connect/client/jvm/pom.xml | 7 --- .../spark/sql/UDFClassLoadingE2ESuite.scala | 17 ++++--- .../client/util/IntegrationTestUtils.scala | 4 +- .../client/util/RemoteSparkSession.scala | 15 +++---- .../SparkConnectArtifactManager.scala | 5 ++- .../artifact/StubClassLoaderSuite.scala | 45 +++++++------------ .../spark/util/ChildFirstURLClassLoader.java | 2 +- .../org/apache/spark/executor/Executor.scala | 8 ++-- .../spark/internal/config/package.scala | 6 +-- .../apache/spark/util/StubClassLoader.scala | 2 + 10 files changed, 51 insertions(+), 60 deletions(-) diff --git a/connector/connect/client/jvm/pom.xml b/connector/connect/client/jvm/pom.xml index 29146d41396d..6605496a1654 100644 --- a/connector/connect/client/jvm/pom.xml +++ b/connector/connect/client/jvm/pom.xml @@ -221,13 +221,6 @@ org.apache.maven.plugins maven-jar-plugin - - default-jar - compile - - jar - - prepare-test-jar test-compile diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UDFClassLoadingE2ESuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UDFClassLoadingE2ESuite.scala index 9c84cd792444..d86144b21792 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UDFClassLoadingE2ESuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UDFClassLoadingE2ESuite.scala @@ -46,7 +46,7 @@ class UDFClassLoadingE2ESuite extends RemoteSparkSession { .range(10) .filter(n => { // Try to use spark result - new SparkResult[Int](null, null, null) + new SparkResult[Int](null, null, null, "") n > 5 }) .collectAsList() @@ -58,17 +58,20 @@ class UDFClassLoadingE2ESuite extends RemoteSparkSession { addClientTestArtifactInServerClasspath(session) val ds = session.range(10).filter(n => n % 2 == 0) - // load SparkResult as a stubbed class + // 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() + 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)) } diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/util/IntegrationTestUtils.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/util/IntegrationTestUtils.scala index 01d1c1204382..3b88722f8c34 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/util/IntegrationTestUtils.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/util/IntegrationTestUtils.scala @@ -29,8 +29,8 @@ object IntegrationTestUtils { // System properties used for testing and debugging private val DEBUG_SC_JVM_CLIENT = "spark.debug.sc.jvm.client" - // Enable this flag to print all client debug log + server logs to the console - private[connect] val isDebug = System.getProperty(DEBUG_SC_JVM_CLIENT, "true").toBoolean + // Enable this flag to print all server logs to the console + private[connect] val isDebug = System.getProperty(DEBUG_SC_JVM_CLIENT, "false").toBoolean private[sql] lazy val scalaVersion = { versionNumberString.split('.') match { diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/util/RemoteSparkSession.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/util/RemoteSparkSession.scala index 869eabf55420..b0cca4592d21 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/util/RemoteSparkSession.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/util/RemoteSparkSession.scala @@ -96,11 +96,9 @@ object SparkConnectServerUtils { // To find InMemoryTableCatalog for V2 writer tests val catalystTestJar = tryFindJar("sql/catalyst", "spark-catalyst", "spark-catalyst", test = true) - .map(clientTestJar => Seq("--jars", clientTestJar.getCanonicalPath)) + .map(jar => Seq("--jars", jar.getCanonicalPath)) .getOrElse(Seq.empty) - val jarsConfigs = Seq("--jars", catalystTestJar.mkString(",")) - // Use InMemoryTableCatalog for V2 writer tests val writerV2Configs = Seq( "--conf", @@ -127,7 +125,7 @@ object SparkConnectServerUtils { Seq("--conf", s"spark.sql.catalogImplementation=$catalogImplementation") } - jarsConfigs ++ writerV2Configs ++ hiveTestConfigs + catalystTestJar ++ writerV2Configs ++ hiveTestConfigs } def start(): Unit = { @@ -198,9 +196,10 @@ trait RemoteSparkSession extends ConnectFunSuite with BeforeAndAfterAll { debug(error) throw error } - } - addClientTestArtifactInServerClasspath(spark) + // Add client test jar into the spark session classpath + addClientTestArtifactInServerClasspath(spark) + } } // For UDF maven E2E tests, the server needs the client test code to find the UDFs defined in @@ -214,8 +213,8 @@ trait RemoteSparkSession extends ConnectFunSuite with BeforeAndAfterAll { // So we skip building or finding this jar for SBT. "sbt-tests-do-not-need-this-jar", "spark-connect-client-jvm", - test = testJar - ).foreach(clientTestJar => session.addArtifact(clientTestJar.getCanonicalPath)) + test = testJar).foreach(clientTestJar => + session.addArtifact(clientTestJar.getCanonicalPath)) } override def afterAll(): Unit = { diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/artifact/SparkConnectArtifactManager.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/artifact/SparkConnectArtifactManager.scala index a64c33415894..4acd719738aa 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/artifact/SparkConnectArtifactManager.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/artifact/SparkConnectArtifactManager.scala @@ -164,7 +164,10 @@ class SparkConnectArtifactManager(sessionHolder: SessionHolder) extends Logging val urls = getSparkConnectAddedJars :+ classDir.toUri.toURL val stubClassLoader = StubClassLoader(null, SparkEnv.get.conf.get(CONNECT_SCALA_UDF_STUB_CLASSES)) - new ChildFirstURLClassLoader(urls.toArray, stubClassLoader, Utils.getContextOrSparkClassLoader) + new ChildFirstURLClassLoader( + urls.toArray, + stubClassLoader, + Utils.getContextOrSparkClassLoader) } /** diff --git a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/artifact/StubClassLoaderSuite.scala b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/artifact/StubClassLoaderSuite.scala index 74b3e3c4306e..1c0442854de7 100644 --- a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/artifact/StubClassLoaderSuite.scala +++ b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/artifact/StubClassLoaderSuite.scala @@ -61,11 +61,8 @@ class StubClassLoaderSuite extends SparkFunSuite { val stubClassLoader = new RecordedStubClassLoader(null, _ => true) // Install artifact without class A. - val sessionClassLoader = new ChildFirstURLClassLoader( - Array(udfNoAJar), - stubClassLoader, - sysClassLoader - ) + val sessionClassLoader = + new ChildFirstURLClassLoader(Array(udfNoAJar), stubClassLoader, sysClassLoader) // Load udf with A used in the same class. deserializeUdf(sessionClassLoader) // Class A should be stubbed. @@ -77,25 +74,22 @@ class StubClassLoaderSuite extends SparkFunSuite { val sysClassLoader = getClass.getClassLoader() val stubClassLoader = new RecordedStubClassLoader(null, _ => true) - val cl1 = new ChildFirstURLClassLoader( - Array.empty, - stubClassLoader, - sysClassLoader) + val cl1 = new ChildFirstURLClassLoader(Array.empty, stubClassLoader, sysClassLoader) // Failed to load dummy udf - intercept[Exception]{ + intercept[Exception] { deserializeUdf(cl1) } // Successfully stubbed the missing class. - assert(stubClassLoader.lastStubbed === - "org.apache.spark.sql.connect.artifact.StubClassDummyUdf") + 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 - ) + sysClassLoader) // Should be able to load after the artifact is added deserializeUdf(cl2) } @@ -105,37 +99,32 @@ class StubClassLoaderSuite extends SparkFunSuite { val sysClassLoader = getClass.getClassLoader() val stubClassLoader = new RecordedStubClassLoader(null, _ => true) - val sessionClassLoader = new ChildFirstURLClassLoader( - Array.empty, - stubClassLoader, - sysClassLoader) + val sessionClassLoader = + new ChildFirstURLClassLoader(Array.empty, stubClassLoader, sysClassLoader) // Failed to load dummy udf - val exception = intercept[Exception]{ + val exception = intercept[Exception] { deserializeUdf(sessionClassLoader) } // Successfully stubbed the missing class. - assert(stubClassLoader.lastStubbed === - "org.apache.spark.sql.connect.artifact.StubClassDummyUdf") + 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 - ) + cause.getMessage) } private def deserializeUdf(sessionClassLoader: ClassLoader): UdfPacket = { - Utils.deserialize[UdfPacket]( - udfByteArray, - sessionClassLoader - ) + Utils.deserialize[UdfPacket](udfByteArray, sessionClassLoader) } } class RecordedStubClassLoader(parent: ClassLoader, shouldStub: String => Boolean) - extends StubClassLoader(parent, shouldStub) { + extends StubClassLoader(parent, shouldStub) { var lastStubbed: String = _ override def findClass(name: String): Class[_] = { diff --git a/core/src/main/java/org/apache/spark/util/ChildFirstURLClassLoader.java b/core/src/main/java/org/apache/spark/util/ChildFirstURLClassLoader.java index 6b6e5cf18ab3..2791209e019b 100644 --- a/core/src/main/java/org/apache/spark/util/ChildFirstURLClassLoader.java +++ b/core/src/main/java/org/apache/spark/util/ChildFirstURLClassLoader.java @@ -42,7 +42,7 @@ public ChildFirstURLClassLoader(URL[] urls, ClassLoader parent) { /** * Specify the grandparent if there is a need to load in the order of - * `grandparent -> urls (child) -> parent`. + * `grandparent -> urls (child) -> parent`. */ public ChildFirstURLClassLoader(URL[] urls, ClassLoader parent, ClassLoader grandparent) { super(urls, grandparent); diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index da5840de2844..9327ea4d3dd7 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -554,7 +554,8 @@ private[spark] class Executor( taskDescription.artifacts.jars, taskDescription.artifacts.archives, isolatedSession) - // Always reset the thread class loader + // Always reset the thread class loader to ensure if any updates, all threads (not only + // the thread that updated the dependencies) can update to the new class loader. Thread.currentThread.setContextClassLoader(isolatedSession.replClassLoader) task = ser.deserialize[Task[Any]]( taskDescription.serializedTask, Thread.currentThread.getContextClassLoader) @@ -1076,7 +1077,7 @@ private[spark] class Executor( } else { parent } - logInfo(s"Created or updated repl class loader for $sessionUUID.") + logInfo(s"Created or updated repl class loader $classLoader for $sessionUUID.") classLoader } @@ -1155,7 +1156,8 @@ private[spark] class Executor( } } if (updated) { - // TODO: only update the class loader if the stub class should be unloaded. + // When a new url is added for non-default class loader, recreate the class loader + // to ensure all classes are updated. state.urlClassLoader = createClassLoader(state.urlClassLoader.getURLs, useStub = true) state.replClassLoader = addReplClassLoaderIfNeeded(state.urlClassLoader, state.replClassDirUri, state.sessionUUID) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 77b20084378d..06c512cee9da 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -2552,10 +2552,10 @@ package object config { ConfigBuilder("spark.connect.scalaUdf.stubClasses") .internal() .doc(""" - |Comma separated list of binary names of classes/packages that should be stub during the - |Scala UDF serdeser and execution if not found on the server classpath. + |Comma-separated list of binary names of classes/packages that should be stubbed during + |the Scala UDF serde and execution if not found on the server classpath. |An empty list effectively disables stubbing for all missing classes. - |By default the server stubs classes from the Scala client package. + |By default, the server stubs classes from the Scala client package. |""".stripMargin) .version("3.5.0") .stringConf diff --git a/core/src/main/scala/org/apache/spark/util/StubClassLoader.scala b/core/src/main/scala/org/apache/spark/util/StubClassLoader.scala index 40f5cadeb0d8..193582598acd 100644 --- a/core/src/main/scala/org/apache/spark/util/StubClassLoader.scala +++ b/core/src/main/scala/org/apache/spark/util/StubClassLoader.scala @@ -33,6 +33,8 @@ class StubClassLoader(parent: ClassLoader, shouldStub: String => Boolean) throw new ClassNotFoundException(name) } val bytes = StubClassLoader.generateStub(name) + // scalastyle:off println + println("###stub class: " + name) defineClass(name, bytes, 0, bytes.length) } } From 0290c586b07f59e546e445390f7a5d343c2f7a6c Mon Sep 17 00:00:00 2001 From: Zhen Li Date: Wed, 26 Jul 2023 16:22:42 -0700 Subject: [PATCH 5/5] Make the tests runnable for sbt too --- .../org/apache/spark/sql/SparkSession.scala | 2 +- .../sql/expressions/UserDefinedFunction.scala | 2 +- .../test/resources/StubClassDummyUdf.scala | 7 +- .../client/jvm/src/test/resources/udf2.12 | Bin 0 -> 1520 bytes .../client/jvm/src/test/resources/udf2.12.jar | Bin 0 -> 5332 bytes .../client/jvm/src/test/resources/udf2.13 | Bin 0 -> 1630 bytes .../client/jvm/src/test/resources/udf2.13.jar | Bin 0 -> 5674 bytes .../spark/sql/UDFClassLoadingE2ESuite.scala | 88 ------------------ .../client/UDFClassLoadingE2ESuite.scala | 83 +++++++++++++++++ .../client/util/RemoteSparkSession.scala | 36 ++++--- .../SparkConnectArtifactManager.scala | 21 +++-- .../connect/planner/SparkConnectPlanner.scala | 23 +++-- .../server/src/test/resources/udf_noA.jar | Bin 4396 -> 5545 bytes .../artifact/StubClassLoaderSuite.scala | 58 ++++++------ .../apache/spark/util/StubClassLoader.scala | 3 - 15 files changed, 163 insertions(+), 160 deletions(-) rename connector/connect/{server => client/jvm}/src/test/resources/StubClassDummyUdf.scala (90%) create mode 100644 connector/connect/client/jvm/src/test/resources/udf2.12 create mode 100644 connector/connect/client/jvm/src/test/resources/udf2.12.jar create mode 100644 connector/connect/client/jvm/src/test/resources/udf2.13 create mode 100644 connector/connect/client/jvm/src/test/resources/udf2.13.jar delete mode 100644 connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UDFClassLoadingE2ESuite.scala create mode 100644 connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/UDFClassLoadingE2ESuite.scala diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index 5a0f33ffd5dc..cc6c8100d3c6 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -554,7 +554,7 @@ class SparkSession private[sql] ( val command = proto.Command.newBuilder().setRegisterFunction(udf).build() val plan = proto.Plan.newBuilder().setCommand(command).build() - client.execute(plan) + client.execute(plan).asScala.foreach(_ => ()) } @DeveloperApi diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala index 18aef8a2e4cf..e5c89d90c19b 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala @@ -92,7 +92,7 @@ sealed abstract class UserDefinedFunction { /** * Holder class for a scalar user-defined function and it's input/output encoder(s). */ -case class ScalarUserDefinedFunction private ( +case class ScalarUserDefinedFunction private[sql] ( // SPARK-43198: Eagerly serialize to prevent the UDF from containing a reference to this class. serializedUdfPacket: Array[Byte], inputTypes: Seq[proto.DataType], diff --git a/connector/connect/server/src/test/resources/StubClassDummyUdf.scala b/connector/connect/client/jvm/src/test/resources/StubClassDummyUdf.scala similarity index 90% rename from connector/connect/server/src/test/resources/StubClassDummyUdf.scala rename to connector/connect/client/jvm/src/test/resources/StubClassDummyUdf.scala index 16953a482d11..ff1b3deafafd 100644 --- a/connector/connect/server/src/test/resources/StubClassDummyUdf.scala +++ b/connector/connect/client/jvm/src/test/resources/StubClassDummyUdf.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.connect.artifact +package org.apache.spark.sql.connect.client // To generate a jar from the source file: // `scalac StubClassDummyUdf.scala -d udf.jar` @@ -31,6 +31,7 @@ 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.catalyst.encoders.AgnosticEncoders.PrimitiveIntEncoder import org.apache.spark.sql.connect.common.UdfPacket import org.apache.spark.util.Utils @@ -39,8 +40,8 @@ object StubClassDummyUdf { Utils.serialize[UdfPacket]( new UdfPacket( new StubClassDummyUdf().udf, - Seq.empty, - null + Seq(PrimitiveIntEncoder), + PrimitiveIntEncoder ) ) val file = new File("src/test/resources/udf") diff --git a/connector/connect/client/jvm/src/test/resources/udf2.12 b/connector/connect/client/jvm/src/test/resources/udf2.12 new file mode 100644 index 0000000000000000000000000000000000000000..1090bc90d9b4bedb635146d0241b5a3fbe3da114 GIT binary patch literal 1520 zcmb_cJ7^R^7@j057vqJ~KnOkxWqlmP9pZ~Xun2j`=|%$=MU2hd?A&FN+1bs^?0FYK zE1Ogn7D5WK5kVVEYpXP3V{M_0g_VVt{<9BrCZ~e9#Vt4c%{Slw&G-K=zo6Y(q0^yi zxFjNUiMcu=YQ@#7!lj{A{+QxZ5g&sc&Nx_6q^}IL!T*a(6FZ|5d_mv2tdF_TxF{k7{+D`Y0$U_qHrn- zC!#Qj!ju>y+S*{2QR2yl47K4jJ7c01K}AQc6xT{|xon0j%&juTiQsF@_efClNl4y3 z8ZBRV(2Mp~&>#;YF~`hO=wIQ|Uqt&TiA=1RKdBm^dT6m*ZG$O@%f^%-8esc7P{|V1 zHIf@dL`52=>V&ck-f39XDBuD7kuLaD*;Nv-Rdf#(Jap{0pek=jVwGpeep^{`UbfIt zY6s!amI2!thPl*+NW+O?eyU`l9i{xO2qCm5R=Gi`XPfx+E)&JEn;UyhDURgAS3Q1nZsHuRCMb&{FaPfjg6Ef?mmE?eZXq4XFv{ zdR!YP&mXawpI2e4ot4znXZ-;R8~83=m<8U=1MhANymsal54hn?1_8^uz&ZE0f8$l* z(jja9|Dg65sb}0DK3*Jn{k61_QcwM>>JG-|{LR{Waq|vLlv`+k>z@UAXb%;7fiw&n z$p(jFbvrn45)k+TEmQEpBb!Nsj(49L20m&$oxxy4^nl;x^vW_qcaBJ2XWGmXYC_d^ zz|4Fpzp1q1!@usEll_mZ)^?lxQ$z>v6u1K7FzI i>Hf2%g<#+4Us9HbjLk^jS-(vezf}i50n z>b=sZSElSj;baCEcF>BF<;y|v5?bWoGjRoYQPy`dxcmCj8tBY=L`MY7X&AgeRbSy= zFsx_fKjm{>3h~Jonj-pe{CIsUz*Gyoliz z)d0T!RA{?u@cEY+Kg^hA; z3#GRypUXpy39;ip#SMohYhQXxhHEB#kijm-H{W;7%g*n;?+bakcu2k0svA$_F^X1> z-J~~8LSQ_kF+a_<7u>v!DPT|!rioUP^@Tc{dvm+Jx%V<87CH@m9xE#K_G-4|oa60jTJtjtwuz6g6|d7pPDht! zUu%)+$_PPurrIoa=^8I|s_i1ux#D_-Nbl5GJ*vUz5f=jU*~MK@+Up{zM_5mP)Bv z804n--unk)`1ugq2KFE(M+iG<3@Z60>OI7~kJoha(t zdHjHaiIDvQUSwI7gIfkwp^0^9A}0pf=b&vIdsGEk7xH@YxuMiRjiLJ4bEV(EUrvtk zI!KPvY%i-Q8I@|=fVsaE)iutZc1PtsJ=vliW+|NMd7?de9T?|+r*+6{iPd>;D-&l` zWhwf$n_Q5|N=b(k){t;>h2n_tYcsJI@ya=;5yuB|tkgEfY@OzFu*LVrXh?|}_%Vsl zZJL_5X5;z%Uuf1>?cyS{f#D%}Sb1#VsM&} z6NK<9$vrZu+L00p(WyRgzPf8uW}WNy+k*op2Soo5Ms_D(##&M}t-U(N9r;5QEF2Kh zr)9tJ>wuIc4;U9~vHvyT@u^=!y{`vF55t=u;s)~5x{N0MUdyt9>gSl$8samvegp=I z7LXs!c||(^^>Zj#=qRXP(SRpweQ~%(6rb07O%ibP-<)1++7E)`0RZob{x41k|3glf z`yUR^GWf2|Fi$1(mc?k79aN26koRwwr&aa03II8Af$3l%&?u^90pEs`U;ar26pX9- zrQu3Y_-aZpYt-#W&u&9_ar*We{5pU9!K(=qe&Jialfo5Md!3sn7g?@*(DQ4|yPsfy zq`xTzZP`rZ^UAV~vlH9=vRRpMG5ZbME(=<(<*08XoJ&IaWq=@LH12LAH9uSl(v~5C ze09qJXHN+VhNHPo)N)l*J#d!irb3{Cb{kX0MpYF=tYj^X=3=hlb#u0o_RP!(Nq+27 zJqL&wX2k36TmAIPadJ_q)YjIzzG{N}t1DO0GOO{VoQKK1sPAHj1FA@k1!hMTouv*B z6&RX9?`PiKDiDX9y_$jhY6(~buBpp;_vM;Plsx_-+w*wO;#hJ;an|agD@bTwiDY8L zd;@c@kcs4L&KCWkO>LK*u7&M2n4xOiEHY}R7RyK|Zk(IS#Tj=B^*xFSZH<%R_=Pm^`PnTf%v0|{PS#ik6o|sEX ze)si!(nbFh0#UYuBUa?MiQT3>l25(`IW__R1>K-6$QSYe$F5XcM&sX~a^z zTm5;L63#y^%0+`7LOqp>mvGrLO;o@fpT~aH-9w>iYEjX+^EDS6K$LwxbS(M;Zl`Ft zZ5^DeB&?Q3ksMpfc5^JZJuNG1$Q1YUZauY@iN&CW;9+TJ)TB&=i^Gbpq!}qKlx=s;(q#C3omvLI=~uwiRxtf;&7KEv?<`%P*99 z7PA}kF!|W_^=S7bYP{xFrFS2yj3L5Bq&EzrkKS?@LK#q<`5d(hK(Ge)zgY8R@z76% zU+83O*d(C!3y$2DA=1CF(0w-%zM>RkNfAj;{g`CV5K9}8abro|TE6|cx}b8;LdWP# zyTf*g8Dq`%gygP8<`n0(2&dK7?9|Y;9gna(7cr8)+G+cv=wWq)NlKe)4&3qCTVSmI zo3J#uw258`*VbM4bXvbrl$O$SQ@8r8RU2hQDn)A+a@0sX_il*#3E3NAed1$LsR=H} zI>T_G_Gx~}PNt{b_nO3%b}N9meKL_Ji%6WAp&*4MMMS!Gopb_i3xA|U<}|7tM_Zw* z+-`NIc@%csKLbr|+4aebIP6b0<8G5$c5_PC-dp;(GyVdp{wDk$`6e@_eY!_%S<*I13B0-yuNv`DY6+m(i&;;5mjN4)FJAQ!KkW$;sgCCUeis-AW z+W`-ccCefr##)GOYoSbDVUHd(r{v2(1ArpUmw;^a0RiOFXQRf*ZtF(D*mkpy149856cViDU6_dGnyKe z@RMvaZHJ#G3I(L{kVPus1>*TWgQtv{5u0LW=OQmaU0&)7qD#34bfXN`oiIwC)G`Wd z!WIX+MyN14b`T+ua-u*Sj?^$RqT#MDWA;7yB>1z)JcZ=Mw!@|3;aujxeNEswS>-10 zU}kcX`Ho_zX3pl|0M?yDE4T&7A{U7Ky0{zY*YW9r{@H4B(Q{iq=1uo4>K*|D658@G>)m6X9r3#>WjeO4Zw|iDQ?qe0KA=i_3jq6}5uf`t# zyyDL)PqAfJ*^lHeWs1;k;Kc{4;k5ypBKj%@p(t%~F>|8&R|uVx-o&Hlv53>aPLa1% zgfy3oO`Hv9j6X-=0~VE~$rH6!$}CFc`&K6_U*_pA!qwz#x(C~Nc(ZZM?5%k)WPs`j zEE~VJbdnl+`J;kdAfuUn`>7g!=c}DqbP`cSWwbBOYWbNea|ah%Yb=vW72_TAf|~N8!<{-6 zKkJn~?=_hf;!Wy~klTIP7$-cub>9lh#=N<0!axZeT3G{d<(PU2teAp?!;YD7_!!qJ z7|uX3<1)RwLk}k}sJA9S(!^?h5%)T8@>^6ej4d9}(lZ8HTb{*UM|IkX?GY*9gqN0P zGEU0W-=XKAmWQJ?k`3HzRioo2e-@T!}kAd3f6m!BDaxk_ph79)zeVjq;#5-PL zMP{G$nCp`b_puv24LXZWJI0^)ODcz*^~J3|WgLZd_bY&QYafMN68Uxyq6zlj!6H_f zB)u+}_+DLLL=&<{KZ7bi(Cyauo}zSR{|?7R<8QI_TG3 z$ZA6CnOCmSBZ!)^=sr9jyd0A8Z~YLW2lJ5-W`4gY#j;o~R(`~-N!76&yLrVgaQf}) ziv+hxe_ivKOIg+@S6b5Q_woMdfl~%XPou{8cuddw&@iq1LX=(nV4LdGa#_5sKFY2YQruXjlI50{}zoRg+8*X=7Vr_`UY z{Kf;!%Wl}TH8F2Q*v*8{-JjI-@o$}vL&0~~>@rV{9Yjd{%u6E<@M4%+cvhy-vU7gn zL>~RJkewx7Q2BJ_K!ZWU_#JS)-n7AqQ*zF%pI%;tE67h#T)yar=+D?~9a2K41_A)` z$p5bvjqjhfXdM6Q(R5!uqmU!>YvNBzVJB46)d3PUD2kI))rO@!tfUm8RK0AGnsaek zDnFB-G{sBvkMFC#f6cvyJ&~oH8Mxo~Hpo zgx^o4dewTFLgvyRa^R}n3TWqWc%L~y02Nb=$vtDC%MTNBLoCxPiYU~rdu`AqN{mMHbOEL4&zcXi8IO`emB zDKzu;JhKkS7LJjPToViX(tm8X<8HS<5I;bR(n2XZG~$_>lE+v%hGQ~0{q;hl2TN+1 zr=+C7Cw4Rnw(z8a!*%q^H8LY9qtDV^UDmfj*gCvdAV#(2Gvl|xUMH=!M&v45Db zvDlXOa7Nx+b+(dqyDmJ?#}dtdC`HQ|qmi>*$W_#k8*7l{0KU7PUT8>TcoNCX?&g1L z^3H{s>39a$F;SOoWxR?>m~9ljfNig`L<~IfrK9n2D=Nt4Zajf_D!?p5TQ1u+0s0g8 zTU7*jfF1sDc*Awna^_a%6Bupq>D>%@lRH|8C{yw!)Yoygsp`-($R#MBprv^&(Q_}h zVuSQ!!r;e*EtId3;Q5RrBj;{@0h=V!Bd1p-M;=B>@h*|=1;O4>B5`bt72)i&G|q@; zr}0iYA&f=uhU1Rt8kuMu`OGLsn9c5AkW;k3%XJT;2aVGW5Mi3k0vu$nv_-+T-5&NH zeKU*c_PC~AA$fVnxPMBsIYS2_hYJ7@5d104TR4<}ztpwAt7U)Gwf~L(q_zFKm%qYKIQh%X`^dCC@7iDRw%Rb*F}c$K literal 0 HcmV?d00001 diff --git a/connector/connect/client/jvm/src/test/resources/udf2.13 b/connector/connect/client/jvm/src/test/resources/udf2.13 new file mode 100644 index 0000000000000000000000000000000000000000..863ac32a76dc9ca057e7e831ea3290b33bbadb93 GIT binary patch literal 1630 zcmb_czi$&U6n1~m7DB2(rK%t*wbg+r0$T-vDiwyRKcv7FDy2w>PL9t>-TLh7*_S3o zVq;-qU}J!hft7)oi3PDRGcYkQ@dto?=WEg?WkB4JALsAA_r1^Gd+&aN6DkJRWL$?h z!eom=72$XXs$C9=6oL|MzQa%o_%NugW3of_lVa(|IJxn)9}M_lq?QPxnG_nF^|x^Y zBaTHKJ*aM*&;=h9nTQg-B8UuVtTdSN6~P=MA~`n*6Np$CCK^{cMHRYhLu8`cF&fPK zh91i3p_Cpn^bo-s^On+xy5wlFE~L_oxM~Z{$n-XCNkK~rGSQGb6jo@=FlYNT@NrlT zu*6?pUY))5tRI{zgHaYne2wZY8Qfzc*aW8uj&u^!U@@*6{$rcHx*W|wOw^Y+Rt9#a z1C_oqeNE(2VO|!R#x+c+gLg_-(Xq*6@RL388MFI1q`TlL81cc(v7id83v8J?A%=n0;Bv2O#4MsgIq%hKZNl^3Vte~az!x-**%E?SkZg5zq z!dQ;LWm>}t*SY_&b^m%Sn=RMUzyRp;!Lad#mT^mi^WC$unw@3VzS32xY;0vG$-%r0 z>3-!E`VUu_>8@}jMOG=(89JTenhX+7kL{x`tx^b6)PduMn~}NpsV>e3;}3nNwAW^8 zys|g=+dRrjd(H&gMbKkUxkUi;MF5X10_Cm8EM%HBs2M!B9lQ%O`?udr7v5V_{*Tl? z#Wk1v+xI(#55G#UveZle%DN-tTlR7PdGXVp*}0EFfvJOrd@w|~+Q4-UCfsGlv}RW; wN|iS!W|fQtMk70`8eHtXO|>~xXxYK+t>>bcxGee@Mim6vO5WV~L7e@bw53r9m^M0v_{Y@By+ zM44AsiF%icdf)ii?ZgcYprwlSVC#j9lHeaDp8RpN{$pW@KVF^PZ1|uqP)l1YK6e+W zn**P_E1b{L*~!Vu(u2!Kdrt36q0E-QDFq9UXo2Us^x1{2jF${4ks}pbT(M z7AaOgBmlt`YP5_@17Bhg5n4IKyr_AvB~P^U$#H^MJvpOL&8w;U8tcsY0xDr6;k)?c z8vX3q;eoo2mYo&kx`tqM)E~9kI)2?T-o9%Q*m+L^sFDlms(^~f&*wM#D>Rr)z2-Ej z7OJ)0Y166i%P8h*<25+&YAn=7nmCDOMi6X?V}(lp!hpQbS3C{(?z{r7(ei`BKX%Pl zP)~Eb`C8}qFhk|J`ylsNMZSoKeY3h%Vo;~AEjKw+J=*=@u58|55}Yg3MmdV7klehs zP$d`)jqY{9E$VHHvJ)`%vm0iBw|t2ROy#=3FWgM^j^4gCygZp;*-(x7Bwt^EsuMqz+b~J9slu-+!f^oh zsGF2CLkY+63ttOj`H2oEun%g*uEeFL5RNa7RTt@4A)q%*Dmh|4bw*c11bEt~=o>}M zRorJCOe;lg6`Tfba7w-fUVQI^zFjSRSrCvu2QZ1)2f?z>DjQBSDyewXd;Lq&5t zv?FSY2Fqz;c~fs!M&sLFSyF4bD;YhiWE=(XiIbu&7#Lq_lZd2czB0qDoThji;@U9E zt8aM7s@WY6uuB7XgVgd89xqFhfXM5#TR6HNn1NxKhRkR!!>j3`1H<0$gpm!#shrd) zGj^jQBXam1y(u5814$nq>lQE0VqUa-C0o!t?O)$EUwjJZjPYB7410BI^QUCiXP%^ptoTb;zK>KmHDGjCMwbZb zpOc`Neuut!`~3y6wHK+pmTF>djE)L>l3>r>A~ zkf{pjmwi~_&M}^7(e#_ka|=y^7**pCQ(o(}V?DpC0b!YfF3O;D2uB_G?)ZHO%J<7% z3|{jZl#r_?WYjlwpxNs&9-0?aFy7NY_#16tgzE|dOckMIJe?ZnRbQ62&``#yae~J) zg5P)Ux{R#|Q6MN)&Wx^1^OnUSrXKLnl)A~1tARC)X4K`2ANL{*r|G73PHuy%Xhb~C z61r?pN(<@P!{uK%4kG5ow&F`W zp8~^)h-`XYosL8)ulmAXhs*}isC6}alSu-=JwCLM40p#}`DD+f z@)Zl-_Ok4wb9yJ>4ZfUr{raNW#cA4;qnSHNa}vUSvhd@(9~TN+8vO3)Qs`I>SmZkTfr}o>{Wr5cfh&h9_X=nA0Fj?2@n~%>1j_vzFyx z@pZ~hf;+U|9Bx=8m}rUt0GQ+cHxB3immDthznq^$wigXl*h{? zQUOj~ONkA3RfPjD_0j`XW47gDY9sm)q)j+_r+lhOjY_$=<< zDiAyvNp{bX2w*fIw5yWjw3S+^{98en$vTJuR~maRMp?KLa|X9nFP--UWCE6fP|Qu> zv}Md#ltneL1xc@-8GOi!ES`qBHf0TENvY7`)9itc`nkPu=Ajur%dmRhTaI$XFot6^ zAXkjfP62kcID*;&Ma@a}(lJF@%Pti=B`&TjbSvjiRv%Zt>|{yeSZfbRjoWq;Hb?Z%m!+_6%MZHE{1S;MtRzD=dX;T?W7c4g_hPI*@n6^!Qbpx+ zHjpet>8|-5xq%7I6Xy!A>Gf^(WFXw86t`vi9w*#mj~Z;Y7<{Z0oYouhN{Fe3;H|2- z4k`N&P-`!WAEuakU0gA%pdj|=i*`r+ljl*Ma))5{hBZfH^6ac^(WN2pegadU1XX?6 zK4C0uo%mlz>h9*#dtO>Tx=HV%E(f8el*=2tRR?-joWJT8AZ+rf2&LET;|E) zcS&91YIz-^KEbb?DhuuPFDvv@Y(7TWnPmF3*EEf}n=a{?8?lj3FA4~J67||4w(L+~ zhV$|DIWL?KSSVC}SRxw^6RCMAmBB7pN2zh9eY6(7lZX&67Bd%aB|Taml1cb62m5ZF z$(+Q@1*Ay1{cM?^WcZ0)YD%Op>TFha)trRPH{ZuIPKB0Cwh!|~VkJ%YWpXfvz>VB7OWd1?wqL%`on57GjZfxij@7d5B=Qf^HYTv* z8|2?&X?`+zQcE#>&?-R=)RcYl&by;V<$`r-$!*|HNUyLC3$@FYm7$BP{rHl0pP%}@ znU5S8c@@A>4}XLmHJBEsvs`sVG#l1gocMWZtx2nOGHbapv$g$`Xjbz$SE%4FZ)iMw z{3klAfsL;QCQZagcDAM#(F|4#87=Rhv4+GWcBhjGfSUU+byK6q)X9$^8>0|FYK9?ZYHm?C;Zw7ciX@x2K!QeIpm7%SH>; z6$Js@{))_dN$UaAhlwI1l~YgEDX0RHc%nOe5G5u0tdmPz)!%3F`;kahbXimYf8!@k zrW1w9Qe$^E(ygNw;WUe%i&FAAgp7R06b~6w>i51GUuAX4TihtNzZ$l1jB~JIg}m!C zXt}w!dm7of&{bmnh1!4Zn&?dqjqxw+)oxmY8Eb~V?bjeD%1Gy7%*s{rJPfCW_f$^C zCu4H&&%=FQoL2|WtWcm7u%ctt-9AH2Si zPKN^wvww!rVAfYM&UnUC>-7@Km(hM(g}gPscu+sZD9eUNo#AXicP!XeJ^or90ZZ%2y0A?f@+f@U*tf6{ri3Iny=7Uns@r`P+XLvYX#3xi4Uw=0($j&(Swxjx6=v+)CB26)EJzCBzA!2T6bL7q_1nxfd*kKT&691#(# z5Fw%@(9OC=iJJ*6T)Tz|D2|YbSs7WjG|54b8Z-0E^EEO}KN%wwRa>KRf$Q}@etg^$ zReWA5!Xcf=6e2I^07m5Fovj67oF>+ag3EGEKZE$Rdj~|&oWyOi9aH@980sreEf*j1 zC}6(C5A)KyVeYDa#<2}1 z?u+;E`;F~HT=}yyAdXa@F~L)IF_SystE0OD!apzckY-)U>lq zQcu5lY|t}uBE!Dn`>UF=;q)r^;b|coXjOC%=NQL^!nMahu1iIs@ny0RxiixpX2qd1 zbOZ0pIQYJ^+kxbZ^ZT$!6!0jwzw~rUYyZt9vUHTD)Ny7|=qH3-O8T8<6>`}vOw+U_ z+_B7_Jy!608p8#JOWd1M`Tkkv4S}?hABtz-rxhksI`qWt z)z06tl;aWG?Nb~8V3Xv($x@zwD@$4a)TR24x-|Nv0ZxoO#)zLVRrOi}(|NYfUNQOl z=(a)mwb+cR>7SiNk&bg-?xIddGph_g9m)mOpIP7R#YzpIEH3e4WX{B<5BWbi&)UvH zWw|GH-re020Al8>T8Ki`Gn-reI1V&2BN7hB5;_hsb#M&7RsF#Ja4?*O)=xMi{^-1w zPEp5)7ivgNN9JbeC_kZZ&PaiAS(`SggQKR|I$k5WuoCzzEC;R=8n({qM$EZZJIpaA2riea z?s>a)*c8~S#ibX6PW+7UdH6m$Cryid)dN4xi0hmvV>7mEE?1(VsnaF2Ar6X3>pA=V z$6B=K_qkTNImS#4c}I(M;kEMaf^UOO-l_($WJi#Ydo4ZI`lU+Og zLSj#$o^s6DGx{dRHoLgDYn$bg^H<6okz?-1^_AOX-+~iJranY0^0n(@5H(x^-1ZC; zA=Jwm>di9YVyMQm!42GhrzKUL&WBzVx^400<*FF5VZDmnB zW7^JD7+5L$@}tQxL3?5OmU1lPOi;=;eaWYt931w2SvpLt1<6g`Jb5K_p(Bo?r&TDg zvLyFU#hY9>yKr$C(l%JnBmqhM_ew6iGzI;qVB0eoHaSN>k1g{~=4^qQR0`&yWsQS-4AJ}>WLme-vNE32*Qz49Ns62a#FGM}l4+G30TMsU z46ccZVkFD-vfDt~Qtsi}?HxccN(bk?rjJ?Pxw^QGCw;QVC4hlAL-CeLVRZaHYO$K* zCisJ8%Gq}A#tiQ3i5?rx4VR|%DM9O{E(&LtbSc+#cJrf&YcJZ>>33DtiTmIV`6pYj zL>J*mpV+P2T{QxW&NyPhH@Ovrtf?d_kCmH>E-B``C5__AR04 z06+-V@99PZk^=tnEdOpQ{`M^YFaDE*`OhqWUu%CGkblAFkL&-#j{Ik#zZ2E(E%`4% vF#lblKX>MTCjNVme)HU4m?Hd_iT`9hEme#^z5@88mmmNP4FISk{T=-en%_^_ literal 0 HcmV?d00001 diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UDFClassLoadingE2ESuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UDFClassLoadingE2ESuite.scala deleted file mode 100644 index d86144b21792..000000000000 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UDFClassLoadingE2ESuite.scala +++ /dev/null @@ -1,88 +0,0 @@ -/* - * 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() - 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, "") - 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)) - } - } -} diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/UDFClassLoadingE2ESuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/UDFClassLoadingE2ESuite.scala new file mode 100644 index 000000000000..8fdb7efbcba7 --- /dev/null +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/UDFClassLoadingE2ESuite.scala @@ -0,0 +1,83 @@ +/* + * 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.client + +import java.io.File +import java.nio.file.{Files, Paths} + +import scala.util.Properties + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connect.client.util.RemoteSparkSession +import org.apache.spark.sql.connect.common.ProtoDataTypes +import org.apache.spark.sql.expressions.ScalarUserDefinedFunction + +class UDFClassLoadingE2ESuite extends RemoteSparkSession { + + private val scalaVersion = Properties.versionNumberString + .split("\\.") + .take(2) + .mkString(".") + + // See src/test/resources/StubClassDummyUdf for how the UDFs and jars are created. + private val udfByteArray: Array[Byte] = + Files.readAllBytes(Paths.get(s"src/test/resources/udf$scalaVersion")) + private val udfJar = + new File(s"src/test/resources/udf$scalaVersion.jar").toURI.toURL + + private def registerUdf(session: SparkSession): Unit = { + val udf = ScalarUserDefinedFunction( + serializedUdfPacket = udfByteArray, + inputTypes = Seq(ProtoDataTypes.IntegerType), + outputType = ProtoDataTypes.IntegerType, + name = Some("dummyUdf"), + nullable = true, + deterministic = true) + session.registerUdf(udf.toProto) + } + + test("update class loader after stubbing: new session") { + // Session1 should stub the missing class, but fail to call methods on it + val session1 = spark.newSession() + + assert( + intercept[Exception] { + registerUdf(session1) + }.getMessage.contains( + "java.lang.NoSuchMethodException: org.apache.spark.sql.connect.client.StubClassDummyUdf")) + + // Session2 uses the real class + val session2 = spark.newSession() + session2.addArtifact(udfJar.toURI) + registerUdf(session2) + } + + test("update class loader after stubbing: same session") { + // Session should stub the missing class, but fail to call methods on it + val session = spark.newSession() + + assert( + intercept[Exception] { + registerUdf(session) + }.getMessage.contains( + "java.lang.NoSuchMethodException: org.apache.spark.sql.connect.client.StubClassDummyUdf")) + + // Session uses the real class + session.addArtifact(udfJar.toURI) + registerUdf(session) + } +} diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/util/RemoteSparkSession.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/util/RemoteSparkSession.scala index b0cca4592d21..1c1cb1403fee 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/util/RemoteSparkSession.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/util/RemoteSparkSession.scala @@ -96,9 +96,23 @@ object SparkConnectServerUtils { // To find InMemoryTableCatalog for V2 writer tests val catalystTestJar = tryFindJar("sql/catalyst", "spark-catalyst", "spark-catalyst", test = true) - .map(jar => Seq("--jars", jar.getCanonicalPath)) + .map(clientTestJar => Seq(clientTestJar.getCanonicalPath)) .getOrElse(Seq.empty) + // For UDF maven E2E tests, the server needs the client code to find the UDFs defined in tests. + val connectClientTestJar = tryFindJar( + "connector/connect/client/jvm", + // SBT passes the client & test jars to the server process automatically. + // So we skip building or finding this jar for SBT. + "sbt-tests-do-not-need-this-jar", + "spark-connect-client-jvm", + test = true) + .map(clientTestJar => Seq(clientTestJar.getCanonicalPath)) + .getOrElse(Seq.empty) + + val allJars = catalystTestJar ++ connectClientTestJar + val jarsConfigs = Seq("--jars", allJars.mkString(",")) + // Use InMemoryTableCatalog for V2 writer tests val writerV2Configs = Seq( "--conf", @@ -125,7 +139,7 @@ object SparkConnectServerUtils { Seq("--conf", s"spark.sql.catalogImplementation=$catalogImplementation") } - catalystTestJar ++ writerV2Configs ++ hiveTestConfigs + jarsConfigs ++ writerV2Configs ++ hiveTestConfigs } def start(): Unit = { @@ -196,27 +210,9 @@ trait RemoteSparkSession extends ConnectFunSuite with BeforeAndAfterAll { debug(error) throw error } - - // Add client test jar into the spark session classpath - addClientTestArtifactInServerClasspath(spark) } } - // For UDF maven E2E tests, the server needs the client test code to find the UDFs defined in - // tests. - private[sql] def addClientTestArtifactInServerClasspath( - session: SparkSession, - testJar: Boolean = true): Unit = { - tryFindJar( - "connector/connect/client/jvm", - // SBT passes the client & test jars to the server process automatically. - // So we skip building or finding this jar for SBT. - "sbt-tests-do-not-need-this-jar", - "spark-connect-client-jvm", - test = testJar).foreach(clientTestJar => - session.addArtifact(clientTestJar.getCanonicalPath)) - } - override def afterAll(): Unit = { try { if (spark != null) spark.stop() diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/artifact/SparkConnectArtifactManager.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/artifact/SparkConnectArtifactManager.scala index 4acd719738aa..03391cef68b0 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/artifact/SparkConnectArtifactManager.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/artifact/SparkConnectArtifactManager.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.connect.artifact import java.io.File -import java.net.{URI, URL} +import java.net.{URI, URL, URLClassLoader} import java.nio.file.{Files, Path, Paths, StandardCopyOption} import java.util.concurrent.CopyOnWriteArrayList import javax.ws.rs.core.UriBuilder @@ -162,12 +162,19 @@ class SparkConnectArtifactManager(sessionHolder: SessionHolder) extends Logging */ def classloader: ClassLoader = { val urls = getSparkConnectAddedJars :+ classDir.toUri.toURL - val stubClassLoader = - StubClassLoader(null, SparkEnv.get.conf.get(CONNECT_SCALA_UDF_STUB_CLASSES)) - new ChildFirstURLClassLoader( - urls.toArray, - stubClassLoader, - Utils.getContextOrSparkClassLoader) + val loader = if (SparkEnv.get.conf.get(CONNECT_SCALA_UDF_STUB_CLASSES).nonEmpty) { + val stubClassLoader = + StubClassLoader(null, SparkEnv.get.conf.get(CONNECT_SCALA_UDF_STUB_CLASSES)) + new ChildFirstURLClassLoader( + urls.toArray, + stubClassLoader, + Utils.getContextOrSparkClassLoader) + } else { + new URLClassLoader(urls.toArray, Utils.getContextOrSparkClassLoader) + } + + logDebug(s"Using class loader: $loader, containing urls: $urls") + loader } /** diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index e4ac34715fb2..ebed8af48f08 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.connect.planner +import java.io.IOException + import scala.collection.JavaConverters._ import scala.collection.mutable import scala.util.Try @@ -1504,15 +1506,24 @@ class SparkConnectPlanner(val sessionHolder: SessionHolder) extends Logging { } private def unpackUdf(fun: proto.CommonInlineUserDefinedFunction): UdfPacket = { - Utils.deserialize[UdfPacket]( - fun.getScalarScalaUdf.getPayload.toByteArray, - Utils.getContextOrSparkClassLoader) + unpackScalarScalaUDF[UdfPacket](fun.getScalarScalaUdf) } private def unpackForeachWriter(fun: proto.ScalarScalaUDF): ForeachWriterPacket = { - Utils.deserialize[ForeachWriterPacket]( - fun.getPayload.toByteArray, - Utils.getContextOrSparkClassLoader) + unpackScalarScalaUDF[ForeachWriterPacket](fun) + } + + private def unpackScalarScalaUDF[T](fun: proto.ScalarScalaUDF): T = { + try { + logDebug(s"Unpack using class loader: ${Utils.getContextOrSparkClassLoader}") + Utils.deserialize[T](fun.getPayload.toByteArray, Utils.getContextOrSparkClassLoader) + } catch { + case e: IOException if e.getCause.isInstanceOf[NoSuchMethodException] => + throw new ClassNotFoundException( + s"Failed to load class correctly due to ${e.getCause}. " + + "Make sure the artifact where the class is defined is installed by calling" + + " session.addArtifact.") + } } /** diff --git a/connector/connect/server/src/test/resources/udf_noA.jar b/connector/connect/server/src/test/resources/udf_noA.jar index 0d6fddee91cb0233818b456335dfc01a9ff70bb2..4d8c423ab6dfbffe0b0f065faa15de74e133e0eb 100644 GIT binary patch literal 5545 zcma)A2{_c<_y5M&C&r&FjR_Uy6^8JZ-t5`e>|)H=%f4^f*9?hDBE}L*_I*!gL`Wo> zs3>HYC5D9mOdI;O9gpYvdfapGx%YFH&pBGE;fW+txquBs*VL7apNTTT*K%5pS~#wb^6$KepC;c5ZwmkInQ}UtXrRQ*0|w zfcr&PpI-mHLkHI~WNpCX&c*QDCoLx%21Act>!DNIVv>P?N=A@^!()#lWFR<}gdzqq zhs@(v5wrV^G1C_56a(@$?ckx_Zi(cqz2(v=6Kd?N0uhl65oT#HL{cz|?UJSrZ&YeV zy5;EFmhZ+IhAS!%+&kzw7_xjr&ZltS)kV>1An7s6hbjGTM>FlGO1kn(UDwVZl7^(b zr6PG0wPZ!U%N_s#vHcEBy02KFoKY6GR=)w2461M#If6=qpomkw zxkP8!gT40RX@*q=yMqmbMR!>Mx6~y}tA|g^V+Sft=*OMddOVz?3EBg2Tr<))B5y4X zsYHr%T1*`7XB&6$<_Z)JieIu2%T4b`#WmushoD%VFtWi5v zExABnU39>vH#zARrr$2}g{@|;Re-nqMVD8cABBQ1>I5H;IF8>E&LWJlHyERaJBDfA zZAEcWhj1-kY~ml>8_E?0JA954G$5bb4jvRMZ|rWBCIX>xoxL$P!$VXNY8x!xb5}&r z7|g&4zJH7J3$g}8WCP`79EtZMW-+bv%2fu~@nc%^l2 zgzK$3vf47G{?n@v#8uhCNG!vL zH27p$Xie#>1q^59aOzRZR525oD(Xr-$j)cH4lLPMDmJf}y(Qq?9UY2ioAuDI$+0(4 z2WsSy2%srH!wZZA04l3QEBz*6fsC=vQsK--sFIiQN9Xl>tpgIJcsj@fevo5AoYZ_) zwf?%LvCPm5fyIMDOMdIhyq+^SY#T9r^W6okhP#$7`;8D13QGdHvG@h-Yh?cef^RRc zS5?5oBU}Qu8!b^o`z3tipr6Ax_i7#+CiNL9eARh*0-U|w1N!{w5wdGJA=comp>^RD zLW{k^3K!UNmAWiY4g!*a3=KCnb)=9$7WfYGv{-rCm+&Pae%==LxQEg>UEMt|2F-%) zpu!405fbVo`oP*fJ|^6KV)$CFZ;gFi{{nK=Y$e4sXLM)bp`T& zOtclTJvD$xWEc7Sqm-Y35{jv-dQ2W2rFf3Gx$rbhdS6-7OU*| zeHKinc+2~>ve;~gl+!FWi-||oDA{idZTuVn(?fsspzTP z-rA&FsiISveKTsg`@SF^NMm+rtuVv}tXM*@)8#~G`Ibd$AI&)i>#`sDny|lywWb_zR6)jV%hBnru_;nmZxO?K zwvghUP{d+|zU3t9HQ+23*`gklBS>dAuOs*w@mN46AMHd!+;_jhN4GyB4*L~+7EX?i zRu=BRL7w+dsyjU3Tfwf~oz(AsP3i-gEy=qJcH_?rHmHeMuxp4(eNSs?n7EBNOip%& zS9@sh^*Z1Df{Ia&mORW0c9tTIgyorDlMkHKTD(oy)VldxuVn; z$VvDUs@0PJw_tzxfqotS6Nnb}=dB!fy1|b-2>j&^JG|i(eKQ;)Zxc(WDk{kL${BD( zo_R!54Nw@Q6MDD@7)zPYYVm~lBrNMk2cGB91n~3KWCe)Q0jylYwgp=kL5bpBxCrYOwq#O zh8fcaMaOVM&dy_N>ryDOEjsQ1HuQ)G$2>i#gTBafdMaXBReBy1p6h6z!T;)IEj4w? zK26edUI3yTN{Ltf#(%!haaZZxZ1&V&K8B9Fhq;_R%FRvQ!@9s!~S=i^p1(IF=5Tvrd5Be0_E?K+O z#K<=5k|umlHw*L-y&`>q-y=5o20s6`xcdc<0hPp{TvuB@UhXA?tBv@LjymOk(%P$) zgO9b`K-4LRubyZgTN+osoI>wMJIBoEeexERg-lgYgk!wWckYbA?S&U(u=jX#Xa0Nd z`-=+lTkB1{jnhSIi?@dL58gb2h_7Sb@U*ZmX{^<|eAVi8OA(sz?+F9v=Bw=>Gt15+ zlkl6himS2+6Y80+wUb6|-6iMz{=1}+a7$|DMSK;nTP7x}nb%+V+=~U4ks7^tI`7UN z#klePCfCx2yd{U$P&#;bKNBWPgg5L-4gDy-LdgX3-}CTC&7$KfN+D`GBWSVZc0%mU zpe9pJ1bVz*{xU|~fTua*zTeTg>9}(Wk>hM z-o$*kfqi-JF|L~p$nre@oOC>);02!hc13bUmM-x^zc5ZS) zPgU##Sl|9l`K;$c`O(sb#Jc=`u^f7`78wX-k8M{3OeN=Ix5-omy^?O&6Rx8{bP+Vw z1*iKgg|~7Kg^G$(W(nJt9}I&z3X@@Y?l?{?UruW><`Gd6=gYMu8{NBE%mCYR8c+4P z-DA#jbjE>lY0@PTj zBw$DMv`Po^Ubh8YK&=<YCA?B~6)Xr6=p36`-&aXZY9Odsz^PLxaf5`<~J>JyTYP{W9{HGfi?xWX%&r$a+-RU9qpdPvF7u9hizjkk7X-x z4YFFpW25HxjT#Cpu^DlLPUS;-D0D90Q;>6$zV|32*eNJKW>cVv*XE{m^&qE0D*m(T4W0oH0#Pm!sy&l+1wdD!6cFyc)59D>aH$$#??5%0@eZ}QG zH<3d2R_RqNt-DN^>6<(CPBTW-<8PRjsE4b-zTh{jJjRD_G0vQx972w=4u+dgSHue6 zLGnw`!)aNIy^vj50xwxd-=4pM#(*wGWX{OWgmfaGTzn}^=)gSTdK=-A)YRBMKW`NY zX*Q{3YC^sXDti?sg}iWNg+)>d9oHQiZe3qoVr4jOs**KXbw%TTc)aInNmKzW(JH`4>C=i*62*}=0T1CwnTA%2oHzSrO19Y*Tkw1HBqjfOtDE7f{%4+fo_C)4o%5dG=RN25evgGQBNGn@3Wb8UCcm=-(HRSf z31q5oqpe_Qwoma(3kbvrvM`1~ffeBI%PfBC%tKr8!`W2Z%y6H+wGGmA-@GZQ)x=l< z+0JgPAWJ5_EVM$s+C4bxk2ErsH#BVze89vC(x3l}DW&thAR`$q;i$#YMEHIfa_l*nX2EfY*m)Ou0-xVsKim%~lzvFfxl9=u^#1 zzl#wG2`%6ZtCMa~SqY0Nov-E`CJ+ON4x~ALsDpAkAD_RGZoD`s226T$i6+>Uts0v+&x0l426(h)Rftwr(Z&*bv5 zn(YcmkJ;)6$$Z6I#(>R`k={OLU(xLdzr!JHOBIFgR93uYmRt#XygWc=fZW#En$9=mKolYtlIorWL)b>;Qes^$9@4$aI@uf z__R6Bp*ZzdL0h@^I8{tFG6tQ$FvwCFZkY*4e5unu)oI8?Xf zJ`(9l`^`^%uJf5aMY@N&%KL`AsuL~N*l{L9aZVVj2K4Z;y7bpO=3Z!v$6(pUf*OKp zXFw4LqfT7SSE`6)kMaNZ@J2);KSShp!q?ZsE#Mb)rGDp1NBQ5Al4t!%$)wHsdE7b} znV5a@mf(Ic(TW<1I-~FDmE!5O6Lj2PwlSPBNkY)8C|OAmUer+5bm`;D_2z^!mm*XcDO0EtdHctGfffa~?mVR)ZHmVzJ;57R!uBCPswuFs#K8Ox_>UOY0fDZ%$8mVP zUZ9Upu&ui%(hc}kW#{t;w*k-7I`AYnXKK1$bF5A@%Re3nyKU!)PSG)Xca-4790w5> z!*2N}?)DlTVr|&A>UP-l-bD&=j0&L|EKteF7(vviHTLmteEB>HKNI@)-ItY@P1T@5 zp&(P2=O`GW|Mry~`rhb2gXLAsm2+Gv6;?#Jw}W^5@ICk9mf(2w^G<=kGzMAS$gbhXw6Ed@-!Bz2X1yYbip zq^W$qNKjMid3F00dmrCWxrxzwnTYxwau2OuRr93J`s>cuvS1FPo!Jt7G~PN!SenBX z;uYD0cKIl!9@_i-gyt1}S2A&%~L+sFUeGLxJPHf+98 zJoGSugwMA$+#ee?mYYSe#3Ejo#&;wXqh%oa=GQf1T7CJd(^PL9-Z__^o^blic};T{ zgy;zO31>$$$F8PSJ~y2ZuZwt7V;#K#;noT!DvT$(^zkE%Q_$YQ{uDQ5MK*FnWXb?L zxtNoONy%`aWlqEg4(Ci?w&Au7a}$>9NOctv=z`ch@;2ISsT?W$a#C8Z!5-bTDB7sVXz`jy>H#OMf8T@qWHD( zl10NVSek@%_|^;1vWGj0V_ZVHCn~n?cYlJFTrrq}v=8wi{Pt=G>OLx_GTF&r+E=?f z$~^)U9E%m!^;&)1=Ov1#w0}6tl>I4SR3ZUfkvs7q@MR`(hQMU<(S_Uh_8rd~3vsZ4 zB(|C|u|}5xf+dN$3iT34817~KWYYWG&QQtrK@PUPAbU2ve#D6t*%4B1WZ882B#MNF zm7;dq6lxZ7Ed^)y_Fz6bvLjk;7V4&Y(Cq;=)Fy+(St0PzkKUN#L&Zt zxS*?A;b=hzb<0OQtT9r~6|b&ZkWzhf0`*-FwsyR|>T4Ft<;`4xGwE>x@9{)tw<|?~ zTKyL)g}IB2gi@^#kNdY}AI?}BQctp1!Sf%|WZIRBNKAW^&<+zq@oxdcpnBD8N?w(= z*VGk4`S>)BbI%r23+7JsNK>{)WIEqtJu6+C3KOl}fpsMCtDh3`=$yV-C1GiMO+-Q_ z!I8VSJ_I^!<7?D)a`K8UxaX<+Oj&-Y+BpIStkZ|e%l5F6pHN&qzZmlD3xV@i?FY?X z0;7C4taq$<9=9~Kcj|)DBSqA;!UJgM$x6!xiGwzQqn5eOb|j}$82k6^CM3;!iKe@jO8 zs~SD)wl6oODT4ko6qWPOvN{#nq9p?N$hI-#x=EJ7iqp2xo?)*^)LinG#qnIyBb_s<&!CPC?GfeY(8Qh1l;9-F0d2`l=&=})Bxd2OiRu$) z7bcnY22cz=et9U3Gp)K%T>fho7gT9@;a`zh+-G<8`@MlNS>>s1~*FH@iF;EDyK;1{eK?!#7RC5%A9GK(yte-5dJpE||yx8AI-`bS~+ zHs#nt`@AtzczVt1(EGE17@yYLk9i0frZ&3Lk2HX`UNhho@xzsF zlma5Kf_^^wtVO7yU49#c*3tz=A^MR37=_ksgmwvd{CM5}>Sn;qux1Mk>jne*eFFES z*VOCxDM}-^*+lWHTLapA&7>LEMfg{VuirGiG6l5wnjP58eNzG(6&Ijm*UXLf;ORE< zEsbxgb}bbk1ihRCbnTj@{w&1*l@1^P*Q}gYted8@CI{E2MX!DUjk9Ld--X&Vz0Gvf wT1*MZZ_@jrfapsJ&{6cA2sFu>dHmlof1;`^jG4Cp;W%iYK3Z8|zP88z0CLwo^#A|> diff --git a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/artifact/StubClassLoaderSuite.scala b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/artifact/StubClassLoaderSuite.scala index 1c0442854de7..0f6e05431518 100644 --- a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/artifact/StubClassLoaderSuite.scala +++ b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/artifact/StubClassLoaderSuite.scala @@ -17,16 +17,16 @@ 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} +import org.apache.spark.util.{ChildFirstURLClassLoader, StubClassLoader} class StubClassLoaderSuite extends SparkFunSuite { - private val udfByteArray: Array[Byte] = Files.readAllBytes(Paths.get("src/test/resources/udf")) + // See src/test/resources/StubClassDummyUdf for how the UDFs and jars are created. private val udfNoAJar = new File("src/test/resources/udf_noA.jar").toURI.toURL + private val classDummyUdf = "org.apache.spark.sql.connect.client.StubClassDummyUdf" + private val classA = "org.apache.spark.sql.connect.client.A" test("find class with stub class") { val cl = new RecordedStubClassLoader(getClass().getClassLoader(), _ => true) @@ -55,8 +55,7 @@ class StubClassLoaderSuite extends SparkFunSuite { } } - test("load udf") { - // See src/test/resources/StubClassDummyUdf for how the udf and jar is created. + test("stub missing class") { val sysClassLoader = getClass.getClassLoader() val stubClassLoader = new RecordedStubClassLoader(null, _ => true) @@ -64,26 +63,23 @@ class StubClassLoaderSuite extends SparkFunSuite { val sessionClassLoader = new ChildFirstURLClassLoader(Array(udfNoAJar), stubClassLoader, sysClassLoader) // Load udf with A used in the same class. - deserializeUdf(sessionClassLoader) + loadDummyUdf(sessionClassLoader) // Class A should be stubbed. - assert(stubClassLoader.lastStubbed === "org.apache.spark.sql.connect.artifact.A") + assert(stubClassLoader.lastStubbed === classA) } 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 + // Failed to load DummyUdf intercept[Exception] { - deserializeUdf(cl1) + loadDummyUdf(cl1) } // Successfully stubbed the missing class. - assert( - stubClassLoader.lastStubbed === - "org.apache.spark.sql.connect.artifact.StubClassDummyUdf") + assert(stubClassLoader.lastStubbed === classDummyUdf) // Creating a new class loader will unpack the udf correctly. val cl2 = new ChildFirstURLClassLoader( @@ -91,35 +87,35 @@ class StubClassLoaderSuite extends SparkFunSuite { stubClassLoader, // even with the same stub class loader. sysClassLoader) // Should be able to load after the artifact is added - deserializeUdf(cl2) + loadDummyUdf(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) - } + // Failed to load DummyUdf because of missing methods + assert(intercept[NoSuchMethodException] { + loadDummyUdf(sessionClassLoader) + }.getMessage.contains(classDummyUdf)) // 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) + assert(stubClassLoader.lastStubbed === classDummyUdf) } - private def deserializeUdf(sessionClassLoader: ClassLoader): UdfPacket = { - Utils.deserialize[UdfPacket](udfByteArray, sessionClassLoader) + private def loadDummyUdf(sessionClassLoader: ClassLoader): Unit = { + // Load DummyUdf and call a method on it. + // scalastyle:off classforname + val cls = Class.forName(classDummyUdf, false, sessionClassLoader) + // scalastyle:on classforname + cls.getDeclaredMethod("dummy") + + // Load class A used inside DummyUdf + // scalastyle:off classforname + Class.forName(classA, false, sessionClassLoader) + // scalastyle:on classforname } } diff --git a/core/src/main/scala/org/apache/spark/util/StubClassLoader.scala b/core/src/main/scala/org/apache/spark/util/StubClassLoader.scala index 193582598acd..a0bc753f4887 100644 --- a/core/src/main/scala/org/apache/spark/util/StubClassLoader.scala +++ b/core/src/main/scala/org/apache/spark/util/StubClassLoader.scala @@ -33,13 +33,10 @@ class StubClassLoader(parent: ClassLoader, shouldStub: String => Boolean) throw new ClassNotFoundException(name) } val bytes = StubClassLoader.generateStub(name) - // scalastyle:off println - println("###stub class: " + name) defineClass(name, bytes, 0, bytes.length) } } -// TODO: provide helpful error message for method not found exception. object StubClassLoader { def apply(parent: ClassLoader, binaryName: Seq[String]): StubClassLoader = { new StubClassLoader(parent, name => binaryName.exists(p => name.startsWith(p)))