You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@spark.apache.org by zsampson <zs...@palantir.com> on 2015/06/02 21:34:55 UTC

DataFrame.withColumn very slow when used iteratively?

Hey,

I'm seeing extreme slowness in withColumn when it's used in a loop. I'm
running this code:

for (int i = 0; i < NUM_ITERATIONS ++i) {
    df = df.withColumn("col"+i, new Column(new Literal(i,
DataTypes.IntegerType)));
}

where df is initially a trivial dataframe. Here are the results of running
with different values of NUM_ITERATIONS:

iterations	time
25	3s
50	11s
75	31s
100	76s
125	159s
150	283s

When I update the DataFrame by manually copying/appending to the column
array and using DataFrame.select, it runs in about half the time, but this
is still untenable at any significant number of iterations.

Any insight?



--
View this message in context: http://apache-spark-developers-list.1001551.n3.nabble.com/DataFrame-withColumn-very-slow-when-used-iteratively-tp12562.html
Sent from the Apache Spark Developers List mailing list archive at Nabble.com.

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@spark.apache.org
For additional commands, e-mail: dev-help@spark.apache.org


Re: DataFrame.withColumn very slow when used iteratively?

Posted by Reynold Xin <rx...@databricks.com>.
.select itself is the bulk add right?

On Tue, Jun 2, 2015 at 5:32 PM, Andrew Ash <an...@andrewash.com> wrote:

> Would it be valuable to create a .withColumns([colName], [ColumnObject])
> method that adds in bulk rather than iteratively?
>
> Alternatively effort might be better spent in making .withColumn()
> singular faster.
>
> On Tue, Jun 2, 2015 at 3:46 PM, Reynold Xin <rx...@databricks.com> wrote:
>
>> We improved this in 1.4. Adding 100 columns took 4s on my laptop.
>> https://issues.apache.org/jira/browse/SPARK-7276
>>
>> Still not the fastest, but much faster.
>>
>> scala> Seq((1, 2)).toDF("a", "b")
>> res6: org.apache.spark.sql.DataFrame = [a: int, b: int]
>>
>> scala>
>>
>> scala> val start = System.nanoTime
>> start: Long = 1433274299441224000
>>
>> scala> for (i <- 1 to 100) {
>>      |   df = df.withColumn("n" + i,
>> org.apache.spark.sql.functions.lit(0))
>>      | }
>>
>> scala> val end = System.nanoTime
>> end: Long = 1433274303250091000
>>
>> scala>
>>
>> scala> println((end - start) / 1000 / 1000 / 1000)
>> 3
>>
>>
>> On Tue, Jun 2, 2015 at 12:34 PM, zsampson <zs...@palantir.com> wrote:
>>
>>> Hey,
>>>
>>> I'm seeing extreme slowness in withColumn when it's used in a loop. I'm
>>> running this code:
>>>
>>> for (int i = 0; i < NUM_ITERATIONS ++i) {
>>> df = df.withColumn("col"+i, new Column(new Literal(i,
>>> DataTypes.IntegerType)));
>>> }
>>>
>>> where df is initially a trivial dataframe. Here are the results of
>>> running
>>> with different values of NUM_ITERATIONS:
>>>
>>> iterations      time
>>> 25      3s
>>> 50      11s
>>> 75      31s
>>> 100     76s
>>> 125     159s
>>> 150     283s
>>>
>>> When I update the DataFrame by manually copying/appending to the column
>>> array and using DataFrame.select, it runs in about half the time, but
>>> this
>>> is still untenable at any significant number of iterations.
>>>
>>> Any insight?
>>>
>>>
>>>
>>> --
>>> View this message in context:
>>> http://apache-spark-developers-list.1001551.n3.nabble.com/DataFrame-withColumn-very-slow-when-used-iteratively-tp12562.html
>>> Sent from the Apache Spark Developers List mailing list archive at
>>> Nabble.com.
>>>
>>> ---------------------------------------------------------------------
>>> To unsubscribe, e-mail: dev-unsubscribe@spark.apache.org
>>> For additional commands, e-mail: dev-help@spark.apache.org
>>>
>>>
>>
>

Re: DataFrame.withColumn very slow when used iteratively?

Posted by Andrew Ash <an...@andrewash.com>.
Would it be valuable to create a .withColumns([colName], [ColumnObject])
method that adds in bulk rather than iteratively?

