[
https://issues.apache.org/jira/browse/SPARK-54753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=18047365#comment-18047365
]
xihuan edited comment on SPARK-54753 at 12/25/25 1:24 AM:
----------------------------------------------------------
as tested with 2000 times, it was found that there would be above 2000
artifactManager instances, which are referenced by the cleaner and not able to
be GCed
!image-2025-12-24-03-18-24-468.png|width=1088,height=416!
PR was submit to fix this issue
[https://github.com/apache/spark/pull/53592]
was (Author: JIRAUSER308635):
as tested with 2000 times, it was found that there would be above 2000
artifactManager instances, which are not cleaned by the Cleaner.
!image-2025-12-24-03-18-24-468.png|width=1088,height=416!
PR was submit to fix this issue
[https://github.com/apache/spark/pull/53592]
> memory leak in Apache Spark 4.0.1 as we persist/unpersist the dataset
> ---------------------------------------------------------------------
>
> Key: SPARK-54753
> URL: https://issues.apache.org/jira/browse/SPARK-54753
> Project: Spark
> Issue Type: Bug
> Components: Spark Core
> Affects Versions: 4.0.0, 4.0.1
> Reporter: xihuan
> Priority: Critical
> Labels: pull-request-available
> Attachments: image-2025-12-23-05-38-53-324.png,
> image-2025-12-24-03-18-24-468.png, pom.xml, screenshot-1.png
>
>
> For Apache Spark *4.0.1* local mode, the memory is not released after long
> time running, while downgrade the spark version {*}v3.5.6{*}, there is no
> issue.
> The issue can be reproduced with simple test cases:
>
>
> {code:java}
> package spark;
> import org.apache.spark.sql.Dataset;
> import org.apache.spark.sql.Row;
> import org.apache.spark.sql.SparkSession;
> import org.slf4j.Logger;
> import org.slf4j.LoggerFactory;
> import java.util.concurrent.Executors;
> import java.util.concurrent.ScheduledExecutorService;
> public class SparkApp {
> private static final Logger log = LoggerFactory.getLogger(SparkApp.class);
> private static final String SPARK_MASTER_URL = "local[4]";
> private static final String SPARK_MEMORY = "500m";
> public static void main(String[] args) {
> log.debug("Starting application...");
> SparkSession sparkSession = SparkSession.builder()
> .appName("Test Application")
> .master(SPARK_MASTER_URL)
> .config("spark.driver.memory", SPARK_MEMORY)
>
> //.config("spark.sql.sources.bucketing.autoBucketedScan.enabled", false) //
> To get rid of the memory leak
> .getOrCreate();
> processData(sparkSession);
> }
> private static void processData(SparkSession sparkSession) {
>
> while (true) {
> //load from a local csv file
> Dataset<Row> dataset = sparkSession.read().csv("c:/City.csv");
> dataset.show(5);
> log.debug("Persist dataset...");
> // This is the place where the memory leak occurs
> dataset.persist();
> // Do something...
> log.debug("Do something with the persisted dataset");
> // ...and unpersist the dataset
> log.debug("Unpersist dataset...");
> dataset.unpersist();
> log.debug("Processing data completed");
> }
> }
> }
> {code}
>
> with *Apache Spark 4.0.0 or 4.0.1* as JVM heap size to 1GB, around 9 minutes,
> it will encounter OOM.
> !screenshot-1.png|width=1070,height=504!
>
> While, {*}with Apache Spark 3.5.6, with max heap size of 500MB{*}, no such
> issue occurs.
> !image-2025-12-23-05-38-53-324.png|width=1075,height=415!
>
>
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]