[jira] [Commented] (SPARK-7116) Intermediate RDD cached but never unpersisted

2015-05-07 Thread Apache Spark (JIRA)

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

Apache Spark commented on SPARK-7116:
-

User 'ksonj' has created a pull request for this issue:
https://github.com/apache/spark/pull/5973

> Intermediate RDD cached but never unpersisted
> -
>
> Key: SPARK-7116
> URL: https://issues.apache.org/jira/browse/SPARK-7116
> Project: Spark
>  Issue Type: Improvement
>  Components: PySpark, SQL
>Affects Versions: 1.3.1
>Reporter: Kalle Jepsen
>
> In 
> https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala#L233
>  an intermediate RDD is cached, but never unpersisted. It shows up in the 
> 'Storage' section of the Web UI, but cannot be removed. There's already a 
> comment in the source, suggesting to 'clean up'. If that cleanup is more 
> involved than simply calling `unpersist`, it probably exceeds my current 
> Scala skills.
> Why that is a problem:
> I'm adding a constant column to a DataFrame of about 20M records resulting 
> from an inner join with {{df.withColumn(colname, ud_func())}} , where 
> {{ud_func}} is simply a wrapped {{lambda: 1}}. Before and after applying the 
> UDF the DataFrame takes up ~430MB in the cache. The cached intermediate RDD 
> however takes up ~10GB(!) of storage, and I know of no way to uncache it.



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



[jira] [Commented] (SPARK-7116) Intermediate RDD cached but never unpersisted

2015-05-07 Thread Kalle Jepsen (JIRA)

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

Kalle Jepsen commented on SPARK-7116:
-

Sure, thanks

> Intermediate RDD cached but never unpersisted
> -
>
> Key: SPARK-7116
> URL: https://issues.apache.org/jira/browse/SPARK-7116
> Project: Spark
>  Issue Type: Improvement
>  Components: PySpark, SQL
>Affects Versions: 1.3.1
>Reporter: Kalle Jepsen
>
> In 
> https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala#L233
>  an intermediate RDD is cached, but never unpersisted. It shows up in the 
> 'Storage' section of the Web UI, but cannot be removed. There's already a 
> comment in the source, suggesting to 'clean up'. If that cleanup is more 
> involved than simply calling `unpersist`, it probably exceeds my current 
> Scala skills.
> Why that is a problem:
> I'm adding a constant column to a DataFrame of about 20M records resulting 
> from an inner join with {{df.withColumn(colname, ud_func())}} , where 
> {{ud_func}} is simply a wrapped {{lambda: 1}}. Before and after applying the 
> UDF the DataFrame takes up ~430MB in the cache. The cached intermediate RDD 
> however takes up ~10GB(!) of storage, and I know of no way to uncache it.



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



[jira] [Commented] (SPARK-7116) Intermediate RDD cached but never unpersisted

2015-05-06 Thread Michael Armbrust (JIRA)

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

Michael Armbrust commented on SPARK-7116:
-

[~kalle], it seems reasonable to remove it.  If you you can open a PR we can 
try to include it in 1.4.0.

> Intermediate RDD cached but never unpersisted
> -
>
> Key: SPARK-7116
> URL: https://issues.apache.org/jira/browse/SPARK-7116
> Project: Spark
>  Issue Type: Improvement
>  Components: PySpark, SQL
>Affects Versions: 1.3.1
>Reporter: Kalle Jepsen
>
> In 
> https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala#L233
>  an intermediate RDD is cached, but never unpersisted. It shows up in the 
> 'Storage' section of the Web UI, but cannot be removed. There's already a 
> comment in the source, suggesting to 'clean up'. If that cleanup is more 
> involved than simply calling `unpersist`, it probably exceeds my current 
> Scala skills.
> Why that is a problem:
> I'm adding a constant column to a DataFrame of about 20M records resulting 
> from an inner join with {{df.withColumn(colname, ud_func())}} , where 
> {{ud_func}} is simply a wrapped {{lambda: 1}}. Before and after applying the 
> UDF the DataFrame takes up ~430MB in the cache. The cached intermediate RDD 
> however takes up ~10GB(!) of storage, and I know of no way to uncache it.



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



[jira] [Commented] (SPARK-7116) Intermediate RDD cached but never unpersisted

2015-05-06 Thread Kalle Jepsen (JIRA)

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

Kalle Jepsen commented on SPARK-7116:
-

[~marmbrus] Do you remember why that {{cache()}} was necessary? I've boldly 
commented it out and the only thing that seems to have changed is that 
everything runs a lot faster...

