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 input files to read by the Hadoop subsystem, which to the console such messages as these:
14/12/18 05:52:49 INFO mapred.FileInputFormat: Total input paths to process : 9 14/12/18 05:54:15 INFO mapred.FileInputFormat: Total input paths to process : 759 14/12/18 05:54:40 INFO mapred.FileInputFormat: Total input paths to process : 228 14/12/18 06:00:11 INFO mapred.FileInputFormat: Total input paths to process : 3076 14/12/18 06:02:02 INFO mapred.FileInputFormat: Total input paths to process : 1013 14/12/18 06:02:21 INFO mapred.FileInputFormat: Total input paths to process : 156 More generally, it would be important for the documentation to clearly point out what RDD transformations are eager, otherwise it is easy to introduce horrible performance bugs by constructing unneeded RDDs, assuming this is a lazy operation. I would venture to suggest introducing one or more traits to collect all the eager RDD-to-RDD transformations, so that the type system can be used to enforce that no eager transformation is used where a lazy one is intended to be used. Alex 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 >> >