[ 
https://issues.apache.org/jira/browse/SPARK-10942?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14944497#comment-14944497
 ] 

Rekha Joshi edited comment on SPARK-10942 at 10/6/15 4:48 AM:
--------------------------------------------------------------

Thanks [~pnpritchard] I tried to replicate the issue few times now. So far I 
see Storage tab getting cleaned out.I do not even specify ttl.Attached job run 
screenshots.I am on 1.6.0-snapshot, and do not currently have any other load on 
the system but first level diagnosis seems automatic unpersist does happen.I do 
see below logs also stating persistence list is getting updated in background, 
and storage cleared. [~sowen] [~vanzin] Your thoughts? Thanks
{panel}
15/10/05 21:42:24 INFO rdd.ParallelCollectionRDD: Removing RDD 30 from 
persistence list
15/10/05 21:42:24 INFO rdd.ParallelCollectionRDD: Removing RDD 30 from 
persistence list
15/10/05 21:42:24 INFO storage.BlockManager: Removing RDD 30
15/10/05 21:42:24 INFO storage.BlockManager: Removing RDD 30
15/10/05 21:42:24 INFO scheduler.ReceivedBlockTracker: Deleting batches 
ArrayBuffer()
15/10/05 21:42:24 INFO scheduler.InputInfoTracker: remove old batch metadata: 
{panel}



was (Author: rekhajoshm):
Thanks [~pnpritchard] I tried to replicate the issue few times now. So far I 
see Storage tab getting cleaned out.I do not even specify ttl.
Attached job run screenshots.I am on 1.6.0-snapshot, and do not currently have 
any other load on the system but first level diagnosis seems automatic 
unpersist does happen.I do see below logs also stating persistence list is 
getting updated in background, and storage cleared. [~sowen] [~vanzin] Your 
thoughts? Thanks
{panel}
15/10/05 21:42:24 INFO rdd.ParallelCollectionRDD: Removing RDD 30 from 
persistence list
15/10/05 21:42:24 INFO rdd.ParallelCollectionRDD: Removing RDD 30 from 
persistence list
15/10/05 21:42:24 INFO storage.BlockManager: Removing RDD 30
15/10/05 21:42:24 INFO storage.BlockManager: Removing RDD 30
15/10/05 21:42:24 INFO scheduler.ReceivedBlockTracker: Deleting batches 
ArrayBuffer()
15/10/05 21:42:24 INFO scheduler.InputInfoTracker: remove old batch metadata: 
{panel}


> Not all cached RDDs are unpersisted
> -----------------------------------
>
>                 Key: SPARK-10942
>                 URL: https://issues.apache.org/jira/browse/SPARK-10942
>             Project: Spark
>          Issue Type: Bug
>          Components: Streaming
>            Reporter: Nick Pritchard
>
> I have a Spark Streaming application that caches RDDs inside of a 
> {{transform}} closure. Looking at the Spark UI, it seems that most of these 
> RDDs are unpersisted after the batch completes, but not all.
> I have copied a minimal reproducible example below to highlight the problem. 
> I run this and monitor the Spark UI "Storage" tab. The example generates and 
> caches 30 RDDs, and I see most get cleaned up. However in the end, some still 
> remain cached. There is some randomness going on because I see different RDDs 
> remain cached for each run.
> I have marked this as Major because I haven't been able to workaround it and 
> it is a memory leak for my application. I tried setting {{spark.cleaner.ttl}} 
> but that did not change anything.
> {code}
> val inputRDDs = mutable.Queue.tabulate(30) { i =>
>   sc.parallelize(Seq(i))
> }
> val input: DStream[Int] = ssc.queueStream(inputRDDs)
> val output = input.transform { rdd =>
>   if (rdd.isEmpty()) {
>     rdd
>   } else {
>     val rdd2 = rdd.map(identity)
>     rdd2.setName(rdd.first().toString)
>     rdd2.cache()
>     val rdd3 = rdd2.map(identity)
>     rdd3
>   }
> }
> output.print()
> ssc.start()
> ssc.awaitTermination()
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org

Reply via email to