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
18 changes: 16 additions & 2 deletions core/src/main/scala/org/apache/spark/executor/Executor.scala
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ package org.apache.spark.executor
import java.io.{File, NotSerializableException}
import java.lang.Thread.UncaughtExceptionHandler
import java.lang.management.ManagementFactory
import java.net.{URI, URL}
import java.net.{URI, URL, URLClassLoader}
import java.nio.ByteBuffer
import java.util.{Locale, Properties}
import java.util.concurrent._
Expand Down Expand Up @@ -212,14 +212,28 @@ private[spark] class Executor(
val defaultSessionState: IsolatedSessionState = newSessionState(JobArtifactState("default", None))

val isolatedSessionCache: Cache[String, IsolatedSessionState] = CacheBuilder.newBuilder()
.maximumSize(100)
.maximumSize(conf.get(EXECUTOR_ISOLATED_SESSION_CACHE_SIZE))
.expireAfterAccess(30, TimeUnit.MINUTES)
.removalListener(new RemovalListener[String, IsolatedSessionState]() {
override def onRemoval(
notification: RemovalNotification[String, IsolatedSessionState]): Unit = {
val state = notification.getValue
// Cache is always used for isolated sessions.
assert(!isDefaultState(state.sessionUUID))
// Close the urlClassLoader to release resources.
try {
state.urlClassLoader match {
case urlClassLoader: URLClassLoader =>
urlClassLoader.close()
logInfo(log"Closed urlClassLoader (URLClassLoader) for evicted session " +
log"${MDC(SESSION_ID, state.sessionUUID)}")
case _ =>
}
} catch {
case NonFatal(e) =>
logWarning(log"Failed to close urlClassLoader for session " +
log"${MDC(SESSION_ID, state.sessionUUID)}", e)
}
val sessionBasedRoot = new File(SparkFiles.getRootDirectory(), state.sessionUUID)
if (sessionBasedRoot.isDirectory && sessionBasedRoot.exists()) {
Utils.deleteRecursively(sessionBasedRoot)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -358,6 +358,15 @@ package object config {
.intConf
.createWithDefault(60)

private[spark] val EXECUTOR_ISOLATED_SESSION_CACHE_SIZE =
ConfigBuilder("spark.executor.isolatedSessionCache.size")
.doc("Maximum number of isolated sessions to cache in the executor. Each cached session " +
"maintains its own classloader for artifact isolation.")
.version("4.1.0")
.intConf
.checkValue(_ > 0, "The cache size must be positive.")
.createWithDefault(100)

private[spark] val EXECUTOR_PROCESS_TREE_METRICS_ENABLED =
ConfigBuilder("spark.executor.processTreeMetrics.enabled")
.doc("Whether to collect process tree metrics (from the /proc filesystem) when collecting " +
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -135,6 +135,8 @@ object SparkConnectServerUtils {
"spark.connect.execute.reattachable.senderMaxStreamSize=123",
// Testing SPARK-49673, setting maxBatchSize to 10MiB
s"spark.connect.grpc.arrow.maxBatchSize=${10 * 1024 * 1024}",
// Cache less sessions to save memory.
"spark.executor.isolatedSessionCache.size=5",
// Disable UI
"spark.ui.enabled=false").flatMap(v => "--conf" :: v :: Nil)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -450,7 +450,20 @@ class ArtifactManager(session: SparkSession) extends AutoCloseable with Logging
pythonIncludeList.clear()
sparkContextRelativePaths.clear()

// Removed cached classloader
// Close and remove cached classloader
cachedClassLoader.foreach {
case urlClassLoader: URLClassLoader =>
try {
urlClassLoader.close()
logDebug(log"Closed URLClassLoader for session " +
log"${MDC(LogKeys.SESSION_ID, session.sessionUUID)}")
} catch {
case e: IOException =>
logWarning(log"Failed to close URLClassLoader for session " +
log"${MDC(LogKeys.SESSION_ID, session.sessionUUID)}", e)
}
case _ =>
}
cachedClassLoader = None
}

Expand Down