Skip to content
Closed
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
19 changes: 16 additions & 3 deletions core/src/main/scala/org/apache/spark/executor/Executor.scala
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, WrappedArray}
import scala.concurrent.duration._
import scala.util.control.NonFatal

import com.google.common.cache.CacheBuilder
import com.google.common.cache.{Cache, CacheBuilder, RemovalListener, RemovalNotification}
import com.google.common.util.concurrent.ThreadFactoryBuilder
import org.slf4j.MDC

Expand Down Expand Up @@ -196,11 +196,24 @@ private[spark] class Executor(

// Classloader isolation
// The default isolation group
val defaultSessionState = newSessionState(JobArtifactState("default", None))
val defaultSessionState: IsolatedSessionState = newSessionState(JobArtifactState("default", None))

val isolatedSessionCache = CacheBuilder.newBuilder()
val isolatedSessionCache: Cache[String, IsolatedSessionState] = CacheBuilder.newBuilder()
.maximumSize(100)
.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))
val sessionBasedRoot = new File(SparkFiles.getRootDirectory(), state.sessionUUID)
if (sessionBasedRoot.isDirectory && sessionBasedRoot.exists()) {
Utils.deleteRecursively(sessionBasedRoot)
}
logInfo(s"Session evicted: ${state.sessionUUID}")
}
})
.build[String, IsolatedSessionState]

// Set the classloader for serializer
Expand Down