Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package org.apache.spark.api.java

import java.{lang => jl}
import java.lang.{Iterable => JIterable}
import java.util.{Comparator, List => JList}
import java.util.{Comparator, Iterator => JIterator, List => JList}

import scala.collection.JavaConverters._
import scala.language.implicitConversions
Expand All @@ -34,7 +34,8 @@ import org.apache.spark.{HashPartitioner, Partitioner}
import org.apache.spark.Partitioner._
import org.apache.spark.api.java.JavaSparkContext.fakeClassTag
import org.apache.spark.api.java.JavaUtils.mapAsSerializableJavaMap
import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, PairFunction}
import org.apache.spark.api.java.function.{FlatMapFunction, Function => JFunction,
Function2 => JFunction2, PairFunction}
import org.apache.spark.partial.{BoundedDouble, PartialResult}
import org.apache.spark.rdd.{OrderedRDDFunctions, RDD}
import org.apache.spark.rdd.RDD.rddToPairRDDFunctions
Expand Down Expand Up @@ -674,8 +675,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])
* Pass each value in the key-value pair RDD through a flatMap function without changing the
* keys; this also retains the original RDD's partitioning.
*/
def flatMapValues[U](f: JFunction[V, java.lang.Iterable[U]]): JavaPairRDD[K, U] = {
def fn: (V) => Iterable[U] = (x: V) => f.call(x).asScala
def flatMapValues[U](f: FlatMapFunction[V, U]): JavaPairRDD[K, U] = {
def fn: (V) => Iterator[U] = (x: V) => f.call(x).asScala
implicit val ctag: ClassTag[U] = fakeClassTag
fromRDD(rdd.flatMapValues(fn))
}
Expand Down
4 changes: 3 additions & 1 deletion project/MimaExcludes.scala
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,9 @@ object MimaExcludes {

// Exclude rules for 3.0.x
lazy val v30excludes = v24excludes ++ Seq(
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.io.SnappyCompressionCodec.version")
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.io.SnappyCompressionCodec.version"),
ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.api.java.JavaPairRDD.flatMapValues"),
ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.streaming.api.java.JavaPairDStream.flatMapValues")
)

// Exclude rules for 2.4.x
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,8 @@ import org.apache.spark.annotation.Experimental
import org.apache.spark.api.java.{JavaPairRDD, JavaSparkContext, JavaUtils, Optional}
import org.apache.spark.api.java.JavaPairRDD._
import org.apache.spark.api.java.JavaSparkContext.fakeClassTag
import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2}
import org.apache.spark.api.java.function.{FlatMapFunction, Function => JFunction,
Function2 => JFunction2}
import org.apache.spark.rdd.RDD
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming._
Expand Down Expand Up @@ -562,9 +563,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
* Return a new DStream by applying a flatmap function to the value of each key-value pairs in
* 'this' DStream without changing the key.
*/
def flatMapValues[U](f: JFunction[V, java.lang.Iterable[U]]): JavaPairDStream[K, U] = {
import scala.collection.JavaConverters._
def fn: (V) => Iterable[U] = (x: V) => f.apply(x).asScala
def flatMapValues[U](f: FlatMapFunction[V, U]): JavaPairDStream[K, U] = {
def fn: (V) => Iterator[U] = (x: V) => f.call(x).asScala
implicit val cm: ClassTag[U] =
implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]]
dstream.flatMapValues(fn)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -841,7 +841,7 @@ public void testFlatMapValues() {
JavaPairDStream<String, String> pairStream = JavaPairDStream.fromJavaDStream(stream);

JavaPairDStream<String, String> flatMapped =
pairStream.flatMapValues(in -> Arrays.asList(in + "1", in + "2"));
pairStream.flatMapValues(in -> Arrays.asList(in + "1", in + "2").iterator());
JavaTestUtils.attachTestOutputStream(flatMapped);
List<List<Tuple2<String, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
Assert.assertEquals(expected, result);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1355,7 +1355,7 @@ public void testFlatMapValues() {
List<String> out = new ArrayList<>();
out.add(in + "1");
out.add(in + "2");
return out;
return out.iterator();
});

JavaTestUtils.attachTestOutputStream(flatMapped);
Expand Down