From 8f4aab21c438cdd2fe371d49286559e038c5c061 Mon Sep 17 00:00:00 2001 From: chengxiang li Date: Fri, 29 Aug 2014 15:12:09 +0800 Subject: [PATCH 1/8] add mappartitionWithContext related support on Spark Java API. --- .../java/function/DoubleFlatMapFunction2.java | 28 ++++++++++ .../java/function/PairFlatMapFunction2.java | 30 +++++++++++ .../apache/spark/api/java/JavaRDDLike.scala | 52 ++++++++++++++++++- 3 files changed, 109 insertions(+), 1 deletion(-) create mode 100644 core/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction2.java create mode 100644 core/src/main/java/org/apache/spark/api/java/function/PairFlatMapFunction2.java diff --git a/core/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction2.java b/core/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction2.java new file mode 100644 index 000000000000..0f9d088d26fb --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction2.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.java.function; + +import java.io.Serializable; + +/** + * A function that takes arguments of type T1 and T2, and returns zero or more records of type + * Double from each input record. + */ +public interface DoubleFlatMapFunction2 extends Serializable { + public Iterable call(T1 t1, T2 t2) throws Exception; +} diff --git a/core/src/main/java/org/apache/spark/api/java/function/PairFlatMapFunction2.java b/core/src/main/java/org/apache/spark/api/java/function/PairFlatMapFunction2.java new file mode 100644 index 000000000000..8b4603d61159 --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/java/function/PairFlatMapFunction2.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.java.function; + +import scala.Tuple2; + +import java.io.Serializable; + +/** + * A function that takes arguments of type T1 and T2, and returns zero or more key-value pair + * records from each input record. The key-value pairs are represented as scala.Tuple2 objects. + */ +public interface PairFlatMapFunction2 extends Serializable { + public Iterable> call(T1 t1, T2 t2) throws Exception; +} diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index f917cfd1419e..f906a540f9f8 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -27,7 +27,7 @@ import com.google.common.base.Optional import org.apache.hadoop.io.compress.CompressionCodec import org.apache.spark.{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, _} @@ -84,6 +84,21 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { new JavaRDD(rdd.mapPartitionsWithIndex(((a,b) => f(a,asJavaIterator(b))), preservesPartitioning)(fakeClassTag))(fakeClassTag) + /** + * :: 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 = false): JavaRDD[R] = + new JavaRDD(rdd.mapPartitionsWithContext(((a,b) => f(a,asJavaIterator(b))), + preservesPartitioning)(fakeClassTag))(fakeClassTag) + /** * Return a new RDD by applying a function to all elements of this RDD. */ @@ -185,6 +200,41 @@ 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 mapPartitionsToDoubleWithContext( + f: DoubleFlatMapFunction2[TaskContext, java.util.Iterator[T]], + preservesPartitioning: Boolean): JavaDoubleRDD = { + def fn = (context: TaskContext, x: Iterator[T]) => + asScalaIterator(f.call(context, asJavaIterator(x)).iterator()) + new JavaDoubleRDD( + rdd.mapPartitionsWithContext(fn, preservesPartitioning).map(x => x.doubleValue())) + } + + /** + * :: 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 mapPartitionsToPairWithContext[K2, V2](f: PairFlatMapFunction2[TaskContext, + java.util.Iterator[T], K2, V2], preservesPartitioning: Boolean): JavaPairRDD[K2, V2] = { + def fn = (context: TaskContext, x: Iterator[T]) => + asScalaIterator(f.call(context, asJavaIterator(x)).iterator()) + JavaPairRDD.fromRDD( + rdd.mapPartitionsWithContext(fn, preservesPartitioning))(fakeClassTag[K2], fakeClassTag[V2]) + } + /** * Applies a function f to each partition of this RDD. */ From 37b5b6b8d00c9cbcc3d9c8dbc47577af29c60196 Mon Sep 17 00:00:00 2001 From: chengxiang li Date: Fri, 29 Aug 2014 15:30:21 +0800 Subject: [PATCH 2/8] fix several code style issue. --- .../apache/spark/api/java/JavaRDDLike.scala | 36 +++++++++---------- 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index f906a540f9f8..1dfe7345abc7 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -81,24 +81,9 @@ 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) - /** - * :: 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 = false): JavaRDD[R] = - new JavaRDD(rdd.mapPartitionsWithContext(((a,b) => f(a,asJavaIterator(b))), - preservesPartitioning)(fakeClassTag))(fakeClassTag) - /** * Return a new RDD by applying a function to all elements of this RDD. */ @@ -209,6 +194,21 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * 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 = false): JavaRDD[R] = + new JavaRDD(rdd.mapPartitionsWithContext(((a, b) => f(a, asJavaIterator(b))), + preservesPartitioning)(fakeClassTag))(fakeClassTag) + + /** + * :: DeveloperApi :: + * Return a new JavaDoubleRDD 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 mapPartitionsToDoubleWithContext( f: DoubleFlatMapFunction2[TaskContext, java.util.Iterator[T]], preservesPartitioning: Boolean): JavaDoubleRDD = { @@ -220,8 +220,8 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * :: 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. + * 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. From ae03a4783a988fe9c6d76838c37656f170b119e9 Mon Sep 17 00:00:00 2001 From: chengxiang li Date: Mon, 1 Sep 2014 17:41:51 +0800 Subject: [PATCH 3/8] fix code style issue. --- .../apache/spark/api/java/JavaRDDLike.scala | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 1dfe7345abc7..3ea8669c4cc2 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -195,10 +195,13 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { */ @DeveloperApi def mapPartitionsWithContext[R]( - f: JFunction2[TaskContext, java.util.Iterator[T], java.util.Iterator[R]], - preservesPartitioning: Boolean = false): JavaRDD[R] = - new JavaRDD(rdd.mapPartitionsWithContext(((a, b) => f(a, asJavaIterator(b))), - preservesPartitioning)(fakeClassTag))(fakeClassTag) + f: JFunction2[TaskContext, java.util.Iterator[T], java.util.Iterator[R]], + preservesPartitioning: Boolean = false): JavaRDD[R] = { + + new JavaRDD(rdd.mapPartitionsWithContext( + ((a, b) => f(a, asJavaIterator(b))), preservesPartitioning)(fakeClassTag))(fakeClassTag) + } + /** * :: DeveloperApi :: @@ -212,6 +215,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def mapPartitionsToDoubleWithContext( f: DoubleFlatMapFunction2[TaskContext, java.util.Iterator[T]], preservesPartitioning: Boolean): JavaDoubleRDD = { + def fn = (context: TaskContext, x: Iterator[T]) => asScalaIterator(f.call(context, asJavaIterator(x)).iterator()) new JavaDoubleRDD( @@ -227,8 +231,10 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * should be `false` unless this is a pair RDD and the input function doesn't modify the keys. */ @DeveloperApi - def mapPartitionsToPairWithContext[K2, V2](f: PairFlatMapFunction2[TaskContext, - java.util.Iterator[T], K2, V2], preservesPartitioning: Boolean): JavaPairRDD[K2, V2] = { + def mapPartitionsToPairWithContext[K2, V2]( + f: PairFlatMapFunction2[TaskContext, java.util.Iterator[T], K2, V2], + preservesPartitioning: Boolean): JavaPairRDD[K2, V2] = { + def fn = (context: TaskContext, x: Iterator[T]) => asScalaIterator(f.call(context, asJavaIterator(x)).iterator()) JavaPairRDD.fromRDD( From 7ba48b0592718e23d5220fc709cde4e0bad00015 Mon Sep 17 00:00:00 2001 From: chengxiang li Date: Tue, 2 Sep 2014 17:24:38 +0800 Subject: [PATCH 4/8] add unit test --- .../java/org/apache/spark/JavaAPISuite.java | 88 +++++++++++++++++++ 1 file changed, 88 insertions(+) diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index e1c13de04a0b..bb4dc2f244f8 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -707,6 +707,94 @@ public Iterable call(Iterator iter) { Assert.assertEquals("[3, 7]", partitionSums.collect().toString()); } + @Test + public void mapPartitionsWithContext() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2); + JavaRDD partitionSumsWithContext = rdd.mapPartitionsWithContext( + new Function2, Iterator>() { + @Override + public Iterator call(TaskContext context, Iterator 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 rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2); + JavaPairRDD pairRdd = rdd.mapPartitionsToPair( + new PairFlatMapFunction, Integer, String>() { + @Override + public Iterable> call(Iterator iter) throws Exception { + int sum = 0; + while (iter.hasNext()) { + sum += iter.next(); + } + return Collections.singletonList(new Tuple2(sum, "a")); + } + } + ); + Assert.assertEquals("[(3,a), (7,a)]", pairRdd.collect().toString()); + } + + @Test + public void mapPartitionsToPairWithContext() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2); + JavaPairRDD pairRdd = rdd.mapPartitionsToPairWithContext( + new PairFlatMapFunction2, Integer, String>() { + @Override + public Iterable> call(TaskContext context, Iterator iter) throws Exception { + int sum = 0; + while (iter.hasNext()) { + sum += iter.next(); + } + return Collections.singletonList(new Tuple2(sum, "partition-" + context.partitionId())); + } + }, false); + Assert.assertEquals("[(3,partition-0), (7,partition-1)]", pairRdd.collect().toString()); + } + + @Test + public void mapPartitionsToDouble() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2); + JavaDoubleRDD pairRdd = rdd.mapPartitionsToDouble( + new DoubleFlatMapFunction>() { + @Override + public Iterable call(Iterator 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 mapPartitionsToDoubleWithContext() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2); + JavaDoubleRDD pairRdd = rdd.mapPartitionsToDoubleWithContext( + new DoubleFlatMapFunction2>() { + @Override + public Iterable call(TaskContext context, Iterator iter) throws Exception { + int sum = 0; + while (iter.hasNext()) { + sum += iter.next(); + } + sum += context.partitionId(); + return Collections.singletonList(Double.valueOf(sum)); + } + }, false); + Assert.assertEquals("[3.0, 8.0]", pairRdd.collect().toString()); + } + @Test public void repartition() { // Shrinking number of partitions From aa6908ef76d5c8de9445eb0143b17b1d1af42259 Mon Sep 17 00:00:00 2001 From: chengxiang li Date: Tue, 2 Sep 2014 17:40:44 +0800 Subject: [PATCH 5/8] fix code style. --- .../java/org/apache/spark/JavaAPISuite.java | 50 +++++++++++-------- 1 file changed, 30 insertions(+), 20 deletions(-) diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index bb4dc2f244f8..0c4eab314211 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -713,7 +713,9 @@ public void mapPartitionsWithContext() { JavaRDD partitionSumsWithContext = rdd.mapPartitionsWithContext( new Function2, Iterator>() { @Override - public Iterator call(TaskContext context, Iterator iter) throws Exception { + public Iterator call(TaskContext context, + Iterator iter) throws Exception { + int sum = 0; while (iter.hasNext()) { sum += iter.next(); @@ -721,7 +723,8 @@ public Iterator call(TaskContext context, Iterator iter) throws return Collections.singletonList(sum + "-partition-" + context.partitionId()).iterator(); } }, false); - Assert.assertEquals("[3-partition-0, 7-partition-1]", partitionSumsWithContext.collect().toString()); + Assert.assertEquals("[3-partition-0, 7-partition-1]", + partitionSumsWithContext.collect().toString()); } @Test @@ -729,16 +732,17 @@ public void mapPartitionsToPair() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2); JavaPairRDD pairRdd = rdd.mapPartitionsToPair( new PairFlatMapFunction, Integer, String>() { - @Override - public Iterable> call(Iterator iter) throws Exception { - int sum = 0; - while (iter.hasNext()) { - sum += iter.next(); - } - return Collections.singletonList(new Tuple2(sum, "a")); + @Override + public Iterable> call(Iterator iter) throws Exception { + int sum = 0; + while (iter.hasNext()) { + sum += iter.next(); } + return Collections.singletonList(new Tuple2(sum, "a")); + } } ); + Assert.assertEquals("[(3,a), (7,a)]", pairRdd.collect().toString()); } @@ -748,14 +752,18 @@ public void mapPartitionsToPairWithContext() { JavaPairRDD pairRdd = rdd.mapPartitionsToPairWithContext( new PairFlatMapFunction2, Integer, String>() { @Override - public Iterable> call(TaskContext context, Iterator iter) throws Exception { + public Iterable> call(TaskContext context, Iterator iter) + throws Exception { + int sum = 0; while (iter.hasNext()) { sum += iter.next(); } - return Collections.singletonList(new Tuple2(sum, "partition-" + context.partitionId())); + return Collections.singletonList( + new Tuple2(sum, "partition-" + context.partitionId())); } }, false); + Assert.assertEquals("[(3,partition-0), (7,partition-1)]", pairRdd.collect().toString()); } @@ -774,6 +782,7 @@ public Iterable call(Iterator iter) throws Exception { } } ); + Assert.assertEquals("[3.0, 7.0]", pairRdd.collect().toString()); } @@ -782,16 +791,17 @@ public void mapPartitionsToDoubleWithContext() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2); JavaDoubleRDD pairRdd = rdd.mapPartitionsToDoubleWithContext( new DoubleFlatMapFunction2>() { - @Override - public Iterable call(TaskContext context, Iterator iter) throws Exception { - int sum = 0; - while (iter.hasNext()) { - sum += iter.next(); - } - sum += context.partitionId(); - return Collections.singletonList(Double.valueOf(sum)); + @Override + public Iterable call(TaskContext context, Iterator iter) throws Exception { + int sum = 0; + while (iter.hasNext()) { + sum += iter.next(); } - }, false); + sum += context.partitionId(); + return Collections.singletonList(Double.valueOf(sum)); + } + }, false); + Assert.assertEquals("[3.0, 8.0]", pairRdd.collect().toString()); } From 50b200c14db91df8f5f2f99eff1672a95339c9df Mon Sep 17 00:00:00 2001 From: chengxiang li Date: Wed, 3 Sep 2014 09:51:49 +0800 Subject: [PATCH 6/8] remove mapPartitionsToDoubleWithContext. --- .../java/function/DoubleFlatMapFunction2.java | 28 ------------------- .../apache/spark/api/java/JavaRDDLike.scala | 20 ------------- .../java/org/apache/spark/JavaAPISuite.java | 19 ------------- 3 files changed, 67 deletions(-) delete mode 100644 core/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction2.java diff --git a/core/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction2.java b/core/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction2.java deleted file mode 100644 index 0f9d088d26fb..000000000000 --- a/core/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction2.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.api.java.function; - -import java.io.Serializable; - -/** - * A function that takes arguments of type T1 and T2, and returns zero or more records of type - * Double from each input record. - */ -public interface DoubleFlatMapFunction2 extends Serializable { - public Iterable call(T1 t1, T2 t2) throws Exception; -} diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 353e9cae4ba5..27fcb45c2f29 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -202,26 +202,6 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { ((a, b) => f(a, asJavaIterator(b))), preservesPartitioning)(fakeClassTag))(fakeClassTag) } - - /** - * :: DeveloperApi :: - * Return a new JavaDoubleRDD 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 mapPartitionsToDoubleWithContext( - f: DoubleFlatMapFunction2[TaskContext, java.util.Iterator[T]], - preservesPartitioning: Boolean): JavaDoubleRDD = { - - def fn = (context: TaskContext, x: Iterator[T]) => - asScalaIterator(f.call(context, asJavaIterator(x)).iterator()) - new JavaDoubleRDD( - rdd.mapPartitionsWithContext(fn, preservesPartitioning).map(x => x.doubleValue())) - } - /** * :: DeveloperApi :: * Return a new JavaPairRDD by applying a function to each partition of this RDD. This is a diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 0c4eab314211..d379bf93155e 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -786,25 +786,6 @@ public Iterable call(Iterator iter) throws Exception { Assert.assertEquals("[3.0, 7.0]", pairRdd.collect().toString()); } - @Test - public void mapPartitionsToDoubleWithContext() { - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2); - JavaDoubleRDD pairRdd = rdd.mapPartitionsToDoubleWithContext( - new DoubleFlatMapFunction2>() { - @Override - public Iterable call(TaskContext context, Iterator iter) throws Exception { - int sum = 0; - while (iter.hasNext()) { - sum += iter.next(); - } - sum += context.partitionId(); - return Collections.singletonList(Double.valueOf(sum)); - } - }, false); - - Assert.assertEquals("[3.0, 8.0]", pairRdd.collect().toString()); - } - @Test public void repartition() { // Shrinking number of partitions From 882b82e51a72c9e853d2795e0ee8d46082dc02dd Mon Sep 17 00:00:00 2001 From: chengxiang li Date: Wed, 3 Sep 2014 11:11:52 +0800 Subject: [PATCH 7/8] implement mapPartitionsToPairWithContext by wrap mapPartitionsWithContext. --- .../java/function/PairFlatMapFunction2.java | 30 ------------------- .../apache/spark/api/java/JavaRDDLike.scala | 7 ++--- .../java/org/apache/spark/JavaAPISuite.java | 11 +++---- 3 files changed, 8 insertions(+), 40 deletions(-) delete mode 100644 core/src/main/java/org/apache/spark/api/java/function/PairFlatMapFunction2.java diff --git a/core/src/main/java/org/apache/spark/api/java/function/PairFlatMapFunction2.java b/core/src/main/java/org/apache/spark/api/java/function/PairFlatMapFunction2.java deleted file mode 100644 index 8b4603d61159..000000000000 --- a/core/src/main/java/org/apache/spark/api/java/function/PairFlatMapFunction2.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.api.java.function; - -import scala.Tuple2; - -import java.io.Serializable; - -/** - * A function that takes arguments of type T1 and T2, and returns zero or more key-value pair - * records from each input record. The key-value pairs are represented as scala.Tuple2 objects. - */ -public interface PairFlatMapFunction2 extends Serializable { - public Iterable> call(T1 t1, T2 t2) throws Exception; -} diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 27fcb45c2f29..d811c4d15a3d 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -212,13 +212,10 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { */ @DeveloperApi def mapPartitionsToPairWithContext[K2, V2]( - f: PairFlatMapFunction2[TaskContext, java.util.Iterator[T], K2, V2], + f: JFunction2[TaskContext, java.util.Iterator[T], java.util.Iterator[(K2, V2)]], preservesPartitioning: Boolean): JavaPairRDD[K2, V2] = { - def fn = (context: TaskContext, x: Iterator[T]) => - asScalaIterator(f.call(context, asJavaIterator(x)).iterator()) - JavaPairRDD.fromRDD( - rdd.mapPartitionsWithContext(fn, preservesPartitioning))(fakeClassTag[K2], fakeClassTag[V2]) + JavaPairRDD.fromJavaRDD(mapPartitionsWithContext(f, preservesPartitioning)) } /** diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index d379bf93155e..4e9f8eeeefdf 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -750,19 +750,20 @@ public Iterable> call(Iterator iter) throws Exc public void mapPartitionsToPairWithContext() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2); JavaPairRDD pairRdd = rdd.mapPartitionsToPairWithContext( - new PairFlatMapFunction2, Integer, String>() { + new Function2, Iterator>>() { @Override - public Iterable> call(TaskContext context, Iterator iter) - throws Exception { + public Iterator> call(TaskContext context, + Iterator iter) throws Exception { int sum = 0; while (iter.hasNext()) { sum += iter.next(); } return Collections.singletonList( - new Tuple2(sum, "partition-" + context.partitionId())); + new Tuple2(sum, "partition-" + context.partitionId())).iterator(); } - }, false); + }, false + ); Assert.assertEquals("[(3,partition-0), (7,partition-1)]", pairRdd.collect().toString()); } From a5b7b412dcff6d680e38a8f1f79fea065762e1c3 Mon Sep 17 00:00:00 2001 From: chengxiang li Date: Fri, 12 Sep 2014 10:50:07 +0800 Subject: [PATCH 8/8] remove default parameter value. --- core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index d811c4d15a3d..0ec7f7bcc477 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -196,7 +196,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { @DeveloperApi def mapPartitionsWithContext[R]( f: JFunction2[TaskContext, java.util.Iterator[T], java.util.Iterator[R]], - preservesPartitioning: Boolean = false): JavaRDD[R] = { + preservesPartitioning: Boolean): JavaRDD[R] = { new JavaRDD(rdd.mapPartitionsWithContext( ((a, b) => f(a, asJavaIterator(b))), preservesPartitioning)(fakeClassTag))(fakeClassTag)