You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Alok Saldanha (JIRA)" <ji...@apache.org> on 2014/11/17 22:20:34 UTC

[jira] [Commented] (SPARK-4459) JavaRDDLike.groupBy[K](f: JFunction[T, K]) may fail with typechecking errors

    [ https://issues.apache.org/jira/browse/SPARK-4459?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14215202#comment-14215202 ] 

Alok Saldanha commented on SPARK-4459:
--------------------------------------

I created a standalone gist to demonstrate the problem, please see https://gist.github.com/alokito/40878fc25af21984463f

> JavaRDDLike.groupBy[K](f: JFunction[T, K]) may fail with typechecking errors
> ----------------------------------------------------------------------------
>
>                 Key: SPARK-4459
>                 URL: https://issues.apache.org/jira/browse/SPARK-4459
>             Project: Spark
>          Issue Type: Bug
>          Components: Java API
>    Affects Versions: 1.0.2, 1.1.0
>            Reporter: Alok Saldanha
>
> I believe this issue is essentially the same as SPARK-668.
> Original error: 
> {code}
> [ERROR] /Users/saldaal1/workspace/JavaSparkSimpleApp/src/main/java/SimpleApp.java:[29,105] no suitable method found for groupBy(org.apache.spark.api.java.function.Function<scala.Tuple2<java.lang.String,java.lang.Long>,java.lang.Long>)
> [ERROR] method org.apache.spark.api.java.JavaPairRDD.<K>groupBy(org.apache.spark.api.java.function.Function<scala.Tuple2<K,java.lang.Long>,K>) is not applicable
> [ERROR] (inferred type does not conform to equality constraint(s)
> {code}
> from core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala 
> {code}
> 211  /**
> 212    * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements
> 213    * mapping to that key.
> 214    */
> 215   def groupBy[K](f: JFunction[T, K]): JavaPairRDD[K, JIterable[T]] = {
> 216     implicit val ctagK: ClassTag[K] = fakeClassTag
> 217     implicit val ctagV: ClassTag[JList[T]] = fakeClassTag
> 218     JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f)(fakeClassTag)))
> 219   }
> {code}
> Then in core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala:
> {code}
>   45 class JavaPairRDD[K, V](val rdd: RDD[(K, V)])
>   46                        (implicit val kClassTag: ClassTag[K], implicit val vClassTag: ClassTag[V])
>   47   extends JavaRDDLike[(K, V), JavaPairRDD[K, V]] {
> {code}
> The problem is that the type parameter T in JavaRDDLike is Tuple2[K,V], which means the combined signature for groupBy in the JavaPairRDD is 
> {code}
> groupBy[K](f: JFunction[Tuple2[K,V], K])
> {code}
> which imposes an unfortunate correlation between the Tuple2 and the return type of the grouping function, namely that the return type of the grouping function must be the same as the first type of the JavaPairRDD.
> If we compare the method signature to flatMap:
> {code}
> 105   /**
> 106    *  Return a new RDD by first applying a function to all elements of this
> 107    *  RDD, and then flattening the results.
> 108    */
> 109   def flatMap[U](f: FlatMapFunction[T, U]): JavaRDD[U] = {
> 110     import scala.collection.JavaConverters._
> 111     def fn = (x: T) => f.call(x).asScala
> 112     JavaRDD.fromRDD(rdd.flatMap(fn)(fakeClassTag[U]))(fakeClassTag[U])
> 113   }
> {code}
> we see there should be an easy fix by changing the type parameter of the groupBy function from K to U.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org