The reason why it worked before was because the UI would directly access
sc.getStorageStatus, instead of getting it through Task and Stage events.
This is not necessarily the best design, however, because the SparkContext
and the SparkUI are closely coupled, and there is no way to create a
SparkUI
got it. makes sense. i am surprised it worked before...
On Apr 18, 2014 9:12 PM, "Andrew Or" 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 someth
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, a
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
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
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 doi
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 wrote:
> i put some println statements in BlockManagerUI
>
> i hav
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;Tac
yes i am definitely using latest
On Tue, Apr 8, 2014 at 1:07 PM, Xiangrui Meng 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-distributio
yes i call an action after cache, and i can see that the RDDs are fully
cached using context.getRDDStorageInfo which we expose via our own api.
i did not run make-distribution.sh, we have our own scripts to build a
distribution. however if your question is if i correctly deployed the
latest build,
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 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 wrote:
> note that for a cached rdd in the
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 wrote:
> i tried again with latest master, which includes commit below, bu
That commit did work for me. Could you confirm the following:
1) After you called cache(), did you make any actions like count() or
reduce()? If you don't materialize the RDD, it won't show up in the
storage tab.
2) Did you run ./make-distribution.sh after you switched to the current master?
Xia
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
Date: Mon Mar 31 23:01:14 2014 -0700
[Hot Fix #42] Persisted RDD disappears on storage page if re-used
got it thanks
On Mon, Apr 7, 2014 at 4:08 PM, Xiangrui Meng 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 wrote:
> > i noticed that for spark 1.0.0-SNAPSHOT which i chec
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 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 show
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.
18 matches
Mail list logo