Re: large volume spark job spends most of the time in AppendOnlyMap.changeValue

2015-05-12 Thread Matei Zaharia
It could also be that your hash function is expensive. What is the key class you have for the reduceByKey / groupByKey? Matei > On May 12, 2015, at 10:08 AM, Night Wolf wrote: > > I'm seeing a similar thing with a slightly different stack trace. Ideas? > > org.apache.spark.util.collection.App

Re: large volume spark job spends most of the time in AppendOnlyMap.changeValue

2015-05-12 Thread Night Wolf
I'm seeing a similar thing with a slightly different stack trace. Ideas? org.apache.spark.util.collection.AppendOnlyMap.changeValue(AppendOnlyMap.scala:150) org.apache.spark.util.collection.SizeTrackingAppendOnlyMap.changeValue(SizeTrackingAppendOnlyMap.scala:32) org.apache.spark.util.collection.E

Re: large volume spark job spends most of the time in AppendOnlyMap.changeValue

2015-05-11 Thread Reynold Xin
Looks like it is spending a lot of time doing hash probing. It could be a number of the following: 1. hash probing itself is inherently expensive compared with rest of your workload 2. murmur3 doesn't work well with this key distribution 3. quadratic probing (triangular sequence) with a power-of

Re: large volume spark job spends most of the time in AppendOnlyMap.changeValue

2015-05-11 Thread Michal Haris
This is the stack trace of the worker thread: org.apache.spark.util.collection.AppendOnlyMap.changeValue(AppendOnlyMap.scala:150) org.apache.spark.util.collection.SizeTrackingAppendOnlyMap.changeValue(SizeTrackingAppendOnlyMap.scala:32) org.apache.spark.util.collection.ExternalAppendOnlyMap.insert

Re: large volume spark job spends most of the time in AppendOnlyMap.changeValue

2015-05-08 Thread Josh Rosen
Do you have any more specific profiling data that you can share? I'm curious to know where AppendOnlyMap.changeValue is being called from. On Fri, May 8, 2015 at 1:26 PM, Michal Haris wrote: > +dev > On 6 May 2015 10:45, "Michal Haris" wrote: > > > Just wanted to check if somebody has seen sim

Re: large volume spark job spends most of the time in AppendOnlyMap.changeValue

2015-05-08 Thread Michal Haris
+dev On 6 May 2015 10:45, "Michal Haris" wrote: > Just wanted to check if somebody has seen similar behaviour or knows what > we might be doing wrong. We have a relatively complex spark application > which processes half a terabyte of data at various stages. We have profiled > it in several ways

large volume spark job spends most of the time in AppendOnlyMap.changeValue

2015-05-06 Thread Michal Haris
Just wanted to check if somebody has seen similar behaviour or knows what we might be doing wrong. We have a relatively complex spark application which processes half a terabyte of data at various stages. We have profiled it in several ways and everything seems to point to one place where 90% of th