> Intermediate RDD cached but never unpersisted
> -
>
> Key: SPARK-7116
> URL: https://issues.apache.org/jira/browse/SPARK-7116
> Project: Spark
>  Issue Type: Improvement
>  Components: PySpark, SQL
>Affects Versions: 1.3.1
>Reporter: Kalle Jepsen
>
> In 
> https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala#L233
>  an intermediate RDD is cached, but never unpersisted. It shows up in the 
> 'Storage' section of the Web UI, but cannot be removed. There's already a 
> comment in the source, suggesting to 'clean up'. If that cleanup is more 
> involved than simply calling `unpersist`, it probably exceeds my current 
> Scala skills.
> Why that is a problem:
> I'm adding a constant column to a DataFrame of about 20M records resulting 
> from an inner join with {{df.withColumn(colname, ud_func())}} , where 
> {{ud_func}} is simply a wrapped {{lambda: 1}}. Before and after applying the 
> UDF the DataFrame takes up ~430MB in the cache. The cached intermediate RDD 
> however takes up ~10GB(!) of storage, and I know of no way to uncache it.



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



[jira] [Commented] (SPARK-7116) Intermediate RDD cached but never unpersisted

2015-05-06 Thread Dennis Proppe (JIRA)

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

Dennis Proppe commented on SPARK-7116:
--

*I second the importance of fixing this.*

For our production workloads, this is really a large issue and blocks the use 
of large data:

1) It creates non-removeable RDDs cached in memory
2) They can't even be removed by deleting the RDD in question in the code
3) The RDDs are up to 10 times bigger that they would be by calling df.cache()

The behaviour can be reproduced easily in the PySpark Shell:


 from pyspark import SQLContext
 from pyspark.sql.types import IntegerType
 from pyspark.sql.functions import udf



sqlcon = SQLContext(sc)
slen = udf(lambda s: len(s), IntegerType())

rows = [["a", "hello"], ["b", "goodbye"],["c", "whatever"]] * 1000
rd = sc.parallelize(rows)
head = ['order','word']
df = rd.toDF(head)
df.cache().count()

`
bigfile = df.withColumn("word_length", slen(df.word))
bigfile.count()
`

If you now compare the size of the cached df versus the automagically appeared 
cached bigfile, you see that bigfile uses about 8X the storage of df, although 
it only has 1 column more.

> Intermediate RDD cached but never unpersisted
> -
>
> Key: SPARK-7116
> URL: https://issues.apache.org/jira/browse/SPARK-7116
> Project: Spark
>  Issue Type: Improvement
>  Components: PySpark, SQL
>Affects Versions: 1.3.1
>Reporter: Kalle Jepsen
>
> In 
> https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala#L233
>  an intermediate RDD is cached, but never unpersisted. It shows up in the 
> 'Storage' section of the Web UI, but cannot be removed. There's already a 
> comment in the source, suggesting to 'clean up'. If that cleanup is more 
> involved than simply calling `unpersist`, it probably exceeds my current 
> Scala skills.
> Why that is a problem:
> I'm adding a constant column to a DataFrame of about 20M records resulting 
> from an inner join with {{df.withColumn(colname, ud_func())}} , where 
> {{ud_func}} is simply a wrapped {{lambda: 1}}. Before and after applying the 
> UDF the DataFrame takes up ~430MB in the cache. The cached intermediate RDD 
> however takes up ~10GB(!) of storage, and I know of no way to uncache it.



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



[jira] [Commented] (SPARK-7116) Intermediate RDD cached but never unpersisted

2015-04-25 Thread Joseph K. Bradley (JIRA)

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

Joseph K. Bradley commented on SPARK-7116:
--

[~davies] Is there any good way to fix this?  It looks like no action has been 
performed on the persisted RDD's children before the method exits, so 
unpersisting might not be a good idea.

> Intermediate RDD cached but never unpersisted
> -
>
> Key: SPARK-7116
> URL: https://issues.apache.org/jira/browse/SPARK-7116
> Project: Spark
>  Issue Type: Improvement
>  Components: PySpark, SQL
>Affects Versions: 1.3.1
>Reporter: Kalle Jepsen
>
> In 
> https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala#L233
>  an intermediate RDD is cached, but never unpersisted. It shows up in the 
> 'Storage' section of the Web UI, but cannot be removed. There's already a 
> comment in the source, suggesting to 'clean up'. If that cleanup is more 
> involved than simply calling `unpersist`, it probably exceeds my current 
> Scala skills.
> Why that is a problem:
> I'm adding a constant column to a DataFrame of about 20M records resulting 
> from an inner join with {{df.withColumn(colname, ud_func())}} , where 
> {{ud_func}} is simply a wrapped {{lambda: 1}}. Before and after applying the 
> UDF the DataFrame takes up ~430MB in the cache. The cached intermediate RDD 
> however takes up ~10GB(!) of storage, and I know of no way to uncache it.



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