Alternatively effort might be better spent in making .withColumn() singular
faster.

On Tue, Jun 2, 2015 at 3:46 PM, Reynold Xin <rx...@databricks.com> wrote:

> We improved this in 1.4. Adding 100 columns took 4s on my laptop.
> https://issues.apache.org/jira/browse/SPARK-7276
>
> Still not the fastest, but much faster.
>
> scala> Seq((1, 2)).toDF("a", "b")
> res6: org.apache.spark.sql.DataFrame = [a: int, b: int]
>
> scala>
>
> scala> val start = System.nanoTime
> start: Long = 1433274299441224000
>
> scala> for (i <- 1 to 100) {
>      |   df = df.withColumn("n" + i, org.apache.spark.sql.functions.lit(0))
>      | }
>
> scala> val end = System.nanoTime
> end: Long = 1433274303250091000
>
> scala>
>
> scala> println((end - start) / 1000 / 1000 / 1000)
> 3
>
>
> On Tue, Jun 2, 2015 at 12:34 PM, zsampson <zs...@palantir.com> wrote:
>
>> Hey,
>>
>> I'm seeing extreme slowness in withColumn when it's used in a loop. I'm
>> running this code:
>>
>> for (int i = 0; i < NUM_ITERATIONS ++i) {
>> df = df.withColumn("col"+i, new Column(new Literal(i,
>> DataTypes.IntegerType)));
>> }
>>
>> where df is initially a trivial dataframe. Here are the results of running
>> with different values of NUM_ITERATIONS:
>>
>> iterations      time
>> 25      3s
>> 50      11s
>> 75      31s
>> 100     76s
>> 125     159s
>> 150     283s
>>
>> When I update the DataFrame by manually copying/appending to the column
>> array and using DataFrame.select, it runs in about half the time, but this
>> is still untenable at any significant number of iterations.
>>
>> Any insight?
>>
>>
>>
>> --
>> View this message in context:
>> http://apache-spark-developers-list.1001551.n3.nabble.com/DataFrame-withColumn-very-slow-when-used-iteratively-tp12562.html
>> Sent from the Apache Spark Developers List mailing list archive at
>> Nabble.com.
>>
>> ---------------------------------------------------------------------
>> To unsubscribe, e-mail: dev-unsubscribe@spark.apache.org
>> For additional commands, e-mail: dev-help@spark.apache.org
>>
>>
>

Re: DataFrame.withColumn very slow when used iteratively?

Posted by Reynold Xin <rx...@databricks.com>.
We improved this in 1.4. Adding 100 columns took 4s on my laptop.
https://issues.apache.org/jira/browse/SPARK-7276

Still not the fastest, but much faster.

scala> Seq((1, 2)).toDF("a", "b")
res6: org.apache.spark.sql.DataFrame = [a: int, b: int]

scala>

scala> val start = System.nanoTime
start: Long = 1433274299441224000

scala> for (i <- 1 to 100) {
     |   df = df.withColumn("n" + i, org.apache.spark.sql.functions.lit(0))
     | }

scala> val end = System.nanoTime
end: Long = 1433274303250091000

scala>

scala> println((end - start) / 1000 / 1000 / 1000)
3


On Tue, Jun 2, 2015 at 12:34 PM, zsampson <zs...@palantir.com> wrote:

> Hey,
>
> I'm seeing extreme slowness in withColumn when it's used in a loop. I'm
> running this code:
>
> for (int i = 0; i < NUM_ITERATIONS ++i) {
> df = df.withColumn("col"+i, new Column(new Literal(i,
> DataTypes.IntegerType)));
> }
>
> where df is initially a trivial dataframe. Here are the results of running
> with different values of NUM_ITERATIONS:
>
> iterations      time
> 25      3s
> 50      11s
> 75      31s
> 100     76s
> 125     159s
> 150     283s
>
> When I update the DataFrame by manually copying/appending to the column
> array and using DataFrame.select, it runs in about half the time, but this
> is still untenable at any significant number of iterations.
>
> Any insight?
>
>
>
> --
> View this message in context:
> http://apache-spark-developers-list.1001551.n3.nabble.com/DataFrame-withColumn-very-slow-when-used-iteratively-tp12562.html
> Sent from the Apache Spark Developers List mailing list archive at
> Nabble.com.
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: dev-unsubscribe@spark.apache.org
> For additional commands, e-mail: dev-help@spark.apache.org
>
>