Fix ClassLoader leak in Spark caused by AuthSessionCache and ensure resource cleanup #14590
+41
−5
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Summary
Fix memory leak in Spark from
AuthSessionCachewhen using Iceberg and ensure resources get cleanup.Background
I am using Spark Connect where end-users will be submitting their spark jobs/queries from their end into the remote Spark Connect server. These queries runtime can ranges from seconds to minutes and query per users can varies as well. Also, this in case, the end-users are the ones who are creating spark session and defined the connection info to Iceberg REST catalog. By default, Spark Connect server will cleanup idle sessions after one hour.
What I found out interesting is the memory usage of Spark Connect is not able to get garbage collected after Spark Connect server killed the idle sessions after reached default TTL. After some debugging, this point me to
ClassLoaderfrom Apache Spark leak inAuthSessionCache.javafrom Apache Iceberg.Changes
Fixing the
ClassLaoderleak in Apache Spark inAuthSessionCache.javaThe existed
ThreadPools.newExitingWorkerPoolcreated aScheduledExecutorServiceand registers a JVM-level shutdown hook. This hook can inadvertently hold a strong reference to session specificClassLoaderin Spark connect via the tasks it manages, which preventing them from being released. This change replacesnewExitingWorkerPoolwithnewScheduledPoolwhich creates a thread pool with daemon threads. Based on my understanding, daemon threads do not block JVM from existing thus prevent the issue mentioned above.Ensure proper resources cleanup in catalogs
CachingCatalogandSparkCatalognow implementsjava.io.Closeablewhich allows them to propagate theclosecall to the underlying wrapped catalog. This will ensure that any resource referenced by catalogs are properly released.Reference
JIRA for Apache Spark: https://issues.apache.org/jira/browse/SPARK-54367