[ https://issues.apache.org/jira/browse/SPARK-2331?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14048041#comment-14048041 ]
Sean Owen edited comment on SPARK-2331 at 6/30/14 7:48 PM: ----------------------------------------------------------- My 2 cents. {code} val rdds = Seq("a","b","c").foldLeft(sc.emptyRDD){ (rdd,path) => rdd.union(sc.textFile(path)) } {code} yields {code} <console>:12: error: type mismatch; found : org.apache.spark.rdd.RDD[String] required: org.apache.spark.rdd.RDD[Nothing] Note: String >: Nothing, but class RDD is invariant in type T. You may wish to define T as -T instead. (SLS 4.5) val rdds = Seq("a","b","c").foldLeft(sc.emptyRDD){ (rdd,path) => rdd.union(sc.textFile(path)) } {code} but even {code} val rdds = Seq("a","b","c").foldLeft(sc.emptyRDD[String]){ (rdd,path) => rdd.union(sc.textFile(path)) } {code} yields {code} <console>:12: error: type mismatch; found : org.apache.spark.rdd.RDD[String] required: org.apache.spark.rdd.EmptyRDD[String] val rdds = Seq("a","b","c").foldLeft(sc.emptyRDD[String]){ (rdd,path) => rdd.union(sc.textFile(path)) } {code} So I think this is really about RDDs being invariant, rather than the return type here, and that seems to be how it's going to be: https://issues.apache.org/jira/browse/SPARK-1296 I think there's an argument for hiding EmptyRDD although that would be a little API change at this point. was (Author: srowen): My 2 cents. You mean the type is EmptyRDD[Nothing] right? {code} val rdds = Seq("a","b","c").foldLeft(sc.emptyRDD){ (rdd,path) => rdd.union(sc.textFile(path)) } {code} yields {code} <console>:12: error: type mismatch; found : org.apache.spark.rdd.RDD[String] required: org.apache.spark.rdd.RDD[Nothing] Note: String >: Nothing, but class RDD is invariant in type T. You may wish to define T as -T instead. (SLS 4.5) val rdds = Seq("a","b","c").foldLeft(sc.emptyRDD){ (rdd,path) => rdd.union(sc.textFile(path)) } {code} but even {code} val rdds = Seq("a","b","c").foldLeft(sc.emptyRDD[String]){ (rdd,path) => rdd.union(sc.textFile(path)) } {code} yields {code} <console>:12: error: type mismatch; found : org.apache.spark.rdd.RDD[String] required: org.apache.spark.rdd.EmptyRDD[String] val rdds = Seq("a","b","c").foldLeft(sc.emptyRDD[String]){ (rdd,path) => rdd.union(sc.textFile(path)) } {code} So I think this is really about RDDs being invariant, rather than the return type here, and that seems to be how it's going to be: https://issues.apache.org/jira/browse/SPARK-1296 I think there's an argument for hiding EmptyRDD although that would be a little API change at this point. > SparkContext.emptyRDD has wrong return type > ------------------------------------------- > > Key: SPARK-2331 > URL: https://issues.apache.org/jira/browse/SPARK-2331 > Project: Spark > Issue Type: Bug > Affects Versions: 1.0.0 > Reporter: Ian Hummel > > The return type for SparkContext.emptyRDD is EmptyRDD[T]. > It should be RDD[T]. That means you have to add extra type annotations on > code like the below (which creates a union of RDDs over some subset of paths > in a folder) > val rdds = Seq("a", "b", "c").foldLeft[RDD[String]](sc.emptyRDD[String]) { > (rdd, path) ⇒ > rdd.union(sc.textFile(path)) > } -- This message was sent by Atlassian JIRA (v6.2#6252)