What RDD transformations trigger computations?

2014-12-18 Thread Alessandro Baretta
All, I noticed that while some operations that return RDDs are very cheap, such as map and flatMap, some are quite expensive, such as union and groupByKey. I'm referring here to the cost of constructing the RDD scala value, not the cost of collecting the values contained in the RDD. This does not

Re: What RDD transformations trigger computations?

2014-12-18 Thread Josh Rosen
Could you provide an example?  These operations are lazy, in the sense that they don’t trigger Spark jobs: scala val a = sc.parallelize(1 to 1, 1).mapPartitions{ x = println(computed a!); x} a: org.apache.spark.rdd.RDD[Int] = MapPartitionsRDD[14] at mapPartitions at console:18 scala

Re: What RDD transformations trigger computations?

2014-12-18 Thread Reynold Xin
Alessandro was probably referring to some transformations whose implementations depend on some actions. For example: sortByKey requires sampling the data to get the histogram. There is a ticket tracking this: https://issues.apache.org/jira/browse/SPARK-2992 On Thu, Dec 18, 2014 at 11:52 AM,

Re: What RDD transformations trigger computations?

2014-12-18 Thread Mark Hamstra
SPARK-2992 is a good start, but it's not exhaustive. For example, zipWithIndex is also an eager transformation, and we occasionally see PRs suggesting additional eager transformations. On Thu, Dec 18, 2014 at 12:14 PM, Reynold Xin r...@databricks.com wrote: Alessandro was probably referring to

Re: What RDD transformations trigger computations?

2014-12-18 Thread Alessandro Baretta
Reynold, Yes, this exactly what I was referring to. I specifically noted this unexpected behavior with sortByKey. I also noted that union is unexpectedly very slow, taking several minutes to define the RDD: although it does not seem to trigger a spark computation per se, it seems to cause the