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 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, Josh Rosen <rosenvi...@gmail.com> wrote: > > > > 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 10000, 1).mapPartitions{ x => > > println("computed a!"); x} > > a: org.apache.spark.rdd.RDD[Int] = MapPartitionsRDD[14] at mapPartitions > > at <console>:18 > > > > scala> a.union(a) > > res4: org.apache.spark.rdd.RDD[Int] = UnionRDD[15] at union at > <console>:22 > > > > scala> a.map(x => (x, x)).groupByKey() > > res5: org.apache.spark.rdd.RDD[(Int, Iterable[Int])] = ShuffledRDD[17] at > > groupByKey at <console>:22 > > > > scala> a.map(x => (x, x)).groupByKey().count() > > computed a! > > res6: Long = 10000 > > > > > > On December 18, 2014 at 1:04:54 AM, Alessandro Baretta ( > > alexbare...@gmail.com) wrote: > > > > 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 match > my > > understanding that RDD transformations only set up a computation without > > actually running it. Oh, Spark developers, can you please provide some > > clarity? > > > > Alex > > >