Xin [mailto:r...@databricks.com]
Sent: Tuesday, August 25, 2015 6:05 PM
To: Ulanov, Alexander
Cc: dev@spark.apache.org
Subject: Re: Dataframe aggregation with Tungsten unsafe
There are a lot of GC activity due to the non-code-gen path being sloppy about
garbage creation. This is not actually what
There are a lot of GC activity due to the non-code-gen path being sloppy
about garbage creation. This is not actually what happens, but just as an
example:
rdd.map { i: Int => i + 1 }
This under the hood becomes a closure that boxes on every input and every
output, creating two extra objects.
Th
Thank you for the explanation. The size if the 100M data is ~1.4GB in memory
and each worker has 32GB of memory. It seems to be a lot of free memory
available. I wonder how Spark can hit GC with such setup?
Reynold Xin mailto:r...@databricks.com>>
On Fri, Aug 21, 2015 at 11:07 AM, Ulanov, Alex
On Fri, Aug 21, 2015 at 11:07 AM, Ulanov, Alexander wrote:
>
>
> It seems that there is a nice improvement with Tungsten enabled given that
> data is persisted in memory 2x and 3x. However, the improvement is not that
> nice for parquet, it is 1.5x. What’s interesting, with Tungsten enabled
> per
t on
this? It seems counterintuitive to me.
Local performance was not as good as Reynold had. I have around 1.5x, he had
5x. However, local mode is not interesting.
From: Reynold Xin [mailto:r...@databricks.com]
Sent: Thursday, August 20, 2015 9:24 PM
To: Ulanov, Alexander
Cc: dev@spark.apa
park.sql.codegen true
>
> spark.sql.unsafe.enabledtrue
>
> spark.unsafe.offHeaptrue
>
>
>
> Unsafe off:
>
> spark.sql.codegen false
>
> spark.sql.unsafe.enabledfalse
>
> spark.unsafe.offHeapfalse
>
>
>
> *From:* Reynol
t 20, 2015 5:43 PM
To: Ulanov, Alexander
Cc: dev@spark.apache.org
Subject: Re: Dataframe aggregation with Tungsten unsafe
Please git pull :)
On Thu, Aug 20, 2015 at 5:35 PM, Ulanov, Alexander
mailto:alexander.ula...@hp.com>> wrote:
I am using Spark 1.5 cloned from master on June 12. (The
August 20, 2015 5:26 PM
>
> *To:* Ulanov, Alexander
> *Cc:* dev@spark.apache.org
> *Subject:* Re: Dataframe aggregation with Tungsten unsafe
>
>
>
> Yes - DataFrame and SQL are the same thing.
>
>
>
> Which version are you running? Spark 1.4 doesn't run Jan
.apache.org>
Subject: Re: Dataframe aggregation with Tungsten unsafe
I think you might need to turn codegen on also in order for the unsafe stuff to
work.
On Thu, Aug 20, 2015 at 4:09 PM, Ulanov, Alexander
mailto:alexander.ula...@hp.com>> wrote:
Hi Reynold,
Thank you for suggestion. Thi
893)
>
> ... 66 more
>
> Caused by: java.lang.ClassNotFoundException: Override
>
> at java.lang.ClassLoader.findClass(ClassLoader.java:531)
>
> at
> org.apache.spark.util.ParentClassLoader.findClass(ParentClassLoader.scala:26)
>
> at java.lang.Cl
: Re: Dataframe aggregation with Tungsten unsafe
I think you might need to turn codegen on also in order for the unsafe stuff to
work.
On Thu, Aug 20, 2015 at 4:09 PM, Ulanov, Alexander
mailto:alexander.ula...@hp.com>> wrote:
Hi Reynold,
Thank you for suggestion. This code takes around 30
I think you might need to turn codegen on also in order for the unsafe
stuff to work.
On Thu, Aug 20, 2015 at 4:09 PM, Ulanov, Alexander
wrote:
> Hi Reynold,
>
> Thank you for suggestion. This code takes around 30 sec on my setup (5
> workers with 32GB). My issue is that I don't see the change
Hi Reynold,
Thank you for suggestion. This code takes around 30 sec on my setup (5 workers
with 32GB). My issue is that I don't see the change in time if I unset the
unsafe flags. Could you explain why it might happen?
20 авг. 2015 г., в 15:32, Reynold Xin
mailto:r...@databricks.com>> написал(
BTW one other thing -- don't use the count() to do benchmark, since the
optimizer is smart enough to figure out that you don't actually need to run
the sum.
For the purpose of benchmarking, you can use
df.foreach(i => do nothing)
On Thu, Aug 20, 2015 at 3:31 PM, Reynold Xin wrote:
> I did
I didn't wait long enough earlier. Actually it did finish when I raised
memory to 8g.
In 1.5 with Tungsten (which should be the same as 1.4 with your unsafe
flags), the query took 40s with 4G of mem.
In 1.4, it took 195s with 8G of mem.
This is not a scientific benchmark and I only ran it once.
How did you run this? I couldn't run your query with 4G of RAM in 1.4, but
in 1.5 it ran.
Also I recommend just dumping the data to parquet on disk to evaluate,
rather than using the in-memory cache, which is super slow and we are
thinking of removing/replacing with something else.
val size = 10
Dear Spark developers,
I am trying to benchmark the new Dataframe aggregation implemented under the
project Tungsten and released with Spark 1.4 (I am using the latest Spark from
the repo, i.e. 1.5):
https://github.com/apache/spark/pull/5725
It tells that the aggregation should be faster due to
17 matches
Mail list logo