Skip to content

SPARK-2895: Add mapPartitionsWithContext related support on Spark Java API. #2194

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 10 commits into from
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
37 changes: 35 additions & 2 deletions core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ import com.google.common.base.Optional
import org.apache.hadoop.io.compress.CompressionCodec

import org.apache.spark.{FutureAction, Partition, SparkContext, TaskContext}
import org.apache.spark.annotation.Experimental
import org.apache.spark.annotation.{DeveloperApi, Experimental}
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, _}
Expand Down Expand Up @@ -81,7 +81,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
def mapPartitionsWithIndex[R](
f: JFunction2[java.lang.Integer, java.util.Iterator[T], java.util.Iterator[R]],
preservesPartitioning: Boolean = false): JavaRDD[R] =
new JavaRDD(rdd.mapPartitionsWithIndex(((a,b) => f(a,asJavaIterator(b))),
new JavaRDD(rdd.mapPartitionsWithIndex(((a, b) => f(a, asJavaIterator(b))),
preservesPartitioning)(fakeClassTag))(fakeClassTag)

/**
Expand Down Expand Up @@ -185,6 +185,39 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
rdd.mapPartitions(fn, preservesPartitioning))(fakeClassTag[K2], fakeClassTag[V2])
}

/**
* :: DeveloperApi ::
* Return a new RDD by applying a function to each partition of this RDD. This is a variant of
* mapPartitions that also passes the TaskContext into the closure.
*
* `preservesPartitioning` indicates whether the input function preserves the partitioner, which
* should be `false` unless this is a pair RDD and the input function doesn't modify the keys.
*/
@DeveloperApi
def mapPartitionsWithContext[R](
f: JFunction2[TaskContext, java.util.Iterator[T], java.util.Iterator[R]],
preservesPartitioning: Boolean): JavaRDD[R] = {

new JavaRDD(rdd.mapPartitionsWithContext(
((a, b) => f(a, asJavaIterator(b))), preservesPartitioning)(fakeClassTag))(fakeClassTag)
}

/**
* :: DeveloperApi ::
* Return a new JavaPairRDD by applying a function to each partition of this RDD. This is a
* variant of mapPartitions that also passes the TaskContext into the closure.
*
* `preservesPartitioning` indicates whether the input function preserves the partitioner, which
* should be `false` unless this is a pair RDD and the input function doesn't modify the keys.
*/
@DeveloperApi
def mapPartitionsToPairWithContext[K2, V2](
f: JFunction2[TaskContext, java.util.Iterator[T], java.util.Iterator[(K2, V2)]],
preservesPartitioning: Boolean): JavaPairRDD[K2, V2] = {

JavaPairRDD.fromJavaRDD(mapPartitionsWithContext(f, preservesPartitioning))
}

/**
* Applies a function f to each partition of this RDD.
*/
Expand Down
80 changes: 80 additions & 0 deletions core/src/test/java/org/apache/spark/JavaAPISuite.java
Original file line number Diff line number Diff line change
Expand Up @@ -707,6 +707,86 @@ public Iterable<Integer> call(Iterator<Integer> iter) {
Assert.assertEquals("[3, 7]", partitionSums.collect().toString());
}

@Test
public void mapPartitionsWithContext() {
JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2);
JavaRDD<String> partitionSumsWithContext = rdd.mapPartitionsWithContext(
new Function2<TaskContext, Iterator<Integer>, Iterator<String>>() {
@Override
public Iterator<String> call(TaskContext context,
Iterator<Integer> iter) throws Exception {

int sum = 0;
while (iter.hasNext()) {
sum += iter.next();
}
return Collections.singletonList(sum + "-partition-" + context.partitionId()).iterator();
}
}, false);
Assert.assertEquals("[3-partition-0, 7-partition-1]",
partitionSumsWithContext.collect().toString());
}

@Test
public void mapPartitionsToPair() {
JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2);
JavaPairRDD<Integer, String> pairRdd = rdd.mapPartitionsToPair(
new PairFlatMapFunction<Iterator<Integer>, Integer, String>() {
@Override
public Iterable<Tuple2<Integer, String>> call(Iterator<Integer> iter) throws Exception {
int sum = 0;
while (iter.hasNext()) {
sum += iter.next();
}
return Collections.singletonList(new Tuple2<Integer, String>(sum, "a"));
}
}
);

Assert.assertEquals("[(3,a), (7,a)]", pairRdd.collect().toString());
}

@Test
public void mapPartitionsToPairWithContext() {
JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2);
JavaPairRDD<Integer, String> pairRdd = rdd.mapPartitionsToPairWithContext(
new Function2<TaskContext, Iterator<Integer>, Iterator<Tuple2<Integer, String>>>() {
@Override
public Iterator<Tuple2<Integer, String>> call(TaskContext context,
Iterator<Integer> iter) throws Exception {

int sum = 0;
while (iter.hasNext()) {
sum += iter.next();
}
return Collections.singletonList(
new Tuple2<Integer, String>(sum, "partition-" + context.partitionId())).iterator();
}
}, false
);

Assert.assertEquals("[(3,partition-0), (7,partition-1)]", pairRdd.collect().toString());
}

@Test
public void mapPartitionsToDouble() {
JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2);
JavaDoubleRDD pairRdd = rdd.mapPartitionsToDouble(
new DoubleFlatMapFunction<Iterator<Integer>>() {
@Override
public Iterable<Double> call(Iterator<Integer> iter) throws Exception {
int sum = 0;
while (iter.hasNext()) {
sum += iter.next();
}
return Collections.singletonList(Double.valueOf(sum));
}
}
);

Assert.assertEquals("[3.0, 7.0]", pairRdd.collect().toString());
}

@Test
public void repartition() {
// Shrinking number of partitions
Expand Down