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

Tom Hubregtsen commented on SPARK-7002:
---------------------------------------

I disagree. I first also thought it was just something wrong in the debug 
output. To verify this, I switched to persisting to disk and running through 
the shell. When looking in the file system, I can see the file is created the 
first time persist is called (and triggered by an action on a child RDD). After 
unpersist and another persist (both with actions on the child RDD to trigger 
the computation), the file does *not* exist on disk (Again, when looking in the 
file system, although the debug message mirrors this observation), even though 
it should. Because of this, I think I can conclude that the caching state is 
not correct for this use case and the bug is not purely cosmetic. 

Again, this only happens when the action after the second persist is called on 
a child RDD , so will not pop up very often. The first persist works correct. 
The second persist also works correct if the action is called on the RDD in 
question. It only fails on the combination of second persist while the action 
is on a child RDD. 

> Persist on RDD fails the second time if the action is called on a child RDD 
> without showing a FAILED message
> ------------------------------------------------------------------------------------------------------------
>
>                 Key: SPARK-7002
>                 URL: https://issues.apache.org/jira/browse/SPARK-7002
>             Project: Spark
>          Issue Type: Bug
>          Components: Spark Core
>    Affects Versions: 1.3.0
>         Environment: Platform: Power8
> OS: Ubuntu 14.10
> Java: java-8-openjdk-ppc64el
>            Reporter: Tom Hubregtsen
>            Priority: Minor
>              Labels: disk, persist, unpersist
>
> The major issue is: Persist on RDD fails the second time if the action is 
> called on a child RDD without showing a FAILED message. This is pointed out 
> at 2)
> next to this:
> toDebugString on a child RDD does not show that the parent RDD is [Disk 
> Serialized 1x Replicated]. This is pointed out at 1)
> Note: I am persisting to disk (DISK_ONLY) to validate that the RDD is or is 
> not physically stored, as I did not want to solely rely on a missing line in 
> .toDebugString (see comments in trace)
> scala> val rdd1 = sc.parallelize(List(1,2,3))
> scala> val rdd2 = rdd1.map(x => (x,x+1))
> scala> val rdd3 = rdd2.reduceByKey( (x,y) => x+y)
> scala> import org.apache.spark.storage.StorageLevel
> scala> rdd2.persist(StorageLevel.DISK_ONLY)
> scala> rdd3.collect()
> scala> rdd2.toDebugString
> res4: String = 
> (100) MapPartitionsRDD[1] at map at <console>:23 [Disk Serialized 1x 
> Replicated]
>   \|        CachedPartitions: 100; MemorySize: 0.0 B; TachyonSize: 0.0 B; 
> DiskSize: 802.0 B
>   \|   ParallelCollectionRDD[0] at parallelize at <console>:21 [Disk 
> Serialized 1x Replicated]
> scala> rdd3.toDebugString
> res5: String = 
> (100) ShuffledRDD[2] at reduceByKey at <console>:25 []
>   +-(100) MapPartitionsRDD[1] at map at <console>:23 []
>       \|       CachedPartitions: 100; MemorySize: 0.0 B; TachyonSize: 0.0 B; 
> DiskSize: 802.0 B
>       \|   ParallelCollectionRDD[0] at parallelize at <console>:21 []
> // 1) rdd3 does not show that the other RDD's are [Disk Serialized 1x 
> Replicated], but the data is on disk. This is verified by
> // a) The line starting with CachedPartitions
> // b) a find in spark_local_dir: "find . -name "\*"  \| grep rdd" returns 
> "./spark-b39bcf9b-e7d7-4284-bdd2-1be7ac3cacef/blockmgr-4f4c0b1c-b47a-4972-b364-7179ea6e0873/1f/rdd_4_*",
>  where * are the number of partitions
> scala> rdd2.unpersist()
> scala> rdd2.toDebugString
> res8: String = 
> (100) MapPartitionsRDD[1] at map at <console>:23 []
>   \|   ParallelCollectionRDD[0] at parallelize at <console>:21 []
> scala> rdd3.toDebugString
> res9: String = 
> (100) ShuffledRDD[2] at reduceByKey at <console>:25 []
>   +-(100) MapPartitionsRDD[1] at map at <console>:23 []
>       \|   ParallelCollectionRDD[0] at parallelize at <console>:21 []
> // successfully unpersisted, also not visible on disk
> scala> rdd2.persist(StorageLevel.DISK_ONLY)
> scala> rdd3.collect()
> scala> rdd2.toDebugString
> res18: String = 
> (100) MapPartitionsRDD[1] at map at <console>:23 [Disk Serialized 1x 
> Replicated]
>   \|   ParallelCollectionRDD[0] at parallelize at <console>:21 [Disk 
> Serialized 1x Replicated]
> scala> rdd3.toDebugString
> res19: String = 
> (100) ShuffledRDD[2] at reduceByKey at <console>:25 []
>   +-(100) MapPartitionsRDD[1] at map at <console>:23 []
>       \|   ParallelCollectionRDD[0] at parallelize at <console>:21 []
> // 2) The data is not visible on disk though the find command previously 
> mentioned, and is also not mentioned in the toDebugString (no line starting 
> with CachedPartitions, even though  [Disk Serialized 1x Replicated] is 
> mentioned). It does work when you call the action on the actual RDD:
> scala> rdd2.collect()
> scala> rdd2.toDebugString
> res21: String = 
> (100) MapPartitionsRDD[1] at map at <console>:23 [Disk Serialized 1x 
> Replicated]
>   \|        CachedPartitions: 100; MemorySize: 0.0 B; TachyonSize: 0.0 B; 
> DiskSize: 802.0 B
>   \|   ParallelCollectionRDD[0] at parallelize at <console>:21 [Disk 
> Serialized 1x Replicated]
> scala> rdd3.toDebugString
> res22: String = 
> (100) ShuffledRDD[2] at reduceByKey at <console>:25 []
>   +-(100) MapPartitionsRDD[1] at map at <console>:23 []
>       \|       CachedPartitions: 100; MemorySize: 0.0 B; TachyonSize: 0.0 B; 
> DiskSize: 802.0 B
>       \|   ParallelCollectionRDD[0] at parallelize at <console>:21 []
> // Data appears on disk again (using find command preciously mentioned), and 
> line with CachedPartitions is back in the .toDebugString



--
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