got it. makes sense. i am surprised it worked before...
On Apr 18, 2014 9:12 PM, "Andrew Or" <and...@databricks.com> wrote:

> Hi Koert,
>
> I've tracked down what the bug is. The caveat is that each StageInfo only
> keeps around the RDDInfo of the last RDD associated with the Stage. More
> concretely, if you have something like
>
> sc.parallelize(1 to 1000).persist.map(i => (i, i)).count()
>
> This creates two RDDs within one Stage, and the persisted RDD doesn't show
> up on the UI because it is not the last RDD of this stage. I filed a JIRA
> for this here: https://issues.apache.org/jira/browse/SPARK-1538.
>
> Thanks again for reporting this. I will push out a fix shortly.
> Andrew
>
>
> On Tue, Apr 8, 2014 at 1:30 PM, Koert Kuipers <ko...@tresata.com> wrote:
>
>> our one cached RDD in this run has id 3
>>
>>
>>
>>
>> ******************* onStageSubmitted **********************
>> rddInfo: RDD "2" (2) Storage: StorageLevel(false, false, false, false,
>> 1); CachedPartitions: 0; TotalPartitions: 1; MemorySize: 0.0 B;TachyonSize:
>> 0.0 B; DiskSize: 0.0 B
>> _rddInfoMap: Map(2 -> RDD "2" (2) Storage: StorageLevel(false, false,
>> false, false, 1); CachedPartitions: 0; TotalPartitions: 1; MemorySize: 0.0
>> B;TachyonSize: 0.0 B; DiskSize: 0.0 B)
>>
>>
>> ******************* onTaskEnd **********************
>> _rddInfoMap: Map(2 -> RDD "2" (2) Storage: StorageLevel(false, false,
>> false, false, 1); CachedPartitions: 0; TotalPartitions: 1; MemorySize: 0.0
>> B;TachyonSize: 0.0 B; DiskSize: 0.0 B)
>> storageStatusList: List(StorageStatus(BlockManagerId(<driver>,
>> 192.168.3.169, 34330, 0),579325132,Map()))
>>
>>
>> ******************* onStageCompleted **********************
>> _rddInfoMap: Map()
>>
>>
>>
>> ******************* onStageSubmitted **********************
>> rddInfo: RDD "7" (7) Storage: StorageLevel(false, false, false, false,
>> 1); CachedPartitions: 0; TotalPartitions: 1; MemorySize: 0.0 B;TachyonSize:
>> 0.0 B; DiskSize: 0.0 B
>> _rddInfoMap: Map(7 -> RDD "7" (7) Storage: StorageLevel(false, false,
>> false, false, 1); CachedPartitions: 0; TotalPartitions: 1; MemorySize: 0.0
>> B;TachyonSize: 0.0 B; DiskSize: 0.0 B)
>>
>>
>> ******************* updateRDDInfo **********************
>>
>>
>> ******************* onTaskEnd **********************
>>
>> _rddInfoMap: Map(7 -> RDD "7" (7) Storage: StorageLevel(false, false,
>> false, false, 1); CachedPartitions: 0; TotalPartitions: 1; MemorySize: 0.0
>> B;TachyonSize: 0.0 B; DiskSize: 0.0 B)
>>  storageStatusList: List(StorageStatus(BlockManagerId(<driver>,
>> 192.168.3.169, 34330, 0),579325132,Map(rdd_3_0 ->
>> BlockStatus(StorageLevel(false, true, false, true, 1),19944,0,0))))
>>
>>
>> ******************* onStageCompleted **********************
>> _rddInfoMap: Map()
>>
>>
>>
>> On Tue, Apr 8, 2014 at 4:20 PM, Koert Kuipers <ko...@tresata.com> wrote:
>>
>>> 1) at the end of the callback
>>>
>>> 2) yes we simply expose sc.getRDDStorageInfo to the user via REST
>>>
>>> 3) yes exactly. we define the RDDs at startup, all of them are cached.
>>> from that point on we only do calculations on these cached RDDs.
>>>
>>> i will add some more println statements for storageStatusList
>>>
>>>
>>>
>>> On Tue, Apr 8, 2014 at 4:01 PM, Andrew Or <and...@databricks.com> wrote:
>>>
>>>> Hi Koert,
>>>>
>>>> Thanks for pointing this out. However, I am unable to reproduce this
>>>> locally. It seems that there is a discrepancy between what the
>>>> BlockManagerUI and the SparkContext think is persisted. This is strange
>>>> because both sources ultimately derive this information from the same place
>>>> - by doing sc.getExecutorStorageStatus. I have a couple of questions for
>>>> you:
>>>>
>>>> 1) In your print statements, do you print them in the beginning or at
>>>> the end of each callback? It would be good to keep them at the end, since
>>>> in the beginning the data structures have not been processed yet.
>>>> 2) You mention that you get the RDD info through your own API. How do
>>>> you get this information? Is it through sc.getRDDStorageInfo?
>>>> 3) What did your application do to produce this behavior? Did you make
>>>> an RDD, persist it once, and then use it many times afterwards or something
>>>> similar?
>>>>
>>>> It would be super helpful if you could also print out what
>>>> StorageStatusListener's storageStatusList looks like by the end of each
>>>> onTaskEnd. I will continue to look into this on my side, but do let me know
>>>> once you have any updates.
>>>>
>>>> Andrew
>>>>
>>>>
>>>> On Tue, Apr 8, 2014 at 11:26 AM, Koert Kuipers <ko...@tresata.com>wrote:
>>>>
>>>>> yet at same time i can see via our own api:
>>>>>
>>>>>     "storageInfo": {
>>>>>         "diskSize": 0,
>>>>>         "memSize": 19944,
>>>>>         "numCachedPartitions": 1,
>>>>>         "numPartitions": 1
>>>>>     }
>>>>>
>>>>>
>>>>>
>>>>> On Tue, Apr 8, 2014 at 2:25 PM, Koert Kuipers <ko...@tresata.com>wrote:
>>>>>
>>>>>> i put some println statements in BlockManagerUI
>>>>>>
>>>>>> i have RDDs that are cached in memory. I see this:
>>>>>>
>>>>>>
>>>>>> ******************* onStageSubmitted **********************
>>>>>> rddInfo: RDD "2" (2) Storage: StorageLevel(false, false, false,
>>>>>> false, 1); CachedPartitions: 0; TotalPartitions: 1; MemorySize: 0.0
>>>>>> B;TachyonSize: 0.0 B; DiskSize: 0.0 B
>>>>>> _rddInfoMap: Map(2 -> RDD "2" (2) Storage: StorageLevel(false, false,
>>>>>> false, false, 1); CachedPartitions: 0; TotalPartitions: 1; MemorySize: 
>>>>>> 0.0
>>>>>> B;TachyonSize: 0.0 B; DiskSize: 0.0 B)
>>>>>>
>>>>>>
>>>>>> ******************* onTaskEnd **********************
>>>>>> Map(2 -> RDD "2" (2) Storage: StorageLevel(false, false, false,
>>>>>> false, 1); CachedPartitions: 0; TotalPartitions: 1; MemorySize: 0.0
>>>>>> B;TachyonSize: 0.0 B; DiskSize: 0.0 B)
>>>>>>
>>>>>>
>>>>>> ******************* onStageCompleted **********************
>>>>>> Map()
>>>>>>
>>>>>> ******************* onStageSubmitted **********************
>>>>>> rddInfo: RDD "7" (7) Storage: StorageLevel(false, false, false,
>>>>>> false, 1); CachedPartitions: 0; TotalPartitions: 1; MemorySize: 0.0
>>>>>> B;TachyonSize: 0.0 B; DiskSize: 0.0 B
>>>>>> _rddInfoMap: Map(7 -> RDD "7" (7) Storage: StorageLevel(false, false,
>>>>>> false, false, 1); CachedPartitions: 0; TotalPartitions: 1; MemorySize: 
>>>>>> 0.0
>>>>>> B;TachyonSize: 0.0 B; DiskSize: 0.0 B)
>>>>>>
>>>>>> ******************* onTaskEnd **********************
>>>>>> Map(7 -> RDD "7" (7) Storage: StorageLevel(false, false, false,
>>>>>> false, 1); CachedPartitions: 0; TotalPartitions: 1; MemorySize: 0.0
>>>>>> B;TachyonSize: 0.0 B; DiskSize: 0.0 B)
>>>>>>
>>>>>> ******************* onStageCompleted **********************
>>>>>> Map()
>>>>>>
>>>>>>
>>>>>> The storagelevels you see here are never the ones of my RDDs. and
>>>>>> apparently updateRDDInfo never gets called (i had println in there too).
>>>>>>
>>>>>>
>>>>>> On Tue, Apr 8, 2014 at 2:13 PM, Koert Kuipers <ko...@tresata.com>wrote:
>>>>>>
>>>>>>> yes i am definitely using latest
>>>>>>>
>>>>>>>
>>>>>>> On Tue, Apr 8, 2014 at 1:07 PM, Xiangrui Meng <men...@gmail.com>wrote:
>>>>>>>
>>>>>>>> That commit fixed the exact problem you described. That is why I
>>>>>>>> want to confirm that you switched to the master branch. bin/spark-shell
>>>>>>>> doesn't detect code changes, so you need to run ./make-distribution.sh 
>>>>>>>> to
>>>>>>>> re-compile Spark first. -Xiangrui
>>>>>>>>
>>>>>>>>
>>>>>>>> On Tue, Apr 8, 2014 at 9:57 AM, Koert Kuipers <ko...@tresata.com>wrote:
>>>>>>>>
>>>>>>>>> sorry, i meant to say: note that for a cached rdd in the spark
>>>>>>>>> shell it all works fine. but something is going wrong with the
>>>>>>>>> SPARK-APPLICATION-UI in our applications that extensively cache and 
>>>>>>>>> re-use
>>>>>>>>> RDDs
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Tue, Apr 8, 2014 at 12:55 PM, Koert Kuipers 
>>>>>>>>> <ko...@tresata.com>wrote:
>>>>>>>>>
>>>>>>>>>> note that for a cached rdd in the spark shell it all works fine.
>>>>>>>>>> but something is going wrong with the spark-shell in our 
>>>>>>>>>> applications that
>>>>>>>>>> extensively cache and re-use RDDs
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Tue, Apr 8, 2014 at 12:33 PM, Koert Kuipers <ko...@tresata.com
>>>>>>>>>> > wrote:
>>>>>>>>>>
>>>>>>>>>>> i tried again with latest master, which includes commit below,
>>>>>>>>>>> but ui page still shows nothing on storage tab.
>>>>>>>>>>>  koert
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> commit ada310a9d3d5419e101b24d9b41398f609da1ad3
>>>>>>>>>>> Author: Andrew Or <andrewo...@gmail.com>
>>>>>>>>>>> Date:   Mon Mar 31 23:01:14 2014 -0700
>>>>>>>>>>>
>>>>>>>>>>>     [Hot Fix #42] Persisted RDD disappears on storage page if
>>>>>>>>>>> re-used
>>>>>>>>>>>
>>>>>>>>>>>     If a previously persisted RDD is re-used, its information
>>>>>>>>>>> disappears from the Storage page.
>>>>>>>>>>>
>>>>>>>>>>>     This is because the tasks associated with re-using the RDD
>>>>>>>>>>> do not report the RDD's blocks as updated (which is correct). On 
>>>>>>>>>>> stage
>>>>>>>>>>> submit, however, we overwrite any existing
>>>>>>>>>>>
>>>>>>>>>>>     Author: Andrew Or <andrewo...@gmail.com>
>>>>>>>>>>>
>>>>>>>>>>>     Closes #281 from andrewor14/ui-storage-fix and squashes the
>>>>>>>>>>> following commits:
>>>>>>>>>>>
>>>>>>>>>>>     408585a [Andrew Or] Fix storage UI bug
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Mon, Apr 7, 2014 at 4:21 PM, Koert Kuipers <ko...@tresata.com
>>>>>>>>>>> > wrote:
>>>>>>>>>>>
>>>>>>>>>>>> got it thanks
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Mon, Apr 7, 2014 at 4:08 PM, Xiangrui Meng <men...@gmail.com
>>>>>>>>>>>> > wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> This is fixed in https://github.com/apache/spark/pull/281.
>>>>>>>>>>>>> Please try
>>>>>>>>>>>>> again with the latest master. -Xiangrui
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Mon, Apr 7, 2014 at 1:06 PM, Koert Kuipers <
>>>>>>>>>>>>> ko...@tresata.com> wrote:
>>>>>>>>>>>>> > i noticed that for spark 1.0.0-SNAPSHOT which i checked out
>>>>>>>>>>>>> a few days ago
>>>>>>>>>>>>> > (apr 5) that the "application detail ui" no longer shows any
>>>>>>>>>>>>> RDDs on the
>>>>>>>>>>>>> > storage tab, despite the fact that they are definitely
>>>>>>>>>>>>> cached.
>>>>>>>>>>>>> >
>>>>>>>>>>>>> > i am running spark in standalone mode.
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
>

Reply via email to