Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,6 @@ import com.google.protobuf.{Any => ProtoAny, ByteString}
import io.grpc.{Context, Status, StatusRuntimeException}
import io.grpc.stub.StreamObserver
import org.apache.commons.lang3.exception.ExceptionUtils
import org.json4s._
import org.json4s.jackson.JsonMethods.parse

import org.apache.spark.{Partition, SparkEnv, TaskContext}
import org.apache.spark.api.python.{PythonEvalType, SimplePythonFunction}
Expand Down Expand Up @@ -91,15 +89,6 @@ class SparkConnectPlanner(val sessionHolder: SessionHolder) extends Logging {
private lazy val pythonExec =
sys.env.getOrElse("PYSPARK_PYTHON", sys.env.getOrElse("PYSPARK_DRIVER_PYTHON", "python3"))

// SparkConnectPlanner is used per request.
private lazy val pythonIncludes = {
implicit val formats = DefaultFormats
parse(session.conf.get("spark.connect.pythonUDF.includes", "[]"))
.extract[Array[String]]
.toList
.asJava
}

// The root of the query plan is a relation and we apply the transformations to it.
def transformRelation(rel: proto.Relation): LogicalPlan = {
val plan = rel.getRelTypeCase match {
Expand Down Expand Up @@ -1519,7 +1508,7 @@ class SparkConnectPlanner(val sessionHolder: SessionHolder) extends Logging {
command = fun.getCommand.toByteArray,
// Empty environment variables
envVars = Maps.newHashMap(),
pythonIncludes = pythonIncludes,
pythonIncludes = sessionHolder.artifactManager.getSparkConnectPythonIncludes.asJava,
Copy link
Contributor Author

Choose a reason for hiding this comment

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

@HyukjinKwon With directly fetching the Python Includes, I've removed the private lazy val pythonIncludes and SessionHolder#withSessionBasedPythonPaths since we don't need to propagate them via confs anymore

Copy link
Member

Choose a reason for hiding this comment

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

Gotya. This is exactly the change I planned to made this week :-). Thank you for doing this here.

pythonExec = pythonExec,
pythonVer = fun.getPythonVer,
// Empty broadcast variables
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,9 +24,6 @@ import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap}
import scala.collection.JavaConverters._
import scala.util.control.NonFatal

import org.json4s.JsonDSL._
import org.json4s.jackson.JsonMethods.{compact, render}

import org.apache.spark.JobArtifactSet
import org.apache.spark.connect.proto
import org.apache.spark.internal.Logging
Expand Down Expand Up @@ -107,7 +104,7 @@ case class SessionHolder(userId: String, sessionId: String, session: SparkSessio
* @param f
* @tparam T
*/
def withContext[T](f: => T): T = {
def withContextClassLoader[T](f: => T): T = {
// Needed for deserializing and evaluating the UDF on the driver
Utils.withContextClassLoader(classloader) {
// Needed for propagating the dependencies to the executors.
Expand All @@ -117,49 +114,15 @@ case class SessionHolder(userId: String, sessionId: String, session: SparkSessio
}
}

/**
* Set the session-based Python paths to include in Python UDF.
* @param f
* @tparam T
*/
def withSessionBasedPythonPaths[T](f: => T): T = {
try {
session.conf.set(
"spark.connect.pythonUDF.includes",
compact(render(artifactManager.getSparkConnectPythonIncludes)))
f
} finally {
session.conf.unset("spark.connect.pythonUDF.includes")
}
}

/**
* Execute a block of code with this session as the active SparkConnect session.
* @param f
* @tparam T
*/
def withSession[T](f: SparkSession => T): T = {
withSessionBasedPythonPaths {
withContext {
session.withActive {
f(session)
}
}
}
}

/**
* Execute a block of code using the session from this [[SessionHolder]] as the active
* SparkConnect session.
* @param f
* @tparam T
*/
def withSessionHolder[T](f: SessionHolder => T): T = {
withSessionBasedPythonPaths {
withContext {
session.withActive {
f(this)
}
withContextClassLoader {
session.withActive {
f(session)
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ private[connect] class SparkConnectAnalyzeHandler(
request.getSessionId)
// `withSession` ensures that session-specific artifacts (such as JARs and class files) are
// available during processing (such as deserialization).
sessionHolder.withSessionHolder { sessionHolder =>
sessionHolder.withSession { _ =>
val response = process(request, sessionHolder)
responseObserver.onNext(response)
responseObserver.onCompleted()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -224,6 +224,7 @@ class ArtifactManagerSuite extends SharedSparkSession with ResourceHelper {
test("Classloaders for spark sessions are isolated") {
val holder1 = SparkConnectService.getOrCreateIsolatedSession("c1", "session1")
val holder2 = SparkConnectService.getOrCreateIsolatedSession("c2", "session2")
val holder3 = SparkConnectService.getOrCreateIsolatedSession("c3", "session3")

def addHelloClass(holder: SessionHolder): Unit = {
val copyDir = Utils.createTempDir().toPath
Expand All @@ -234,7 +235,7 @@ class ArtifactManagerSuite extends SharedSparkSession with ResourceHelper {
holder.addArtifact(remotePath, stagingPath, None)
}

// Add the classfile only for the first user
// Add the "Hello" classfile for the first user
addHelloClass(holder1)

val classLoader1 = holder1.classloader
Expand All @@ -246,7 +247,8 @@ class ArtifactManagerSuite extends SharedSparkSession with ResourceHelper {
val udf1 = org.apache.spark.sql.functions.udf(instance1)

holder1.withSession { session =>
session.range(10).select(udf1(col("id").cast("string"))).collect()
val result = session.range(10).select(udf1(col("id").cast("string"))).collect()
assert(result.forall(_.getString(0).contains("Talon")))
}

assertThrows[ClassNotFoundException] {
Expand All @@ -257,6 +259,20 @@ class ArtifactManagerSuite extends SharedSparkSession with ResourceHelper {
.newInstance("Talon")
.asInstanceOf[String => String]
}

// Add the "Hello" classfile for the third user
addHelloClass(holder3)
val instance3 = holder3.classloader
.loadClass("Hello")
.getDeclaredConstructor(classOf[String])
.newInstance("Ahri")
.asInstanceOf[String => String]
val udf3 = org.apache.spark.sql.functions.udf(instance3)

holder3.withSession { session =>
val result = session.range(10).select(udf3(col("id").cast("string"))).collect()
assert(result.forall(_.getString(0).contains("Ahri")))
}
}
}

Expand Down
Binary file added core/src/test/resources/TestHelloV2.jar
Binary file not shown.
Binary file added core/src/test/resources/TestHelloV3.jar
Binary file not shown.
Original file line number Diff line number Diff line change
@@ -0,0 +1,102 @@
/*
* 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.executor

import org.apache.spark.{JobArtifactSet, LocalSparkContext, SparkConf, SparkContext, SparkFunSuite}
import org.apache.spark.util.Utils

class ClassLoaderIsolationSuite extends SparkFunSuite with LocalSparkContext {
Copy link
Contributor

Choose a reason for hiding this comment

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

Copy link
Contributor

@LuciferYang LuciferYang Jul 4, 2023

Choose a reason for hiding this comment

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

Perhaps we should upload both TestHelloV2 and TestHelloV3 compiled jar in Scala 2.13 at the same time?

On the other hand, I think we should provide the source code of TestHelloV2 and `TestHelloV3 (such as in comments?). Otherwise, when upgrading to the major version of Scala(for example Scala 3.x), we will not know how to generate the new jar package for testing

Sorry, I found the source code in commnets

Copy link
Contributor

@LuciferYang LuciferYang Jul 4, 2023

Choose a reason for hiding this comment

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

Try fix #41852

Copy link
Member

Choose a reason for hiding this comment

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

Thanks @LuciferYang

val jar1 = Thread.currentThread().getContextClassLoader.getResource("TestUDTF.jar").toString

// package com.example
// object Hello { def test(): Int = 2 }
// case class Hello(x: Int, y: Int)
val jar2 = Thread.currentThread().getContextClassLoader.getResource("TestHelloV2.jar").toString

// package com.example
// object Hello { def test(): Int = 3 }
// case class Hello(x: String)
val jar3 = Thread.currentThread().getContextClassLoader.getResource("TestHelloV3.jar").toString

test("Executor classloader isolation with JobArtifactSet") {
sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local"))
sc.addJar(jar1)
sc.addJar(jar2)
sc.addJar(jar3)

// TestHelloV2's test method returns '2'
val artifactSetWithHelloV2 = new JobArtifactSet(
uuid = Some("hello2"),
replClassDirUri = None,
jars = Map(jar2 -> 1L),
files = Map.empty,
archives = Map.empty
)

JobArtifactSet.withActive(artifactSetWithHelloV2) {
sc.parallelize(1 to 1).foreach { i =>
val cls = Utils.classForName("com.example.Hello$")
val module = cls.getField("MODULE$").get(null)
val result = cls.getMethod("test").invoke(module).asInstanceOf[Int]
if (result != 2) {
throw new RuntimeException("Unexpected result: " + result)
}
}
}

// TestHelloV3's test method returns '3'
val artifactSetWithHelloV3 = new JobArtifactSet(
uuid = Some("hello3"),
replClassDirUri = None,
jars = Map(jar3 -> 1L),
files = Map.empty,
archives = Map.empty
)

JobArtifactSet.withActive(artifactSetWithHelloV3) {
sc.parallelize(1 to 1).foreach { i =>
val cls = Utils.classForName("com.example.Hello$")
val module = cls.getField("MODULE$").get(null)
val result = cls.getMethod("test").invoke(module).asInstanceOf[Int]
if (result != 3) {
throw new RuntimeException("Unexpected result: " + result)
}
}
}

// Should not be able to see any "Hello" class if they're excluded from the artifact set
val artifactSetWithoutHello = new JobArtifactSet(
uuid = Some("Jar 1"),
replClassDirUri = None,
jars = Map(jar1 -> 1L),
files = Map.empty,
archives = Map.empty
)

JobArtifactSet.withActive(artifactSetWithoutHello) {
sc.parallelize(1 to 1).foreach { i =>
try {
Utils.classForName("com.example.Hello$")
throw new RuntimeException("Import should fail")
} catch {
case _: ClassNotFoundException =>
}
}
}
}
}