From 687ffbc82fe4c53b41fcb6bf186a9660edd4b0af Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 25 Mar 2014 15:32:55 -0700 Subject: [PATCH 01/39] This is the it compiles point of replacing Seq with Iterator and JList with JIterator in the groupby and cogroup signatures --- .../scala/org/apache/spark/bagel/Bagel.scala | 12 +++--- .../apache/spark/api/java/JavaPairRDD.scala | 36 ++++++++-------- .../apache/spark/api/java/JavaRDDLike.scala | 4 +- .../apache/spark/rdd/PairRDDFunctions.scala | 38 ++++++++--------- .../main/scala/org/apache/spark/rdd/RDD.scala | 6 +-- .../apache/spark/examples/JavaPageRank.java | 21 ++++++---- .../bagel/WikipediaPageRankStandalone.scala | 10 +++-- .../spark/mllib/recommendation/ALS.scala | 4 +- .../org/apache/spark/mllib/util/LAUtils.scala | 5 ++- .../streaming/api/java/JavaPairDStream.scala | 42 +++++++++---------- .../dstream/PairDStreamFunctions.scala | 28 ++++++------- .../streaming/dstream/StateDStream.scala | 12 ++++-- 12 files changed, 116 insertions(+), 102 deletions(-) diff --git a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala index 70c7474a936dc..8edf6a1356ad6 100644 --- a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala +++ b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala @@ -220,7 +220,7 @@ object Bagel extends Logging { */ private def comp[K: Manifest, V <: Vertex, M <: Message[K], C]( sc: SparkContext, - grouped: RDD[(K, (Seq[C], Seq[V]))], + grouped: RDD[(K, (Iterator[C], Iterator[V]))], compute: (V, Option[C]) => (V, Array[M]), storageLevel: StorageLevel ): (RDD[(K, (V, Array[M]))], Int, Int) = { @@ -230,10 +230,12 @@ object Bagel extends Logging { case (_, vs) if vs.size == 0 => None case (c, vs) => val (newVert, newMsgs) = - compute(vs(0), c match { - case Seq(comb) => Some(comb) - case Seq() => None - }) + compute(vs.next, + c.size match { + case 1 => Some(c.next) + case _ => None + } + ) numMsgs += newMsgs.size if (newVert.active) { diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 9596dbaf75488..acea9780d25fd 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.api.java -import java.util.{Comparator, List => JList} +import java.util.{Comparator, List => JList, Iterator => JIterator} import scala.collection.JavaConversions._ import scala.reflect.ClassTag @@ -250,14 +250,14 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * Group the values for each key in the RDD into a single sequence. Allows controlling the * partitioning of the resulting key-value pair RDD by passing a Partitioner. */ - def groupByKey(partitioner: Partitioner): JavaPairRDD[K, JList[V]] = + def groupByKey(partitioner: Partitioner): JavaPairRDD[K, JIterator[V]] = fromRDD(groupByResultToJava(rdd.groupByKey(partitioner))) /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with into `numPartitions` partitions. */ - def groupByKey(numPartitions: Int): JavaPairRDD[K, JList[V]] = + def groupByKey(numPartitions: Int): JavaPairRDD[K, JIterator[V]] = fromRDD(groupByResultToJava(rdd.groupByKey(numPartitions))) /** @@ -367,7 +367,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with the existing partitioner/parallelism level. */ - def groupByKey(): JavaPairRDD[K, JList[V]] = + def groupByKey(): JavaPairRDD[K, JIterator[V]] = fromRDD(groupByResultToJava(rdd.groupByKey())) /** @@ -462,7 +462,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * list of values for that key in `this` as well as `other`. */ def cogroup[W](other: JavaPairRDD[K, W], partitioner: Partitioner) - : JavaPairRDD[K, (JList[V], JList[W])] = + : JavaPairRDD[K, (JIterator[V], JIterator[W])] = fromRDD(cogroupResultToJava(rdd.cogroup(other, partitioner))) /** @@ -470,14 +470,14 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], - partitioner: Partitioner): JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + partitioner: Partitioner): JavaPairRDD[K, (JIterator[V], JIterator[W1], JIterator[W2])] = fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, partitioner))) /** * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] = + def cogroup[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JIterator[V], JIterator[W])] = fromRDD(cogroupResultToJava(rdd.cogroup(other))) /** @@ -485,7 +485,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2]) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + : JavaPairRDD[K, (JIterator[V], JIterator[W1], JIterator[W2])] = fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2))) /** @@ -493,7 +493,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * list of values for that key in `this` as well as `other`. */ def cogroup[W](other: JavaPairRDD[K, W], numPartitions: Int) - : JavaPairRDD[K, (JList[V], JList[W])] = + : JavaPairRDD[K, (JIterator[V], JIterator[W])] = fromRDD(cogroupResultToJava(rdd.cogroup(other, numPartitions))) /** @@ -501,16 +501,16 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], numPartitions: Int) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + : JavaPairRDD[K, (JIterator[V], JIterator[W1], JIterator[W2])] = fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, numPartitions))) /** Alias for cogroup. */ - def groupWith[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] = + def groupWith[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JIterator[V], JIterator[W])] = fromRDD(cogroupResultToJava(rdd.groupWith(other))) /** Alias for cogroup. */ def groupWith[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2]) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + : JavaPairRDD[K, (JIterator[V], JIterator[W1], JIterator[W2])] = fromRDD(cogroupResult2ToJava(rdd.groupWith(other1, other2))) /** @@ -695,21 +695,21 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) object JavaPairRDD { private[spark] - def groupByResultToJava[K: ClassTag, T](rdd: RDD[(K, Seq[T])]): RDD[(K, JList[T])] = { - rddToPairRDDFunctions(rdd).mapValues(seqAsJavaList) + def groupByResultToJava[K: ClassTag, T](rdd: RDD[(K, Iterator[T])]): RDD[(K, JIterator[T])] = { + rddToPairRDDFunctions(rdd).mapValues(asJavaIterator) } private[spark] def cogroupResultToJava[K: ClassTag, V, W]( - rdd: RDD[(K, (Seq[V], Seq[W]))]): RDD[(K, (JList[V], JList[W]))] = { - rddToPairRDDFunctions(rdd).mapValues(x => (seqAsJavaList(x._1), seqAsJavaList(x._2))) + rdd: RDD[(K, (Iterator[V], Iterator[W]))]): RDD[(K, (JIterator[V], JIterator[W]))] = { + rddToPairRDDFunctions(rdd).mapValues(x => (asJavaIterator(x._1), asJavaIterator(x._2))) } private[spark] def cogroupResult2ToJava[K: ClassTag, V, W1, W2]( - rdd: RDD[(K, (Seq[V], Seq[W1], Seq[W2]))]): RDD[(K, (JList[V], JList[W1], JList[W2]))] = { + rdd: RDD[(K, (Iterator[V], Iterator[W1], Iterator[W2]))]): RDD[(K, (JIterator[V], JIterator[W1], JIterator[W2]))] = { rddToPairRDDFunctions(rdd) - .mapValues(x => (seqAsJavaList(x._1), seqAsJavaList(x._2), seqAsJavaList(x._3))) + .mapValues(x => (asJavaIterator(x._1), asJavaIterator(x._2), asJavaIterator(x._3))) } def fromRDD[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]): JavaPairRDD[K, V] = { 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 6e8ec8e0c7629..fd74e8218f925 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 @@ -204,7 +204,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ - def groupBy[K](f: JFunction[T, K]): JavaPairRDD[K, JList[T]] = { + def groupBy[K](f: JFunction[T, K]): JavaPairRDD[K, JIterator[T]] = { implicit val ctagK: ClassTag[K] = fakeClassTag implicit val ctagV: ClassTag[JList[T]] = fakeClassTag JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f)(fakeClassTag))) @@ -214,7 +214,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ - def groupBy[K](f: JFunction[T, K], numPartitions: Int): JavaPairRDD[K, JList[T]] = { + def groupBy[K](f: JFunction[T, K], numPartitions: Int): JavaPairRDD[K, JIterator[T]] = { implicit val ctagK: ClassTag[K] = fakeClassTag implicit val ctagV: ClassTag[JList[T]] = fakeClassTag JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f, numPartitions)(fakeClassTag[K]))) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 14386ff5b9127..3e005a7e1dc50 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -261,7 +261,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * Group the values for each key in the RDD into a single sequence. Allows controlling the * partitioning of the resulting key-value pair RDD by passing a Partitioner. */ - def groupByKey(partitioner: Partitioner): RDD[(K, Seq[V])] = { + def groupByKey(partitioner: Partitioner): RDD[(K, Iterator[V])] = { // groupByKey shouldn't use map side combine because map side combine does not // reduce the amount of data shuffled and requires all map side data be inserted // into a hash table, leading to more objects in the old gen. @@ -270,14 +270,14 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) def mergeCombiners(c1: ArrayBuffer[V], c2: ArrayBuffer[V]) = c1 ++ c2 val bufs = combineByKey[ArrayBuffer[V]]( createCombiner _, mergeValue _, mergeCombiners _, partitioner, mapSideCombine=false) - bufs.asInstanceOf[RDD[(K, Seq[V])]] + bufs.asInstanceOf[RDD[(K, Iterator[V])]] } /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with into `numPartitions` partitions. */ - def groupByKey(numPartitions: Int): RDD[(K, Seq[V])] = { + def groupByKey(numPartitions: Int): RDD[(K, Iterator[V])] = { groupByKey(new HashPartitioner(numPartitions)) } @@ -298,7 +298,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) */ def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => - for (v <- vs.iterator; w <- ws.iterator) yield (v, w) + for (v <- vs; w <- ws) yield (v, w) } } @@ -311,9 +311,9 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) def leftOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, Option[W]))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => if (ws.isEmpty) { - vs.iterator.map(v => (v, None)) + vs.map(v => (v, None)) } else { - for (v <- vs.iterator; w <- ws.iterator) yield (v, Some(w)) + for (v <- vs; w <- ws) yield (v, Some(w)) } } } @@ -328,9 +328,9 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) : RDD[(K, (Option[V], W))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => if (vs.isEmpty) { - ws.iterator.map(w => (None, w)) + ws.map(w => (None, w)) } else { - for (v <- vs.iterator; w <- ws.iterator) yield (Some(v), w) + for (v <- vs; w <- ws) yield (Some(v), w) } } } @@ -358,7 +358,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with the existing partitioner/parallelism level. */ - def groupByKey(): RDD[(K, Seq[V])] = { + def groupByKey(): RDD[(K, Iterator[V])] = { groupByKey(defaultPartitioner(self)) } @@ -453,13 +453,13 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (Seq[V], Seq[W]))] = { + def cogroup[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (Iterator[V], Iterator[W]))] = { if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { throw new SparkException("Default partitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K](Seq(self, other), partitioner) cg.mapValues { case Seq(vs, ws) => - (vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]]) + (vs.asInstanceOf[Seq[V]].iterator, ws.asInstanceOf[Seq[W]].iterator) } } @@ -468,13 +468,13 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], partitioner: Partitioner) - : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { + : RDD[(K, (Iterator[V], Iterator[W1], Iterator[W2]))] = { if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { throw new SparkException("Default partitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K](Seq(self, other1, other2), partitioner) cg.mapValues { case Seq(vs, w1s, w2s) => - (vs.asInstanceOf[Seq[V]], w1s.asInstanceOf[Seq[W1]], w2s.asInstanceOf[Seq[W2]]) + (vs.asInstanceOf[Seq[V]].iterator, w1s.asInstanceOf[Seq[W1]].iterator, w2s.asInstanceOf[Seq[W2]].iterator) } } @@ -482,7 +482,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { + def cogroup[W](other: RDD[(K, W)]): RDD[(K, (Iterator[V], Iterator[W]))] = { cogroup(other, defaultPartitioner(self, other)) } @@ -491,7 +491,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) - : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { + : RDD[(K, (Iterator[V], Iterator[W1], Iterator[W2]))] = { cogroup(other1, other2, defaultPartitioner(self, other1, other2)) } @@ -499,7 +499,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Seq[V], Seq[W]))] = { + def cogroup[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Iterator[V], Iterator[W]))] = { cogroup(other, new HashPartitioner(numPartitions)) } @@ -508,18 +508,18 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numPartitions: Int) - : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { + : RDD[(K, (Iterator[V], Iterator[W1], Iterator[W2]))] = { cogroup(other1, other2, new HashPartitioner(numPartitions)) } /** Alias for cogroup. */ - def groupWith[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { + def groupWith[W](other: RDD[(K, W)]): RDD[(K, (Iterator[V], Iterator[W]))] = { cogroup(other, defaultPartitioner(self, other)) } /** Alias for cogroup. */ def groupWith[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) - : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { + : RDD[(K, (Iterator[V], Iterator[W1], Iterator[W2]))] = { cogroup(other1, other2, defaultPartitioner(self, other1, other2)) } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index bf3c57ad41eb2..a702ef4348268 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -438,20 +438,20 @@ abstract class RDD[T: ClassTag]( /** * Return an RDD of grouped items. */ - def groupBy[K: ClassTag](f: T => K): RDD[(K, Seq[T])] = + def groupBy[K: ClassTag](f: T => K): RDD[(K, Iterator[T])] = groupBy[K](f, defaultPartitioner(this)) /** * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ - def groupBy[K: ClassTag](f: T => K, numPartitions: Int): RDD[(K, Seq[T])] = + def groupBy[K: ClassTag](f: T => K, numPartitions: Int): RDD[(K, Iterator[T])] = groupBy(f, new HashPartitioner(numPartitions)) /** * Return an RDD of grouped items. */ - def groupBy[K: ClassTag](f: T => K, p: Partitioner): RDD[(K, Seq[T])] = { + def groupBy[K: ClassTag](f: T => K, p: Partitioner): RDD[(K, Iterator[T])] = { val cleanF = sc.clean(f) this.map(t => (cleanF(t), t)).groupByKey(p) } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java index eb70fb547564c..7352160a55fec 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -26,8 +26,9 @@ import org.apache.spark.api.java.function.PairFlatMapFunction; import org.apache.spark.api.java.function.PairFunction; -import java.util.List; import java.util.ArrayList; +import java.util.List; +import java.util.Iterator; import java.util.regex.Pattern; /** @@ -66,7 +67,7 @@ public static void main(String[] args) throws Exception { JavaRDD lines = ctx.textFile(args[1], 1); // Loads all URLs from input file and initialize their neighbors. - JavaPairRDD> links = lines.mapToPair(new PairFunction() { + JavaPairRDD> links = lines.mapToPair(new PairFunction() { @Override public Tuple2 call(String s) { String[] parts = SPACES.split(s); @@ -75,9 +76,9 @@ public Tuple2 call(String s) { }).distinct().groupByKey().cache(); // Loads all URLs with other URL(s) link to from input file and initialize ranks of them to one. - JavaPairRDD ranks = links.mapValues(new Function, Double>() { + JavaPairRDD ranks = links.mapValues(new Function, Double>() { @Override - public Double call(List rs) { + public Double call(Iterator rs) { return 1.0; } }); @@ -86,12 +87,16 @@ public Double call(List rs) { for (int current = 0; current < Integer.parseInt(args[2]); current++) { // Calculates URL contributions to the rank of other URLs. JavaPairRDD contribs = links.join(ranks).values() - .flatMapToPair(new PairFlatMapFunction, Double>, String, Double>() { + .flatMapToPair(new PairFlatMapFunction, Double>, String, Double>() { @Override - public Iterable> call(Tuple2, Double> s) { + public Iterable> call(Tuple2, Double> s) { + List urls = new ArrayList(); + while (s._1.hasNext()) { + urls.add(s._1.next()); + } List> results = new ArrayList>(); - for (String n : s._1()) { - results.add(new Tuple2(n, s._2() / s._1().size())); + for (String n : urls) { + results.add(new Tuple2(n, s._2() / urls.size())); } return results; } diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala index 27afa6b642758..698c7cb2e85d1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala @@ -116,11 +116,13 @@ object WikipediaPageRankStandalone { for (i <- 1 to numIterations) { val contribs = links.groupWith(ranks).flatMap { case (id, (linksWrapper, rankWrapper)) => - if (linksWrapper.length > 0) { - if (rankWrapper.length > 0) { - linksWrapper(0).map(dest => (dest, rankWrapper(0) / linksWrapper(0).size)) + if (linksWrapper.hasNext) { + val linksWrapperHead = linksWrapper.next + if (rankWrapper.hasNext) { + val rankWrapperHead = rankWrapper.next + linksWrapperHead.map(dest => (dest, rankWrapperHead / linksWrapperHead.size)) } else { - linksWrapper(0).map(dest => (dest, defaultRank / linksWrapper(0).size)) + linksWrapperHead.map(dest => (dest, defaultRank / linksWrapperHead.size)) } } else { Array[(String, Double)]() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 0cc9f48769f83..fc9a74ffbef86 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -421,12 +421,12 @@ class ALS private ( * Compute the new feature vectors for a block of the users matrix given the list of factors * it received from each product and its InLinkBlock. */ - private def updateBlock(messages: Seq[(Int, Array[Array[Double]])], inLinkBlock: InLinkBlock, + private def updateBlock(messages: Iterator[(Int, Array[Array[Double]])], inLinkBlock: InLinkBlock, rank: Int, lambda: Double, alpha: Double, YtY: Option[Broadcast[DoubleMatrix]]) : Array[Array[Double]] = { // Sort the incoming block factor messages by block ID and make them an array - val blockFactors = messages.sortBy(_._1).map(_._2).toArray // Array[Array[Double]] + val blockFactors = messages.toArray.sortBy(_._1).map(_._2) // Array[Array[Double]] val numBlocks = blockFactors.length val numUsers = inLinkBlock.elementIds.length diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala index afe081295bfae..8445afef12357 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala @@ -38,8 +38,9 @@ object LAUtils { case (i, cols) => val rowArray = Array.ofDim[Double](n) var j = 0 - while (j < cols.size) { - rowArray(cols(j)._1) = cols(j)._2 + while (cols.hasNext) { + val element = cols.next + rowArray(element._1) = element._2 j += 1 } MatrixRow(i, rowArray) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index ac451d1913aaa..4b29f3e2f93b2 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming.api.java import java.lang.{Long => JLong} -import java.util.{List => JList} +import java.util.{List => JList, Iterator => JIterator} import scala.collection.JavaConversions._ import scala.reflect.ClassTag @@ -115,15 +115,15 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with Spark's default number of partitions. */ - def groupByKey(): JavaPairDStream[K, JList[V]] = - dstream.groupByKey().mapValues(seqAsJavaList _) + def groupByKey(): JavaPairDStream[K, JIterator[V]] = + dstream.groupByKey().mapValues(asJavaIterator _) /** * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with `numPartitions` partitions. */ - def groupByKey(numPartitions: Int): JavaPairDStream[K, JList[V]] = - dstream.groupByKey(numPartitions).mapValues(seqAsJavaList _) + def groupByKey(numPartitions: Int): JavaPairDStream[K, JIterator[V]] = + dstream.groupByKey(numPartitions).mapValues(asJavaIterator _) /** * Return a new DStream by applying `groupByKey` on each RDD of `this` DStream. @@ -131,8 +131,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * single sequence to generate the RDDs of the new DStream. org.apache.spark.Partitioner * is used to control the partitioning of each RDD. */ - def groupByKey(partitioner: Partitioner): JavaPairDStream[K, JList[V]] = - dstream.groupByKey(partitioner).mapValues(seqAsJavaList _) + def groupByKey(partitioner: Partitioner): JavaPairDStream[K, JIterator[V]] = + dstream.groupByKey(partitioner).mapValues(asJavaIterator _) /** * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are @@ -196,8 +196,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval */ - def groupByKeyAndWindow(windowDuration: Duration): JavaPairDStream[K, JList[V]] = { - dstream.groupByKeyAndWindow(windowDuration).mapValues(seqAsJavaList _) + def groupByKeyAndWindow(windowDuration: Duration): JavaPairDStream[K, JIterator[V]] = { + dstream.groupByKeyAndWindow(windowDuration).mapValues(asJavaIterator _) } /** @@ -211,8 +211,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * DStream's batching interval */ def groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration) - : JavaPairDStream[K, JList[V]] = { - dstream.groupByKeyAndWindow(windowDuration, slideDuration).mapValues(seqAsJavaList _) + : JavaPairDStream[K, JIterator[V]] = { + dstream.groupByKeyAndWindow(windowDuration, slideDuration).mapValues(asJavaIterator _) } /** @@ -227,9 +227,9 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * @param numPartitions Number of partitions of each RDD in the new DStream. */ def groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration, numPartitions: Int) - :JavaPairDStream[K, JList[V]] = { + :JavaPairDStream[K, JIterator[V]] = { dstream.groupByKeyAndWindow(windowDuration, slideDuration, numPartitions) - .mapValues(seqAsJavaList _) + .mapValues(asJavaIterator _) } /** @@ -247,9 +247,9 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( windowDuration: Duration, slideDuration: Duration, partitioner: Partitioner - ):JavaPairDStream[K, JList[V]] = { + ):JavaPairDStream[K, JIterator[V]] = { dstream.groupByKeyAndWindow(windowDuration, slideDuration, partitioner) - .mapValues(seqAsJavaList _) + .mapValues(asJavaIterator _) } /** @@ -518,9 +518,9 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * Hash partitioning is used to generate the RDDs with Spark's default number * of partitions. */ - def cogroup[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (JList[V], JList[W])] = { + def cogroup[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (JIterator[V], JIterator[W])] = { implicit val cm: ClassTag[W] = fakeClassTag - dstream.cogroup(other.dstream).mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2)))) + dstream.cogroup(other.dstream).mapValues(t => (asJavaIterator(t._1), asJavaIterator((t._2)))) } /** @@ -530,10 +530,10 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( def cogroup[W]( other: JavaPairDStream[K, W], numPartitions: Int - ): JavaPairDStream[K, (JList[V], JList[W])] = { + ): JavaPairDStream[K, (JIterator[V], JIterator[W])] = { implicit val cm: ClassTag[W] = fakeClassTag dstream.cogroup(other.dstream, numPartitions) - .mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2)))) + .mapValues(t => (asJavaIterator(t._1), asJavaIterator((t._2)))) } /** @@ -543,10 +543,10 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( def cogroup[W]( other: JavaPairDStream[K, W], partitioner: Partitioner - ): JavaPairDStream[K, (JList[V], JList[W])] = { + ): JavaPairDStream[K, (JIterator[V], JIterator[W])] = { implicit val cm: ClassTag[W] = fakeClassTag dstream.cogroup(other.dstream, partitioner) - .mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2)))) + .mapValues(t => (asJavaIterator(t._1), asJavaIterator((t._2)))) } /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala index 2473496949360..8beace1d8046a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala @@ -51,7 +51,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with Spark's default number of partitions. */ - def groupByKey(): DStream[(K, Seq[V])] = { + def groupByKey(): DStream[(K, Iterator[V])] = { groupByKey(defaultPartitioner()) } @@ -59,7 +59,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with `numPartitions` partitions. */ - def groupByKey(numPartitions: Int): DStream[(K, Seq[V])] = { + def groupByKey(numPartitions: Int): DStream[(K, Iterator[V])] = { groupByKey(defaultPartitioner(numPartitions)) } @@ -67,12 +67,12 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * Return a new DStream by applying `groupByKey` on each RDD. The supplied * org.apache.spark.Partitioner is used to control the partitioning of each RDD. */ - def groupByKey(partitioner: Partitioner): DStream[(K, Seq[V])] = { + def groupByKey(partitioner: Partitioner): DStream[(K, Iterator[V])] = { val createCombiner = (v: V) => ArrayBuffer[V](v) val mergeValue = (c: ArrayBuffer[V], v: V) => (c += v) val mergeCombiner = (c1: ArrayBuffer[V], c2: ArrayBuffer[V]) => (c1 ++ c2) combineByKey(createCombiner, mergeValue, mergeCombiner, partitioner) - .asInstanceOf[DStream[(K, Seq[V])]] + .asInstanceOf[DStream[(K, Iterator[V])]] } /** @@ -126,7 +126,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval */ - def groupByKeyAndWindow(windowDuration: Duration): DStream[(K, Seq[V])] = { + def groupByKeyAndWindow(windowDuration: Duration): DStream[(K, Iterator[V])] = { groupByKeyAndWindow(windowDuration, self.slideDuration, defaultPartitioner()) } @@ -140,7 +140,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * the new DStream will generate RDDs); must be a multiple of this * DStream's batching interval */ - def groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration): DStream[(K, Seq[V])] = + def groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration): DStream[(K, Iterator[V])] = { groupByKeyAndWindow(windowDuration, slideDuration, defaultPartitioner()) } @@ -161,7 +161,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) windowDuration: Duration, slideDuration: Duration, numPartitions: Int - ): DStream[(K, Seq[V])] = { + ): DStream[(K, Iterator[V])] = { groupByKeyAndWindow(windowDuration, slideDuration, defaultPartitioner(numPartitions)) } @@ -180,14 +180,14 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) windowDuration: Duration, slideDuration: Duration, partitioner: Partitioner - ): DStream[(K, Seq[V])] = { - val createCombiner = (v: Seq[V]) => new ArrayBuffer[V] ++= v - val mergeValue = (buf: ArrayBuffer[V], v: Seq[V]) => buf ++= v + ): DStream[(K, Iterator[V])] = { + val createCombiner = (v: Iterator[V]) => new ArrayBuffer[V] ++= v + val mergeValue = (buf: ArrayBuffer[V], v: Iterator[V]) => buf ++= v val mergeCombiner = (buf1: ArrayBuffer[V], buf2: ArrayBuffer[V]) => buf1 ++= buf2 self.groupByKey(partitioner) .window(windowDuration, slideDuration) .combineByKey[ArrayBuffer[V]](createCombiner, mergeValue, mergeCombiner, partitioner) - .asInstanceOf[DStream[(K, Seq[V])]] + .asInstanceOf[DStream[(K, Iterator[V])]] } /** @@ -438,7 +438,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * Hash partitioning is used to generate the RDDs with Spark's default number * of partitions. */ - def cogroup[W: ClassTag](other: DStream[(K, W)]): DStream[(K, (Seq[V], Seq[W]))] = { + def cogroup[W: ClassTag](other: DStream[(K, W)]): DStream[(K, (Iterator[V], Iterator[W]))] = { cogroup(other, defaultPartitioner()) } @@ -447,7 +447,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. */ def cogroup[W: ClassTag](other: DStream[(K, W)], numPartitions: Int) - : DStream[(K, (Seq[V], Seq[W]))] = { + : DStream[(K, (Iterator[V], Iterator[W]))] = { cogroup(other, defaultPartitioner(numPartitions)) } @@ -458,7 +458,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) def cogroup[W: ClassTag]( other: DStream[(K, W)], partitioner: Partitioner - ): DStream[(K, (Seq[V], Seq[W]))] = { + ): DStream[(K, (Iterator[V], Iterator[W]))] = { self.transformWith( other, (rdd1: RDD[(K, V)], rdd2: RDD[(K, W)]) => rdd1.cogroup(rdd2, partitioner) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala index 5f7d3ba26c656..09ff4163985c9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala @@ -56,9 +56,13 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag]( // first map the cogrouped tuple to tuples of required type, // and then apply the update function val updateFuncLocal = updateFunc - val finalFunc = (iterator: Iterator[(K, (Seq[V], Seq[S]))]) => { + val finalFunc = (iterator: Iterator[(K, (Iterator[V], Iterator[S]))]) => { val i = iterator.map(t => { - (t._1, t._2._1, t._2._2.headOption) + val headOption = t._2._2.hasNext match { + case true => Some(t._2._2.next()) + case false => None + } + (t._1, t._2._1.toSeq, headOption) }) updateFuncLocal(i) } @@ -90,8 +94,8 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag]( // first map the grouped tuple to tuples of required type, // and then apply the update function val updateFuncLocal = updateFunc - val finalFunc = (iterator: Iterator[(K, Seq[V])]) => { - updateFuncLocal(iterator.map(tuple => (tuple._1, tuple._2, None))) + val finalFunc = (iterator: Iterator[(K, Iterator[V])]) => { + updateFuncLocal(iterator.map(tuple => (tuple._1, tuple._2.toSeq, None))) } val groupedRDD = parentRDD.groupByKey(partitioner) From be50246ba9d4c4e946a1be9f67e77b0dc79d78e3 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 25 Mar 2014 15:40:31 -0700 Subject: [PATCH 02/39] Calling size on an iterator is not so good if we want to use it after --- bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala index 8edf6a1356ad6..a65964860fffc 100644 --- a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala +++ b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala @@ -227,7 +227,7 @@ object Bagel extends Logging { var numMsgs = sc.accumulator(0) var numActiveVerts = sc.accumulator(0) val processed = grouped.flatMapValues { - case (_, vs) if vs.size == 0 => None + case (_, vs) if !vs.hasNext => None case (c, vs) => val (newVert, newMsgs) = compute(vs.next, From 4991af61bf8a687e2c65d86ee26f8e3fdf172421 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 25 Mar 2014 15:40:37 -0700 Subject: [PATCH 03/39] Fix some tests --- core/src/test/scala/org/apache/spark/FailureSuite.scala | 5 ++--- core/src/test/scala/org/apache/spark/PipedRDDSuite.scala | 2 +- .../org/apache/spark/streaming/BasicOperationsSuite.scala | 8 ++++---- 3 files changed, 7 insertions(+), 8 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index f3fb64d87a2fd..fa052a10fd326 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -72,7 +72,8 @@ class FailureSuite extends FunSuite with LocalSparkContext { throw new Exception("Intentional task failure") } } - (k, v(0) * v(0)) + val vHead = v.next() + (k, vHead * vHead) }.collect() FailureSuiteState.synchronized { assert(FailureSuiteState.tasksRun === 4) @@ -137,5 +138,3 @@ class FailureSuite extends FunSuite with LocalSparkContext { // TODO: Need to add tests with shuffle fetch failures. } - - diff --git a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala index 627e9b5cd9060..d57b1d9c9dd81 100644 --- a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala @@ -85,7 +85,7 @@ class PipedRDDSuite extends FunSuite with SharedSparkContext { (f: String => Unit) => { bl.value.map(f(_)); f("\u0001") }, - (i: Tuple2[String, Seq[String]], f: String => Unit) => { + (i: Tuple2[String, Iterator[String]], f: String => Unit) => { for (e <- i._2) { f(e + "_") } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala index bb73dbf29b649..c269b274d4625 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -118,7 +118,7 @@ class BasicOperationsSuite extends TestSuiteBase { testOperation( Seq( Seq("a", "a", "b"), Seq("", ""), Seq() ), (s: DStream[String]) => s.map(x => (x, 1)).groupByKey(), - Seq( Seq(("a", Seq(1, 1)), ("b", Seq(1))), Seq(("", Seq(1, 1))), Seq() ), + Seq( Seq(("a", Seq(1, 1).toIterator), ("b", Seq(1).toIterator)), Seq(("", Seq(1, 1).toIterator)), Seq() ), true ) } @@ -245,9 +245,9 @@ class BasicOperationsSuite extends TestSuiteBase { val inputData1 = Seq( Seq("a", "a", "b"), Seq("a", ""), Seq(""), Seq() ) val inputData2 = Seq( Seq("a", "a", "b"), Seq("b", ""), Seq(), Seq() ) val outputData = Seq( - Seq( ("a", (Seq(1, 1), Seq("x", "x"))), ("b", (Seq(1), Seq("x"))) ), - Seq( ("a", (Seq(1), Seq())), ("b", (Seq(), Seq("x"))), ("", (Seq(1), Seq("x"))) ), - Seq( ("", (Seq(1), Seq())) ), + Seq( ("a", (Seq(1, 1).toIterator, Seq("x", "x"))).toIterator, ("b", (Seq(1).toIterator, Seq("x").toIterator)) ), + Seq( ("a", (Seq(1).toIterator, Seq())).toIterator, ("b", (Seq().toIterator, Seq("x").toIterator)), ("", (Seq(1).toIterator, Seq("x").toIterator)) ), + Seq( ("", (Seq(1).toIterator, Seq().toIterator)) ), Seq( ) ) val operation = (s1: DStream[String], s2: DStream[String]) => { From 88b3329c7cedf4a4ea4326208787de59f55a1cec Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 25 Mar 2014 16:06:02 -0700 Subject: [PATCH 04/39] Fix groupbykey to actually give back an iterator --- core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 3e005a7e1dc50..35e10c14f43c0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -270,7 +270,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) def mergeCombiners(c1: ArrayBuffer[V], c2: ArrayBuffer[V]) = c1 ++ c2 val bufs = combineByKey[ArrayBuffer[V]]( createCombiner _, mergeValue _, mergeCombiners _, partitioner, mapSideCombine=false) - bufs.asInstanceOf[RDD[(K, Iterator[V])]] + bufs.mapValues(_.toIterator) } /** From caafa63ffa5374c12ac8fecc2adef0501cb0fa12 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 25 Mar 2014 17:20:41 -0700 Subject: [PATCH 05/39] I think this might be a bad rabbit hole. Started work to make CoGroupedRDD use iterator and then went crazy --- .../org/apache/spark/rdd/CoGroupedRDD.scala | 27 ++++++++++++------- .../collection/ExternalAppendOnlyMap.scala | 24 ++++++++++++----- 2 files changed, 34 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 9aa454a5c8b88..6f22486465a8b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -23,7 +23,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv, TaskContext} import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency} -import org.apache.spark.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap} +import org.apache.spark.util.collection.{FlexibleExternalAppendOnlyMap, AppendOnlyMap} import org.apache.spark.serializer.Serializer private[spark] sealed trait CoGroupSplitDep extends Serializable @@ -58,14 +58,14 @@ private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep] * @param part partitioner used to partition the shuffle output. */ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: Partitioner) - extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) { + extends RDD[(K, Seq[Iterator[_]])](rdds.head.context, Nil) { // For example, `(k, a) cogroup (k, b)` produces k -> Seq(ArrayBuffer as, ArrayBuffer bs). // Each ArrayBuffer is represented as a CoGroup, and the resulting Seq as a CoGroupCombiner. // CoGroupValue is the intermediate state of each value before being merged in compute. private type CoGroup = ArrayBuffer[Any] private type CoGroupValue = (Any, Int) // Int is dependency number - private type CoGroupCombiner = Seq[CoGroup] + private type CoGroupCombiner = Array[CoGroup] private var serializer: Serializer = null @@ -105,7 +105,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: override val partitioner: Some[Partitioner] = Some(part) - override def compute(s: Partition, context: TaskContext): Iterator[(K, CoGroupCombiner)] = { + override def compute(s: Partition, context: TaskContext): Iterator[(K, Iterator[CoGroup])] = { val sparkConf = SparkEnv.get.conf val externalSorting = sparkConf.getBoolean("spark.shuffle.spill", true) val split = s.asInstanceOf[CoGroupPartition] @@ -141,7 +141,12 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: getCombiner(kv._1)(depNum) += kv._2 } } - new InterruptibleIterator(context, map.iterator) + // Convert to iterators + val finalMap = new AppendOnlyMap[K, Iterator[CoGroup]](math.max(map.size, 64)) + map.foreach { case (it, k) => + finalMap.update(it, k.iterator) + } + new InterruptibleIterator(context, finalMap.iterator) } else { val map = createExternalMap(numRdds) rddIterators.foreach { case (it, depNum) => @@ -157,7 +162,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: } private def createExternalMap(numRdds: Int) - : ExternalAppendOnlyMap[K, CoGroupValue, CoGroupCombiner] = { + : FlexibleExternalAppendOnlyMap[K, CoGroupValue, CoGroupCombiner, Iterator[CoGroup]] = { val createCombiner: (CoGroupValue => CoGroupCombiner) = value => { val newCombiner = Array.fill(numRdds)(new CoGroup) @@ -169,12 +174,14 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: value match { case (v, depNum) => combiner(depNum) += v } combiner } - val mergeCombiners: (CoGroupCombiner, CoGroupCombiner) => CoGroupCombiner = + val mergeCombiners: (CoGroupCombiner, Iterator[CoGroup]) => Iterator[CoGroup] = (combiner1, combiner2) => { - combiner1.zip(combiner2).map { case (v1, v2) => v1 ++ v2 } + combiner1.toIterator.zip(combiner2).map { case (v1, v2) => v1 ++ v2 } } - new ExternalAppendOnlyMap[K, CoGroupValue, CoGroupCombiner]( - createCombiner, mergeValue, mergeCombiners) + val returnCombiner: (CoGroupCombiner) => Iterator[CoGroup] = + (combiner) => combiner.toIterator + new FlexibleExternalAppendOnlyMap[K, CoGroupValue, CoGroupCombiner, Iterator[CoGroup]]( + createCombiner, mergeValue, mergeCombiners, returnCombiner) } override def clearDependencies() { diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index caa06d5b445b4..868351b5a2458 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -55,16 +55,25 @@ import org.apache.spark.storage.{BlockId, BlockManager} * `spark.shuffle.safetyFraction` specifies an additional margin of safety as a fraction of * this threshold, in case map size estimation is not sufficiently accurate. */ - private[spark] class ExternalAppendOnlyMap[K, V, C]( createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, serializer: Serializer = SparkEnv.get.serializer, blockManager: BlockManager = SparkEnv.get.blockManager) - extends Iterable[(K, C)] with Serializable with Logging { + extends FlexibleExternalAppendOnlyMap[K, V, C, C](createCombiner, mergeValue, mergeCombiners, (x => x), + serializer, blockManager) { +} +private[spark] class FlexibleExternalAppendOnlyMap[K, V, C, T]( + createCombiner: V => C, + mergeValue: (C, V) => C, + mergeCombiners: (C, T) => T, + returnCombiner: C => T, + serializer: Serializer = SparkEnv.get.serializer, + blockManager: BlockManager = SparkEnv.get.blockManager) + extends Iterable[(K, T)] with Serializable with Logging { - import ExternalAppendOnlyMap._ + import FlexibleExternalAppendOnlyMap._ private var currentMap = new SizeTrackingAppendOnlyMap[K, C] private val spilledMaps = new ArrayBuffer[DiskMapIterator] @@ -263,13 +272,13 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( * If the given buffer contains a value for the given key, merge that value into * baseCombiner and remove the corresponding (K, C) pair from the buffer. */ - private def mergeIfKeyExists(key: K, baseCombiner: C, buffer: StreamBuffer): C = { + private def mergeIfKeyExists(key: K, baseCombiner: T, buffer: StreamBuffer): T = { var i = 0 while (i < buffer.pairs.length) { val (k, c) = buffer.pairs(i) if (k == key) { buffer.pairs.remove(i) - return mergeCombiners(baseCombiner, c) + return mergeCombiners(c, baseCombiner) } i += 1 } @@ -292,7 +301,8 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( // Select a key from the StreamBuffer that holds the lowest key hash val minBuffer = mergeHeap.dequeue() val (minPairs, minHash) = (minBuffer.pairs, minBuffer.minKeyHash) - var (minKey, minCombiner) = minPairs.remove(0) + var (minKey, minCombinerC) = minPairs.remove(0) + var minCombiner = returnCombiner(minCombinerC) assert(minKey.hashCode() == minHash) // For all other streams that may have this key (i.e. have the same minimum key hash), @@ -418,7 +428,7 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( } } -private[spark] object ExternalAppendOnlyMap { +private[spark] object FlexibleExternalAppendOnlyMap { private class KCComparator[K, C] extends Comparator[(K, C)] { def compare(kc1: (K, C), kc2: (K, C)): Int = { kc1._1.hashCode().compareTo(kc2._1.hashCode()) From 172705c0e2a9d97f3a418e4913b976c36065f76e Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 25 Mar 2014 17:21:28 -0700 Subject: [PATCH 06/39] Fix Java API suite --- .../java/org/apache/spark/JavaAPISuite.java | 23 +++++++++++++------ 1 file changed, 16 insertions(+), 7 deletions(-) diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 762405be2a8f9..971bfbfbf1561 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -77,6 +77,15 @@ public int compare(Integer a, Integer b) { } } + public int iteratorSize(Iterator a) { + int size = 0; + while (a.hasNext()) { + size++; + a.next(); + } + return size; + } + @SuppressWarnings("unchecked") @Test public void sparkContextUnion() { @@ -197,7 +206,7 @@ public void lookup() { new Tuple2("Oranges", "Citrus") )); Assert.assertEquals(2, categories.lookup("Oranges").size()); - Assert.assertEquals(2, categories.groupByKey().lookup("Oranges").get(0).size()); + Assert.assertEquals(2, iteratorSize(categories.groupByKey().lookup("Oranges").get(0))); } @Test @@ -209,15 +218,15 @@ public Boolean call(Integer x) { return x % 2 == 0; } }; - JavaPairRDD> oddsAndEvens = rdd.groupBy(isOdd); + JavaPairRDD> oddsAndEvens = rdd.groupBy(isOdd); Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, oddsAndEvens.lookup(true).get(0).size()); // Evens - Assert.assertEquals(5, oddsAndEvens.lookup(false).get(0).size()); // Odds + Assert.assertEquals(2, iteratorSize(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, iteratorSize(oddsAndEvens.lookup(false).get(0))); // Odds oddsAndEvens = rdd.groupBy(isOdd, 1); Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, oddsAndEvens.lookup(true).get(0).size()); // Evens - Assert.assertEquals(5, oddsAndEvens.lookup(false).get(0).size()); // Odds + Assert.assertEquals(2, iteratorSize(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, iteratorSize(oddsAndEvens.lookup(false).get(0))); // Odds } @SuppressWarnings("unchecked") @@ -232,7 +241,7 @@ public void cogroup() { new Tuple2("Oranges", 2), new Tuple2("Apples", 3) )); - JavaPairRDD, List>> cogrouped = categories.cogroup(prices); + JavaPairRDD, Iterator>> cogrouped = categories.cogroup(prices); Assert.assertEquals("[Fruit, Citrus]", cogrouped.lookup("Oranges").get(0)._1().toString()); Assert.assertEquals("[2]", cogrouped.lookup("Oranges").get(0)._2().toString()); From fe992fec8254811e3fdc2e592db9976cae105593 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 25 Mar 2014 17:21:45 -0700 Subject: [PATCH 07/39] hmmm try and fix up basic operation suite --- .../org/apache/spark/streaming/BasicOperationsSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala index c269b274d4625..270ceb081480f 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -245,8 +245,8 @@ class BasicOperationsSuite extends TestSuiteBase { val inputData1 = Seq( Seq("a", "a", "b"), Seq("a", ""), Seq(""), Seq() ) val inputData2 = Seq( Seq("a", "a", "b"), Seq("b", ""), Seq(), Seq() ) val outputData = Seq( - Seq( ("a", (Seq(1, 1).toIterator, Seq("x", "x"))).toIterator, ("b", (Seq(1).toIterator, Seq("x").toIterator)) ), - Seq( ("a", (Seq(1).toIterator, Seq())).toIterator, ("b", (Seq().toIterator, Seq("x").toIterator)), ("", (Seq(1).toIterator, Seq("x").toIterator)) ), + Seq( ("a", (Seq(1, 1).toIterator, Seq("x", "x").toIterator)), ("b", (Seq(1).toIterator, Seq("x").toIterator)) ), + Seq( ("a", (Seq(1).toIterator, Seq().toIterator)), ("b", (Seq().toIterator, Seq("x").toIterator)), ("", (Seq(1).toIterator, Seq("x").toIterator)) ), Seq( ("", (Seq(1).toIterator, Seq().toIterator)) ), Seq( ) ) From 669818600c0f9b6a7d5fbb4f645ed55e6a21634d Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 25 Mar 2014 17:22:20 -0700 Subject: [PATCH 08/39] Revert "I think this might be a bad rabbit hole. Started work to make CoGroupedRDD use iterator and then went crazy" This reverts commit df9afbec7e9fb558cf75d4e8dc94d8f44f101301. --- .../org/apache/spark/rdd/CoGroupedRDD.scala | 27 +++++++------------ .../collection/ExternalAppendOnlyMap.scala | 24 +++++------------ 2 files changed, 17 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 6f22486465a8b..9aa454a5c8b88 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -23,7 +23,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv, TaskContext} import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency} -import org.apache.spark.util.collection.{FlexibleExternalAppendOnlyMap, AppendOnlyMap} +import org.apache.spark.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap} import org.apache.spark.serializer.Serializer private[spark] sealed trait CoGroupSplitDep extends Serializable @@ -58,14 +58,14 @@ private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep] * @param part partitioner used to partition the shuffle output. */ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: Partitioner) - extends RDD[(K, Seq[Iterator[_]])](rdds.head.context, Nil) { + extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) { // For example, `(k, a) cogroup (k, b)` produces k -> Seq(ArrayBuffer as, ArrayBuffer bs). // Each ArrayBuffer is represented as a CoGroup, and the resulting Seq as a CoGroupCombiner. // CoGroupValue is the intermediate state of each value before being merged in compute. private type CoGroup = ArrayBuffer[Any] private type CoGroupValue = (Any, Int) // Int is dependency number - private type CoGroupCombiner = Array[CoGroup] + private type CoGroupCombiner = Seq[CoGroup] private var serializer: Serializer = null @@ -105,7 +105,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: override val partitioner: Some[Partitioner] = Some(part) - override def compute(s: Partition, context: TaskContext): Iterator[(K, Iterator[CoGroup])] = { + override def compute(s: Partition, context: TaskContext): Iterator[(K, CoGroupCombiner)] = { val sparkConf = SparkEnv.get.conf val externalSorting = sparkConf.getBoolean("spark.shuffle.spill", true) val split = s.asInstanceOf[CoGroupPartition] @@ -141,12 +141,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: getCombiner(kv._1)(depNum) += kv._2 } } - // Convert to iterators - val finalMap = new AppendOnlyMap[K, Iterator[CoGroup]](math.max(map.size, 64)) - map.foreach { case (it, k) => - finalMap.update(it, k.iterator) - } - new InterruptibleIterator(context, finalMap.iterator) + new InterruptibleIterator(context, map.iterator) } else { val map = createExternalMap(numRdds) rddIterators.foreach { case (it, depNum) => @@ -162,7 +157,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: } private def createExternalMap(numRdds: Int) - : FlexibleExternalAppendOnlyMap[K, CoGroupValue, CoGroupCombiner, Iterator[CoGroup]] = { + : ExternalAppendOnlyMap[K, CoGroupValue, CoGroupCombiner] = { val createCombiner: (CoGroupValue => CoGroupCombiner) = value => { val newCombiner = Array.fill(numRdds)(new CoGroup) @@ -174,14 +169,12 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: value match { case (v, depNum) => combiner(depNum) += v } combiner } - val mergeCombiners: (CoGroupCombiner, Iterator[CoGroup]) => Iterator[CoGroup] = + val mergeCombiners: (CoGroupCombiner, CoGroupCombiner) => CoGroupCombiner = (combiner1, combiner2) => { - combiner1.toIterator.zip(combiner2).map { case (v1, v2) => v1 ++ v2 } + combiner1.zip(combiner2).map { case (v1, v2) => v1 ++ v2 } } - val returnCombiner: (CoGroupCombiner) => Iterator[CoGroup] = - (combiner) => combiner.toIterator - new FlexibleExternalAppendOnlyMap[K, CoGroupValue, CoGroupCombiner, Iterator[CoGroup]]( - createCombiner, mergeValue, mergeCombiners, returnCombiner) + new ExternalAppendOnlyMap[K, CoGroupValue, CoGroupCombiner]( + createCombiner, mergeValue, mergeCombiners) } override def clearDependencies() { diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 868351b5a2458..caa06d5b445b4 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -55,25 +55,16 @@ import org.apache.spark.storage.{BlockId, BlockManager} * `spark.shuffle.safetyFraction` specifies an additional margin of safety as a fraction of * this threshold, in case map size estimation is not sufficiently accurate. */ + private[spark] class ExternalAppendOnlyMap[K, V, C]( createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, serializer: Serializer = SparkEnv.get.serializer, blockManager: BlockManager = SparkEnv.get.blockManager) - extends FlexibleExternalAppendOnlyMap[K, V, C, C](createCombiner, mergeValue, mergeCombiners, (x => x), - serializer, blockManager) { -} -private[spark] class FlexibleExternalAppendOnlyMap[K, V, C, T]( - createCombiner: V => C, - mergeValue: (C, V) => C, - mergeCombiners: (C, T) => T, - returnCombiner: C => T, - serializer: Serializer = SparkEnv.get.serializer, - blockManager: BlockManager = SparkEnv.get.blockManager) - extends Iterable[(K, T)] with Serializable with Logging { + extends Iterable[(K, C)] with Serializable with Logging { - import FlexibleExternalAppendOnlyMap._ + import ExternalAppendOnlyMap._ private var currentMap = new SizeTrackingAppendOnlyMap[K, C] private val spilledMaps = new ArrayBuffer[DiskMapIterator] @@ -272,13 +263,13 @@ private[spark] class FlexibleExternalAppendOnlyMap[K, V, C, T]( * If the given buffer contains a value for the given key, merge that value into * baseCombiner and remove the corresponding (K, C) pair from the buffer. */ - private def mergeIfKeyExists(key: K, baseCombiner: T, buffer: StreamBuffer): T = { + private def mergeIfKeyExists(key: K, baseCombiner: C, buffer: StreamBuffer): C = { var i = 0 while (i < buffer.pairs.length) { val (k, c) = buffer.pairs(i) if (k == key) { buffer.pairs.remove(i) - return mergeCombiners(c, baseCombiner) + return mergeCombiners(baseCombiner, c) } i += 1 } @@ -301,8 +292,7 @@ private[spark] class FlexibleExternalAppendOnlyMap[K, V, C, T]( // Select a key from the StreamBuffer that holds the lowest key hash val minBuffer = mergeHeap.dequeue() val (minPairs, minHash) = (minBuffer.pairs, minBuffer.minKeyHash) - var (minKey, minCombinerC) = minPairs.remove(0) - var minCombiner = returnCombiner(minCombinerC) + var (minKey, minCombiner) = minPairs.remove(0) assert(minKey.hashCode() == minHash) // For all other streams that may have this key (i.e. have the same minimum key hash), @@ -428,7 +418,7 @@ private[spark] class FlexibleExternalAppendOnlyMap[K, V, C, T]( } } -private[spark] object FlexibleExternalAppendOnlyMap { +private[spark] object ExternalAppendOnlyMap { private class KCComparator[K, C] extends Comparator[(K, C)] { def compare(kc1: (K, C), kc2: (K, C)): Int = { kc1._1.hashCode().compareTo(kc2._1.hashCode()) From 249abdec9a964178675aab6eb91dca9adc112872 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 25 Mar 2014 17:45:13 -0700 Subject: [PATCH 09/39] org.apache.spark.rdd.PairRDDFunctionsSuite passes --- .../org/apache/spark/rdd/PairRDDFunctions.scala | 9 ++++++--- .../org/apache/spark/rdd/PairRDDFunctionsSuite.scala | 12 ++++++------ 2 files changed, 12 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 35e10c14f43c0..58afa72e099bb 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -298,7 +298,8 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) */ def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => - for (v <- vs; w <- ws) yield (v, w) + val wlist = ws.toList + for (v <- vs; w <- wlist.iterator) yield (v, w) } } @@ -313,7 +314,8 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) if (ws.isEmpty) { vs.map(v => (v, None)) } else { - for (v <- vs; w <- ws) yield (v, Some(w)) + val wlist = ws.toList + for (v <- vs; w <- wlist.iterator) yield (v, Some(w)) } } } @@ -330,7 +332,8 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) if (vs.isEmpty) { ws.map(w => (None, w)) } else { - for (v <- vs; w <- ws) yield (Some(v), w) + val wlist = ws.toList + for (v <- vs; w <- wlist) yield (Some(v), w) } } } diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index f9e994b13dfbc..8f3e6bd21b752 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -225,11 +225,12 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) val joined = rdd1.groupWith(rdd2).collect() assert(joined.size === 4) - assert(joined.toSet === Set( - (1, (ArrayBuffer(1, 2), ArrayBuffer('x'))), - (2, (ArrayBuffer(1), ArrayBuffer('y', 'z'))), - (3, (ArrayBuffer(1), ArrayBuffer())), - (4, (ArrayBuffer(), ArrayBuffer('w'))) + val joinedSet = joined.map(x => (x._1, (x._2._1.toList, x._2._2.toList))).toSet + assert(joinedSet === Set( + (1, (List(1, 2), List('x'))), + (2, (List(1), List('y', 'z'))), + (3, (List(1), List())), + (4, (List(), List('w'))) )) } @@ -447,4 +448,3 @@ class ConfigTestFormat() extends FakeFormat() with Configurable { super.getRecordWriter(p1) } } - From 37888ecd4e2a89be6244e56f734988b068285aca Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 25 Mar 2014 18:29:18 -0700 Subject: [PATCH 10/39] core/tests now pass --- .../java/org/apache/spark/JavaAPISuite.java | 18 ++++++++++++++++-- .../ExternalAppendOnlyMapSuite.scala | 4 ++-- 2 files changed, 18 insertions(+), 4 deletions(-) diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 971bfbfbf1561..980a6236d3cf3 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -18,6 +18,7 @@ package org.apache.spark; import java.io.*; +import java.lang.StringBuilder; import java.util.*; import scala.Tuple2; @@ -86,6 +87,19 @@ public int iteratorSize(Iterator a) { return size; } + public String iteratorStr(Iterator a) { + StringBuilder str = new StringBuilder(); + str.append("["); + while (a.hasNext()) { + str.append(a.next().toString()); + if (a.hasNext()) { + str.append(", "); + } + } + str.append("]"); + return str.toString(); + } + @SuppressWarnings("unchecked") @Test public void sparkContextUnion() { @@ -242,8 +256,8 @@ public void cogroup() { new Tuple2("Apples", 3) )); JavaPairRDD, Iterator>> cogrouped = categories.cogroup(prices); - Assert.assertEquals("[Fruit, Citrus]", cogrouped.lookup("Oranges").get(0)._1().toString()); - Assert.assertEquals("[2]", cogrouped.lookup("Oranges").get(0)._2().toString()); + Assert.assertEquals("[Fruit, Citrus]", iteratorStr(cogrouped.lookup("Oranges").get(0)._1())); + Assert.assertEquals("[2]", iteratorStr(cogrouped.lookup("Oranges").get(0)._2())); cogrouped.collect(); } diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index fce1184d46364..cdebefb67510c 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -174,9 +174,9 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { assert(result1.toSet == Set[(Int, Int)]((0, 5), (1, 5))) // groupByKey - val result2 = rdd.groupByKey().collect() + val result2 = rdd.groupByKey().collect().map(x => (x._1, x._2.toList)).toSet assert(result2.toSet == Set[(Int, Seq[Int])] - ((0, ArrayBuffer[Int](1, 1, 1, 1, 1)), (1, ArrayBuffer[Int](1, 1, 1, 1, 1)))) + ((0, List[Int](1, 1, 1, 1, 1)), (1, List[Int](1, 1, 1, 1, 1)))) } test("simple cogroup") { From b1ae51a55b62e390cfe5d9621015bfc888fe9b47 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 25 Mar 2014 18:37:32 -0700 Subject: [PATCH 11/39] Fix some of the types in the streaming JavaAPI suite. Probably still needs more work --- .../test/java/org/apache/spark/streaming/JavaAPISuite.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index e93bf18b6d0b9..bfda617d60a08 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -1016,7 +1016,7 @@ public void testPairGroupByKey() { JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream> grouped = pairStream.groupByKey(); + JavaPairDStream> grouped = pairStream.groupByKey(); JavaTestUtils.attachTestOutputStream(grouped); List>>> result = JavaTestUtils.runStreams(ssc, 2, 2); @@ -1128,7 +1128,7 @@ public void testGroupByKeyAndWindow() { JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream> groupWindowed = + JavaPairDStream> groupWindowed = pairStream.groupByKeyAndWindow(new Duration(2000), new Duration(1000)); JavaTestUtils.attachTestOutputStream(groupWindowed); List>>> result = JavaTestUtils.runStreams(ssc, 3, 3); @@ -1471,7 +1471,7 @@ public void testCoGroup() { ssc, stringStringKVStream2, 1); JavaPairDStream pairStream2 = JavaPairDStream.fromJavaDStream(stream2); - JavaPairDStream, List>> grouped = pairStream1.cogroup(pairStream2); + JavaPairDStream, Iterator>> grouped = pairStream1.cogroup(pairStream2); JavaTestUtils.attachTestOutputStream(grouped); List, List>>>> result = JavaTestUtils.runStreams(ssc, 2, 2); From 71e8b9f66e35633856d5c170e6ea069a8523f287 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Wed, 26 Mar 2014 09:38:46 -0700 Subject: [PATCH 12/39] I really need to stop calling size on iterators, it is the path of sadness. --- bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala index a65964860fffc..a389d5077bd47 100644 --- a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala +++ b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala @@ -228,12 +228,12 @@ object Bagel extends Logging { var numActiveVerts = sc.accumulator(0) val processed = grouped.flatMapValues { case (_, vs) if !vs.hasNext => None - case (c, vs) => + case (c, vs) => { val (newVert, newMsgs) = compute(vs.next, - c.size match { - case 1 => Some(c.next) - case _ => None + c.hasNext match { + case true => Some(c.next) + case false => None } ) @@ -243,6 +243,7 @@ object Bagel extends Logging { } Some((newVert, newMsgs)) + } }.persist(storageLevel) // Force evaluation of processed RDD for accurate performance measurements From b4e0b1d65611aa150aecdcec57ec95f1510cde0f Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Wed, 26 Mar 2014 09:51:07 -0700 Subject: [PATCH 13/39] Fix style issues --- .../main/scala/org/apache/spark/api/java/JavaPairRDD.scala | 3 ++- .../main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 7 +++++-- .../spark/streaming/dstream/PairDStreamFunctions.scala | 3 ++- 3 files changed, 9 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index acea9780d25fd..fd82230f122c8 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -707,7 +707,8 @@ object JavaPairRDD { private[spark] def cogroupResult2ToJava[K: ClassTag, V, W1, W2]( - rdd: RDD[(K, (Iterator[V], Iterator[W1], Iterator[W2]))]): RDD[(K, (JIterator[V], JIterator[W1], JIterator[W2]))] = { + rdd: RDD[(K, (Iterator[V], Iterator[W1], Iterator[W2]))]) + : RDD[(K, (JIterator[V], JIterator[W1], JIterator[W2]))] = { rddToPairRDDFunctions(rdd) .mapValues(x => (asJavaIterator(x._1), asJavaIterator(x._2), asJavaIterator(x._3))) } diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 58afa72e099bb..1b9af45c6bdbb 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -456,7 +456,8 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (Iterator[V], Iterator[W]))] = { + def cogroup[W](other: RDD[(K, W)], partitioner: Partitioner) + : RDD[(K, (Iterator[V], Iterator[W]))] = { if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { throw new SparkException("Default partitioner cannot partition array keys.") } @@ -477,7 +478,9 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) } val cg = new CoGroupedRDD[K](Seq(self, other1, other2), partitioner) cg.mapValues { case Seq(vs, w1s, w2s) => - (vs.asInstanceOf[Seq[V]].iterator, w1s.asInstanceOf[Seq[W1]].iterator, w2s.asInstanceOf[Seq[W2]].iterator) + (vs.asInstanceOf[Seq[V]].iterator, + w1s.asInstanceOf[Seq[W1]].iterator, + w2s.asInstanceOf[Seq[W2]].iterator) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala index 8beace1d8046a..c5d9b059c1410 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala @@ -140,7 +140,8 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * the new DStream will generate RDDs); must be a multiple of this * DStream's batching interval */ - def groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration): DStream[(K, Iterator[V])] = + def groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration) + : DStream[(K, Iterator[V])] = { groupByKeyAndWindow(windowDuration, slideDuration, defaultPartitioner()) } From c4510e2840194d40eb9492a5a69172c7e15959ef Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Wed, 26 Mar 2014 10:59:10 -0700 Subject: [PATCH 14/39] Revert this but for now put things in list pandas --- .../scala/org/apache/spark/rdd/PairRDDFunctions.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 1b9af45c6bdbb..609b30f87b264 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -298,8 +298,9 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) */ def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => + val vlist = vs.toList val wlist = ws.toList - for (v <- vs; w <- wlist.iterator) yield (v, w) + for (v <- vlist; w <- wlist) yield (v, w) } } @@ -314,8 +315,9 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) if (ws.isEmpty) { vs.map(v => (v, None)) } else { + val vlist = vs.toList val wlist = ws.toList - for (v <- vs; w <- wlist.iterator) yield (v, Some(w)) + for (v <- vlist; w <- wlist) yield (v, Some(w)) } } } @@ -332,8 +334,9 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) if (vs.isEmpty) { ws.map(w => (None, w)) } else { + val vlist = vs.toList val wlist = ws.toList - for (v <- vs; w <- wlist) yield (Some(v), w) + for (v <- vlist; w <- wlist) yield (Some(v), w) } } } From b4e86a94d5c3056ac2469be9d14ea6a840389d88 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Wed, 26 Mar 2014 13:21:41 -0700 Subject: [PATCH 15/39] Add a join based on the problem in SVD --- .../org/apache/spark/rdd/PairRDDFunctionsSuite.scala | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 8f3e6bd21b752..1787fe929865d 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -157,6 +157,18 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { )) } + test("join repro") { + val rdd1 = sc.parallelize(Array((0,(0,2.0)), (1,(0,2.0)), (2,(0,2.0)), (0,(1,1.5)), (1,(1,2.0)), (2,(1,2.25)), (0,(2,1.3333333333333333)), (1,(2,2.0)), + (2,(2,2.4)), (0,(3,1.25)), (1,(3,2.0)), (2,(3,2.5)), (0,(4,1.2)), (1,(4,2.0)), (2,(4,2.5714285714285716)), + (0,(5,1.1666666666666667)), (1,(5,2.0)), (2,(5,2.625)), (0,(6,1.1428571428571428)), (1,(6,2.0)), (2,(6,2.6666666666666665)), (0,(7,1.125)), + (1,(7,2.0)), (2,(7,2.7)), (0,(8,1.1111111111111112)), (1,(8,2.0)), (2,(8,2.727272727272727)), (0,(9,1.1)), (1,(9,2.0)), (2,(9,2.75)) + )) + val rdd2 = sc.parallelize(Array((0,(0,-0.03387252842009158)), (0,(1,0.789223562443648)), (0,(2,4.541482001513894)), (1,(0,-0.05259151798283574)), (1,(1,0.10552081450086591)), (1,(2,-9.596168335700813)), (2,(0,-0.06636448259750038)), (2,(1,-0.48644238753215385)), (2,(2,5.2866468281774734)) + )) + val joined = rdd1.join(rdd2).collect() + assert(joined.toSet === Set((1,((6,2.0),(0,-0.05259151798283574))), (2,((6,2.6666666666666665),(0,-0.06636448259750038))), (1,((2,2.0),(2,-9.596168335700813))), (1,((1,2.0),(2,-9.596168335700813))), (1,((3,2.0),(2,-9.596168335700813))), (0,((3,1.25),(2,4.541482001513894))), (1,((6,2.0),(2,-9.596168335700813))), (2,((4,2.5714285714285716),(2,5.2866468281774734))), (0,((2,1.3333333333333333),(2,4.541482001513894))), (0,((4,1.2),(0,-0.03387252842009158))), (1,((8,2.0),(0,-0.05259151798283574))), (1,((4,2.0),(1,0.10552081450086591))), (0,((9,1.1),(0,-0.03387252842009158))), (0,((7,1.125),(1,0.789223562443648))), (0,((2,1.3333333333333333),(0,-0.03387252842009158))), (1,((9,2.0),(1,0.10552081450086591))), (2,((8,2.727272727272727),(0,-0.06636448259750038))), (2,((6,2.6666666666666665),(2,5.2866468281774734))), (2,((0,2.0),(0,-0.06636448259750038))), (1,((3,2.0),(0,-0.05259151798283574))), (2,((4,2.5714285714285716),(0,-0.06636448259750038))), (1,((4,2.0),(0,-0.05259151798283574))), (0,((6,1.1428571428571428),(2,4.541482001513894))), (0,((7,1.125),(2,4.541482001513894))), (2,((0,2.0),(2,5.2866468281774734))), (1,((0,2.0),(0,-0.05259151798283574))), (2,((3,2.5),(2,5.2866468281774734))), (1,((7,2.0),(2,-9.596168335700813))), (1,((4,2.0),(2,-9.596168335700813))), (0,((4,1.2),(2,4.541482001513894))), (2,((7,2.7),(0,-0.06636448259750038))), (1,((7,2.0),(1,0.10552081450086591))), (2,((8,2.727272727272727),(1,-0.48644238753215385))), (2,((1,2.25),(2,5.2866468281774734))), (0,((8,1.1111111111111112),(0,-0.03387252842009158))), (0,((3,1.25),(0,-0.03387252842009158))), (1,((3,2.0),(1,0.10552081450086591))), (2,((0,2.0),(1,-0.48644238753215385))), (2,((5,2.625),(0,-0.06636448259750038))), (1,((5,2.0),(0,-0.05259151798283574))), (2,((1,2.25),(0,-0.06636448259750038))), (2,((2,2.4),(1,-0.48644238753215385))), (2,((5,2.625),(1,-0.48644238753215385))), (2,((7,2.7),(2,5.2866468281774734))), (0,((6,1.1428571428571428),(0,-0.03387252842009158))), (0,((8,1.1111111111111112),(2,4.541482001513894))), (0,((1,1.5),(2,4.541482001513894))), (0,((5,1.1666666666666667),(2,4.541482001513894))), (2,((3,2.5),(0,-0.06636448259750038))), (1,((2,2.0),(0,-0.05259151798283574))), (2,((7,2.7),(1,-0.48644238753215385))), (0,((2,1.3333333333333333),(1,0.789223562443648))), (2,((9,2.75),(2,5.2866468281774734))), (0,((5,1.1666666666666667),(1,0.789223562443648))), (0,((0,2.0),(0,-0.03387252842009158))), (1,((8,2.0),(2,-9.596168335700813))), (0,((1,1.5),(0,-0.03387252842009158))), (2,((9,2.75),(1,-0.48644238753215385))), (2,((4,2.5714285714285716),(1,-0.48644238753215385))), (0,((0,2.0),(2,4.541482001513894))), (0,((8,1.1111111111111112),(1,0.789223562443648))), (0,((9,1.1),(2,4.541482001513894))), (0,((0,2.0),(1,0.789223562443648))), (1,((1,2.0),(1,0.10552081450086591))), (2,((6,2.6666666666666665),(1,-0.48644238753215385))), (0,((6,1.1428571428571428),(1,0.789223562443648))), (2,((9,2.75),(0,-0.06636448259750038))), (1,((9,2.0),(2,-9.596168335700813))), (2,((8,2.727272727272727),(2,5.2866468281774734))), (0,((4,1.2),(1,0.789223562443648))), (1,((5,2.0),(1,0.10552081450086591))), (1,((6,2.0),(1,0.10552081450086591))), (1,((5,2.0),(2,-9.596168335700813))), (1,((0,2.0),(2,-9.596168335700813))), (2,((5,2.625),(2,5.2866468281774734))), (1,((7,2.0),(0,-0.05259151798283574))), (0,((5,1.1666666666666667),(0,-0.03387252842009158))), (1,((1,2.0),(0,-0.05259151798283574))), (0,((9,1.1),(1,0.789223562443648))), (1,((9,2.0),(0,-0.05259151798283574))), (0,((7,1.125),(0,-0.03387252842009158))), (0,((3,1.25),(1,0.789223562443648))), (2,((3,2.5),(1,-0.48644238753215385))), (1,((2,2.0),(1,0.10552081450086591))), (1,((8,2.0),(1,0.10552081450086591))), (1,((0,2.0),(1,0.10552081450086591))), (2,((2,2.4),(0,-0.06636448259750038))), (2,((1,2.25),(1,-0.48644238753215385))), (0,((1,1.5),(1,0.789223562443648))), (2,((2,2.4),(2,5.2866468281774734))))) + } + test("join all-to-all") { val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (1, 3))) val rdd2 = sc.parallelize(Array((1, 'x'), (1, 'y'))) From 8a5153a8b573f5a51a6725c4dfdfab95b63a5359 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Wed, 26 Mar 2014 13:35:20 -0700 Subject: [PATCH 16/39] Revert me, but we have some stuff to debug more loggings --- .../org/apache/spark/mllib/linalg/SVD.scala | 19 +++++++++++--- .../apache/spark/mllib/linalg/SVDSuite.scala | 25 ++++++++++--------- 2 files changed, 28 insertions(+), 16 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala index 3e7cc648d1d37..a9618c0b7c688 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala @@ -69,11 +69,11 @@ class SVD { /** * Compute SVD using the current set parameters - * Returns (U, S, V) such that A = USV^T + * Returns (U, S, V) such that A = USV^T * U is a row-by-row dense matrix * S is a simple double array of singular values * V is a 2d array matrix - * See [[denseSVD]] for more documentation + * See [[denseSVD]] for more documentation */ def compute(matrix: RDD[Array[Double]]): (RDD[Array[Double]], Array[Double], Array[Array[Double]]) = { @@ -118,6 +118,7 @@ class SVD { val m = matrix.m val n = matrix.n + println("matrix "+matrix) if (m < n || m <= 0 || n <= 0) { throw new IllegalArgumentException("Expecting a tall and skinny matrix m=$m n=$n") } @@ -129,6 +130,7 @@ class SVD { val rowIndices = matrix.rows.map(_.i) // compute SVD + println("Computing SVD") val (u, sigma, v) = denseSVD(matrix.rows.map(_.data)) if (computeU) { @@ -286,6 +288,7 @@ class SVD { // Compute A^T A, assuming rows are sparse enough to fit in memory val rows = data.map(entry => (entry.i, (entry.j, entry.mval))).groupByKey() + println("rows "+rows) val emits = rows.flatMap { case (rowind, cols) => cols.flatMap { @@ -296,18 +299,26 @@ class SVD { } } }.reduceByKey(_ + _) + println("emits ") + emits.foreach{e => println("e "+e)} // Construct jblas A^T A locally val ata = DoubleMatrix.zeros(n, n) for (entry <- emits.collect()) { ata.put(entry._1._1, entry._1._2, entry._2) } + println("ata "+ata) // Since A^T A is small, we can compute its SVD directly val svd = Singular.sparseSVD(ata) + println("init svd") + svd.foreach{x => println(x)} + println("done") val V = svd(0) + println("V" + V) // This will be updated to rcond val sigmas = MatrixFunctions.sqrt(svd(1)).toArray.filter(x => x > 1e-9) + println("Sigmas "+sigmas) if (sigmas.size < k) { throw new Exception("Not enough singular values to return k=" + k + " s=" + sigmas.size) @@ -341,6 +352,8 @@ class SVD { // Multiply A by VS^-1 val aCols = data.map(entry => (entry.j, (entry.i, entry.mval))) val bRows = vsirdd.map(entry => (entry._1._1, (entry._1._2, entry._2))) + aCols.foreach{x => println("a "+x)} + bRows.foreach{x => println("b "+x)} val retUdata = aCols.join(bRows).map { case (key, ((rowInd, rowVal), (colInd, colVal))) => ((rowInd, colInd), rowVal * colVal) @@ -393,5 +406,3 @@ object SVD { System.exit(0) } } - - diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala index 20e2b0f84be06..3a9811a72240b 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala @@ -87,17 +87,17 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll { val retu = getDenseMatrix(u) val rets = getDenseMatrix(s) val retv = getDenseMatrix(v) - - - // check individual decomposition + + + // check individual decomposition assertMatrixApproximatelyEquals(retu, svd(0)) assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1))) assertMatrixApproximatelyEquals(retv, svd(2)) // check multiplication guarantee - assertMatrixApproximatelyEquals(retu.mmul(rets).mmul(retv.transpose), denseA) + assertMatrixApproximatelyEquals(retu.mmul(rets).mmul(retv.transpose), denseA) } - +/* test("dense full rank matrix svd") { val m = 10 val n = 3 @@ -120,7 +120,7 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll { val retv = new DoubleMatrix(v) - // check individual decomposition + // check individual decomposition assertMatrixApproximatelyEquals(retu, svd(0)) assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1))) assertMatrixApproximatelyEquals(retv, svd(2)) @@ -131,7 +131,7 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll { test("rank one matrix svd") { val m = 10 - val n = 3 + val n = 3 val data = sc.makeRDD(Array.tabulate(m, n){ (a,b) => MatrixEntry(a, b, 1.0) }.flatten ) val k = 1 @@ -153,13 +153,13 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll { val rets = getDenseMatrix(s) val retv = getDenseMatrix(v) - // check individual decomposition + // check individual decomposition assertMatrixApproximatelyEquals(retu, svd(0).getColumn(0)) assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1).getRow(0))) assertMatrixApproximatelyEquals(retv, svd(2).getColumn(0)) // check multiplication guarantee - assertMatrixApproximatelyEquals(retu.mmul(rets).mmul(retv.transpose), denseA) + assertMatrixApproximatelyEquals(retu.mmul(rets).mmul(retv.transpose), denseA) } test("truncated with k") { @@ -168,7 +168,7 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll { val data = sc.makeRDD(Array.tabulate(m,n){ (a, b) => MatrixEntry(a, b, (a + 2).toDouble * (b + 1)/(1 + a + b)) }.flatten ) val a = SparseMatrix(data, m, n) - + val k = 1 // only one svalue above this val decomposed = new SVD().setK(k).compute(a) @@ -185,10 +185,11 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll { val retv = getDenseMatrix(v) assert(retrank == 1, "rank returned not one") - - // check individual decomposition + + // check individual decomposition assertMatrixApproximatelyEquals(retu, svd(0).getColumn(0)) assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1).getRow(0))) assertMatrixApproximatelyEquals(retv, svd(2).getColumn(0)) } + */ } From 7e533f767d96817de191038d3c148f4172deb240 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Wed, 26 Mar 2014 14:11:33 -0700 Subject: [PATCH 17/39] Fix the bug --- mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala index a9618c0b7c688..9157eb9db7791 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala @@ -289,8 +289,10 @@ class SVD { val rows = data.map(entry => (entry.i, (entry.j, entry.mval))).groupByKey() println("rows "+rows) + rows.foreach{e => println("r "+e)} val emits = rows.flatMap { - case (rowind, cols) => + case (rowind, colsI) => + val cols = colsI.toList cols.flatMap { case (colind1, mval1) => cols.map { From b692868af13a24b5e75a300d4a0ad9907b9a2512 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Wed, 26 Mar 2014 15:12:19 -0700 Subject: [PATCH 18/39] Revert --- .../org/apache/spark/mllib/linalg/SVD.scala | 14 ----------- .../apache/spark/mllib/linalg/SVDSuite.scala | 25 +++++++++---------- 2 files changed, 12 insertions(+), 27 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala index 9157eb9db7791..d5b7d9f855f03 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala @@ -118,7 +118,6 @@ class SVD { val m = matrix.m val n = matrix.n - println("matrix "+matrix) if (m < n || m <= 0 || n <= 0) { throw new IllegalArgumentException("Expecting a tall and skinny matrix m=$m n=$n") } @@ -130,7 +129,6 @@ class SVD { val rowIndices = matrix.rows.map(_.i) // compute SVD - println("Computing SVD") val (u, sigma, v) = denseSVD(matrix.rows.map(_.data)) if (computeU) { @@ -288,8 +286,6 @@ class SVD { // Compute A^T A, assuming rows are sparse enough to fit in memory val rows = data.map(entry => (entry.i, (entry.j, entry.mval))).groupByKey() - println("rows "+rows) - rows.foreach{e => println("r "+e)} val emits = rows.flatMap { case (rowind, colsI) => val cols = colsI.toList @@ -301,26 +297,18 @@ class SVD { } } }.reduceByKey(_ + _) - println("emits ") - emits.foreach{e => println("e "+e)} // Construct jblas A^T A locally val ata = DoubleMatrix.zeros(n, n) for (entry <- emits.collect()) { ata.put(entry._1._1, entry._1._2, entry._2) } - println("ata "+ata) // Since A^T A is small, we can compute its SVD directly val svd = Singular.sparseSVD(ata) - println("init svd") - svd.foreach{x => println(x)} - println("done") val V = svd(0) - println("V" + V) // This will be updated to rcond val sigmas = MatrixFunctions.sqrt(svd(1)).toArray.filter(x => x > 1e-9) - println("Sigmas "+sigmas) if (sigmas.size < k) { throw new Exception("Not enough singular values to return k=" + k + " s=" + sigmas.size) @@ -354,8 +342,6 @@ class SVD { // Multiply A by VS^-1 val aCols = data.map(entry => (entry.j, (entry.i, entry.mval))) val bRows = vsirdd.map(entry => (entry._1._1, (entry._1._2, entry._2))) - aCols.foreach{x => println("a "+x)} - bRows.foreach{x => println("b "+x)} val retUdata = aCols.join(bRows).map { case (key, ((rowInd, rowVal), (colInd, colVal))) => ((rowInd, colInd), rowVal * colVal) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala index 3a9811a72240b..20e2b0f84be06 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala @@ -87,17 +87,17 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll { val retu = getDenseMatrix(u) val rets = getDenseMatrix(s) val retv = getDenseMatrix(v) - - - // check individual decomposition + + + // check individual decomposition assertMatrixApproximatelyEquals(retu, svd(0)) assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1))) assertMatrixApproximatelyEquals(retv, svd(2)) // check multiplication guarantee - assertMatrixApproximatelyEquals(retu.mmul(rets).mmul(retv.transpose), denseA) + assertMatrixApproximatelyEquals(retu.mmul(rets).mmul(retv.transpose), denseA) } -/* + test("dense full rank matrix svd") { val m = 10 val n = 3 @@ -120,7 +120,7 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll { val retv = new DoubleMatrix(v) - // check individual decomposition + // check individual decomposition assertMatrixApproximatelyEquals(retu, svd(0)) assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1))) assertMatrixApproximatelyEquals(retv, svd(2)) @@ -131,7 +131,7 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll { test("rank one matrix svd") { val m = 10 - val n = 3 + val n = 3 val data = sc.makeRDD(Array.tabulate(m, n){ (a,b) => MatrixEntry(a, b, 1.0) }.flatten ) val k = 1 @@ -153,13 +153,13 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll { val rets = getDenseMatrix(s) val retv = getDenseMatrix(v) - // check individual decomposition + // check individual decomposition assertMatrixApproximatelyEquals(retu, svd(0).getColumn(0)) assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1).getRow(0))) assertMatrixApproximatelyEquals(retv, svd(2).getColumn(0)) // check multiplication guarantee - assertMatrixApproximatelyEquals(retu.mmul(rets).mmul(retv.transpose), denseA) + assertMatrixApproximatelyEquals(retu.mmul(rets).mmul(retv.transpose), denseA) } test("truncated with k") { @@ -168,7 +168,7 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll { val data = sc.makeRDD(Array.tabulate(m,n){ (a, b) => MatrixEntry(a, b, (a + 2).toDouble * (b + 1)/(1 + a + b)) }.flatten ) val a = SparseMatrix(data, m, n) - + val k = 1 // only one svalue above this val decomposed = new SVD().setK(k).compute(a) @@ -185,11 +185,10 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll { val retv = getDenseMatrix(v) assert(retrank == 1, "rank returned not one") - - // check individual decomposition + + // check individual decomposition assertMatrixApproximatelyEquals(retu, svd(0).getColumn(0)) assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1).getRow(0))) assertMatrixApproximatelyEquals(retv, svd(2).getColumn(0)) } - */ } From ec99e3204e4d315e946bd39596b558c80b8547fc Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Wed, 26 Mar 2014 16:37:22 -0700 Subject: [PATCH 19/39] Revert "Revert this but for now put things in list pandas" This reverts commit 4b088062f9fcdaf419c44dca60c1e57aae2ef194. --- .../scala/org/apache/spark/rdd/PairRDDFunctions.scala | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 609b30f87b264..1b9af45c6bdbb 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -298,9 +298,8 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) */ def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => - val vlist = vs.toList val wlist = ws.toList - for (v <- vlist; w <- wlist) yield (v, w) + for (v <- vs; w <- wlist.iterator) yield (v, w) } } @@ -315,9 +314,8 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) if (ws.isEmpty) { vs.map(v => (v, None)) } else { - val vlist = vs.toList val wlist = ws.toList - for (v <- vlist; w <- wlist) yield (v, Some(w)) + for (v <- vs; w <- wlist.iterator) yield (v, Some(w)) } } } @@ -334,9 +332,8 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) if (vs.isEmpty) { ws.map(w => (None, w)) } else { - val vlist = vs.toList val wlist = ws.toList - for (v <- vlist; w <- wlist) yield (Some(v), w) + for (v <- vs; w <- wlist) yield (Some(v), w) } } } From fa395c9274be33de08ffe7e9ae3f637e49d03bef Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Wed, 26 Mar 2014 16:39:24 -0700 Subject: [PATCH 20/39] Revert "Add a join based on the problem in SVD" This reverts commit b5e0849ad5ce7bba1f77c55da5b3164ed29f6558. --- .../org/apache/spark/rdd/PairRDDFunctionsSuite.scala | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 1787fe929865d..8f3e6bd21b752 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -157,18 +157,6 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { )) } - test("join repro") { - val rdd1 = sc.parallelize(Array((0,(0,2.0)), (1,(0,2.0)), (2,(0,2.0)), (0,(1,1.5)), (1,(1,2.0)), (2,(1,2.25)), (0,(2,1.3333333333333333)), (1,(2,2.0)), - (2,(2,2.4)), (0,(3,1.25)), (1,(3,2.0)), (2,(3,2.5)), (0,(4,1.2)), (1,(4,2.0)), (2,(4,2.5714285714285716)), - (0,(5,1.1666666666666667)), (1,(5,2.0)), (2,(5,2.625)), (0,(6,1.1428571428571428)), (1,(6,2.0)), (2,(6,2.6666666666666665)), (0,(7,1.125)), - (1,(7,2.0)), (2,(7,2.7)), (0,(8,1.1111111111111112)), (1,(8,2.0)), (2,(8,2.727272727272727)), (0,(9,1.1)), (1,(9,2.0)), (2,(9,2.75)) - )) - val rdd2 = sc.parallelize(Array((0,(0,-0.03387252842009158)), (0,(1,0.789223562443648)), (0,(2,4.541482001513894)), (1,(0,-0.05259151798283574)), (1,(1,0.10552081450086591)), (1,(2,-9.596168335700813)), (2,(0,-0.06636448259750038)), (2,(1,-0.48644238753215385)), (2,(2,5.2866468281774734)) - )) - val joined = rdd1.join(rdd2).collect() - assert(joined.toSet === Set((1,((6,2.0),(0,-0.05259151798283574))), (2,((6,2.6666666666666665),(0,-0.06636448259750038))), (1,((2,2.0),(2,-9.596168335700813))), (1,((1,2.0),(2,-9.596168335700813))), (1,((3,2.0),(2,-9.596168335700813))), (0,((3,1.25),(2,4.541482001513894))), (1,((6,2.0),(2,-9.596168335700813))), (2,((4,2.5714285714285716),(2,5.2866468281774734))), (0,((2,1.3333333333333333),(2,4.541482001513894))), (0,((4,1.2),(0,-0.03387252842009158))), (1,((8,2.0),(0,-0.05259151798283574))), (1,((4,2.0),(1,0.10552081450086591))), (0,((9,1.1),(0,-0.03387252842009158))), (0,((7,1.125),(1,0.789223562443648))), (0,((2,1.3333333333333333),(0,-0.03387252842009158))), (1,((9,2.0),(1,0.10552081450086591))), (2,((8,2.727272727272727),(0,-0.06636448259750038))), (2,((6,2.6666666666666665),(2,5.2866468281774734))), (2,((0,2.0),(0,-0.06636448259750038))), (1,((3,2.0),(0,-0.05259151798283574))), (2,((4,2.5714285714285716),(0,-0.06636448259750038))), (1,((4,2.0),(0,-0.05259151798283574))), (0,((6,1.1428571428571428),(2,4.541482001513894))), (0,((7,1.125),(2,4.541482001513894))), (2,((0,2.0),(2,5.2866468281774734))), (1,((0,2.0),(0,-0.05259151798283574))), (2,((3,2.5),(2,5.2866468281774734))), (1,((7,2.0),(2,-9.596168335700813))), (1,((4,2.0),(2,-9.596168335700813))), (0,((4,1.2),(2,4.541482001513894))), (2,((7,2.7),(0,-0.06636448259750038))), (1,((7,2.0),(1,0.10552081450086591))), (2,((8,2.727272727272727),(1,-0.48644238753215385))), (2,((1,2.25),(2,5.2866468281774734))), (0,((8,1.1111111111111112),(0,-0.03387252842009158))), (0,((3,1.25),(0,-0.03387252842009158))), (1,((3,2.0),(1,0.10552081450086591))), (2,((0,2.0),(1,-0.48644238753215385))), (2,((5,2.625),(0,-0.06636448259750038))), (1,((5,2.0),(0,-0.05259151798283574))), (2,((1,2.25),(0,-0.06636448259750038))), (2,((2,2.4),(1,-0.48644238753215385))), (2,((5,2.625),(1,-0.48644238753215385))), (2,((7,2.7),(2,5.2866468281774734))), (0,((6,1.1428571428571428),(0,-0.03387252842009158))), (0,((8,1.1111111111111112),(2,4.541482001513894))), (0,((1,1.5),(2,4.541482001513894))), (0,((5,1.1666666666666667),(2,4.541482001513894))), (2,((3,2.5),(0,-0.06636448259750038))), (1,((2,2.0),(0,-0.05259151798283574))), (2,((7,2.7),(1,-0.48644238753215385))), (0,((2,1.3333333333333333),(1,0.789223562443648))), (2,((9,2.75),(2,5.2866468281774734))), (0,((5,1.1666666666666667),(1,0.789223562443648))), (0,((0,2.0),(0,-0.03387252842009158))), (1,((8,2.0),(2,-9.596168335700813))), (0,((1,1.5),(0,-0.03387252842009158))), (2,((9,2.75),(1,-0.48644238753215385))), (2,((4,2.5714285714285716),(1,-0.48644238753215385))), (0,((0,2.0),(2,4.541482001513894))), (0,((8,1.1111111111111112),(1,0.789223562443648))), (0,((9,1.1),(2,4.541482001513894))), (0,((0,2.0),(1,0.789223562443648))), (1,((1,2.0),(1,0.10552081450086591))), (2,((6,2.6666666666666665),(1,-0.48644238753215385))), (0,((6,1.1428571428571428),(1,0.789223562443648))), (2,((9,2.75),(0,-0.06636448259750038))), (1,((9,2.0),(2,-9.596168335700813))), (2,((8,2.727272727272727),(2,5.2866468281774734))), (0,((4,1.2),(1,0.789223562443648))), (1,((5,2.0),(1,0.10552081450086591))), (1,((6,2.0),(1,0.10552081450086591))), (1,((5,2.0),(2,-9.596168335700813))), (1,((0,2.0),(2,-9.596168335700813))), (2,((5,2.625),(2,5.2866468281774734))), (1,((7,2.0),(0,-0.05259151798283574))), (0,((5,1.1666666666666667),(0,-0.03387252842009158))), (1,((1,2.0),(0,-0.05259151798283574))), (0,((9,1.1),(1,0.789223562443648))), (1,((9,2.0),(0,-0.05259151798283574))), (0,((7,1.125),(0,-0.03387252842009158))), (0,((3,1.25),(1,0.789223562443648))), (2,((3,2.5),(1,-0.48644238753215385))), (1,((2,2.0),(1,0.10552081450086591))), (1,((8,2.0),(1,0.10552081450086591))), (1,((0,2.0),(1,0.10552081450086591))), (2,((2,2.4),(0,-0.06636448259750038))), (2,((1,2.25),(1,-0.48644238753215385))), (0,((1,1.5),(1,0.789223562443648))), (2,((2,2.4),(2,5.2866468281774734))))) - } - test("join all-to-all") { val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (1, 3))) val rdd2 = sc.parallelize(Array((1, 'x'), (1, 'y'))) From 4b0eeb9a6ffdae53db56e99f72057fd3f36740ee Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Wed, 26 Mar 2014 16:50:07 -0700 Subject: [PATCH 21/39] Switch cast in PairDStreamFunctions --- .../apache/spark/streaming/dstream/PairDStreamFunctions.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala index c5d9b059c1410..fa269da89b905 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala @@ -72,7 +72,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) val mergeValue = (c: ArrayBuffer[V], v: V) => (c += v) val mergeCombiner = (c1: ArrayBuffer[V], c2: ArrayBuffer[V]) => (c1 ++ c2) combineByKey(createCombiner, mergeValue, mergeCombiner, partitioner) - .asInstanceOf[DStream[(K, Iterator[V])]] + .asInstanceOf[DStream[(K, Seq[V])]].mapValues(_.toIterator) } /** @@ -188,7 +188,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) self.groupByKey(partitioner) .window(windowDuration, slideDuration) .combineByKey[ArrayBuffer[V]](createCombiner, mergeValue, mergeCombiner, partitioner) - .asInstanceOf[DStream[(K, Iterator[V])]] + .asInstanceOf[DStream[(K, Seq[V])]].mapValues(_.toIterator) } /** From ec8cc3ee3b728d35de0a73ffa5c839e880136ef1 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Wed, 26 Mar 2014 19:33:18 -0700 Subject: [PATCH 22/39] Fix test issues\! --- .../spark/streaming/BasicOperationsSuite.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala index 270ceb081480f..8aec27e39478a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -117,8 +117,8 @@ class BasicOperationsSuite extends TestSuiteBase { test("groupByKey") { testOperation( Seq( Seq("a", "a", "b"), Seq("", ""), Seq() ), - (s: DStream[String]) => s.map(x => (x, 1)).groupByKey(), - Seq( Seq(("a", Seq(1, 1).toIterator), ("b", Seq(1).toIterator)), Seq(("", Seq(1, 1).toIterator)), Seq() ), + (s: DStream[String]) => s.map(x => (x, 1)).groupByKey().mapValues(_.toSeq), + Seq( Seq(("a", Seq(1, 1)), ("b", Seq(1))), Seq(("", Seq(1, 1))), Seq() ), true ) } @@ -245,13 +245,13 @@ class BasicOperationsSuite extends TestSuiteBase { val inputData1 = Seq( Seq("a", "a", "b"), Seq("a", ""), Seq(""), Seq() ) val inputData2 = Seq( Seq("a", "a", "b"), Seq("b", ""), Seq(), Seq() ) val outputData = Seq( - Seq( ("a", (Seq(1, 1).toIterator, Seq("x", "x").toIterator)), ("b", (Seq(1).toIterator, Seq("x").toIterator)) ), - Seq( ("a", (Seq(1).toIterator, Seq().toIterator)), ("b", (Seq().toIterator, Seq("x").toIterator)), ("", (Seq(1).toIterator, Seq("x").toIterator)) ), - Seq( ("", (Seq(1).toIterator, Seq().toIterator)) ), + Seq( ("a", (Seq(1, 1), Seq("x", "x"))), ("b", (Seq(1), Seq("x"))) ), + Seq( ("a", (Seq(1), Seq())), ("b", (Seq(), Seq("x"))), ("", (Seq(1), Seq("x"))) ), + Seq( ("", (Seq(1), Seq())) ), Seq( ) ) val operation = (s1: DStream[String], s2: DStream[String]) => { - s1.map(x => (x,1)).cogroup(s2.map(x => (x, "x"))) + s1.map(x => (x,1)).cogroup(s2.map(x => (x, "x"))).mapValues(x => (x._1.toSeq, x._2.toSeq)) } testOperation(inputData1, inputData2, operation, outputData, true) } From e687f21866dd7a1fc84697af9514acd19694d499 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 27 Mar 2014 13:44:25 -0700 Subject: [PATCH 23/39] Fix groupbykey test in JavaAPISuite of streaming --- .../apache/spark/streaming/JavaAPISuite.java | 26 ++++++++++++++++--- 1 file changed, 23 insertions(+), 3 deletions(-) diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index bfda617d60a08..2a1becc74cbc7 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -45,6 +45,13 @@ // see http://stackoverflow.com/questions/758570/. public class JavaAPISuite extends LocalJavaStreamingContext implements Serializable { + public void equalIterator(Iterator a, Iterator b) { + while (a.hasNext() && b.hasNext()) { + Assert.assertEquals(a.next(), b.next()); + } + Assert.assertEquals(a.hasNext(), b.hasNext()); + } + @SuppressWarnings("unchecked") @Test public void testCount() { @@ -1018,9 +1025,22 @@ public void testPairGroupByKey() { JavaPairDStream> grouped = pairStream.groupByKey(); JavaTestUtils.attachTestOutputStream(grouped); - List>>> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); + List>>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected.size(), result.size()); + Iterator>>> resultItr = result.iterator(); + Iterator>>> expectedItr = expected.iterator(); + while (resultItr.hasNext() && expectedItr.hasNext()) { + Iterator>> resultElements = resultItr.next().iterator(); + Iterator>> expectedElements = expectedItr.next().iterator(); + while (resultElements.hasNext() && expectedElements.hasNext()) { + Tuple2> resultElement = resultElements.next(); + Tuple2> expectedElement = expectedElements.next(); + Assert.assertEquals(expectedElement._1(), resultElement._1()); + equalIterator(expectedElement._2().iterator(), resultElement._2()); + } + Assert.assertEquals(resultElements.hasNext(), expectedItr.hasNext()); + } } @SuppressWarnings("unchecked") From a5ee7141d9d5de150feb34bc99904d900960549f Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 27 Mar 2014 13:47:34 -0700 Subject: [PATCH 24/39] oops, was checking wrong iterator --- .../src/test/java/org/apache/spark/streaming/JavaAPISuite.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index 2a1becc74cbc7..c0e125e3af88b 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -1039,7 +1039,7 @@ public void testPairGroupByKey() { Assert.assertEquals(expectedElement._1(), resultElement._1()); equalIterator(expectedElement._2().iterator(), resultElement._2()); } - Assert.assertEquals(resultElements.hasNext(), expectedItr.hasNext()); + Assert.assertEquals(resultElements.hasNext(), expectedElements.hasNext()); } } From 88a5cefb61f06ea1fe0169c96bbef4ba5c548a4d Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 27 Mar 2014 14:39:42 -0700 Subject: [PATCH 25/39] Fix cogroup test in JavaAPISuite for streaming --- .../apache/spark/streaming/JavaAPISuite.java | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index c0e125e3af88b..0644202de3434 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -1493,9 +1493,23 @@ public void testCoGroup() { JavaPairDStream, Iterator>> grouped = pairStream1.cogroup(pairStream2); JavaTestUtils.attachTestOutputStream(grouped); - List, List>>>> result = JavaTestUtils.runStreams(ssc, 2, 2); + List, Iterator>>>> result = JavaTestUtils.runStreams(ssc, 2, 2); - Assert.assertEquals(expected, result); + Assert.assertEquals(expected.size(), result.size()); + Iterator, Iterator>>>> resultItr = result.iterator(); + Iterator, List>>>> expectedItr = expected.iterator(); + while (resultItr.hasNext() && expectedItr.hasNext()) { + Iterator, Iterator>>> resultElements = resultItr.next().iterator(); + Iterator, List>>> expectedElements = expectedItr.next().iterator(); + while (resultElements.hasNext() && expectedElements.hasNext()) { + Tuple2, Iterator>> resultElement = resultElements.next(); + Tuple2, List>> expectedElement = expectedElements.next(); + Assert.assertEquals(expectedElement._1(), resultElement._1()); + equalIterator(expectedElement._2()._1().iterator(), resultElement._2()._1()); + equalIterator(expectedElement._2()._2().iterator(), resultElement._2()._2()); + } + Assert.assertEquals(resultElements.hasNext(), expectedElements.hasNext()); + } } @SuppressWarnings("unchecked") From c60233aaf0c7dcde64ea122f7bcffa38b44e4b5c Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 27 Mar 2014 16:14:11 -0700 Subject: [PATCH 26/39] Start investigating moving to iterators for python API like the Java/Scala one. tl;dr: We will have to write our own iterator since the default one doesn't pickle well --- python/pyspark/join.py | 2 +- python/pyspark/rdd.py | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/python/pyspark/join.py b/python/pyspark/join.py index 5f4294fb1b777..9feb4362dc469 100644 --- a/python/pyspark/join.py +++ b/python/pyspark/join.py @@ -88,5 +88,5 @@ def dispatch(seq): vbuf.append(v) elif n == 2: wbuf.append(v) - return (vbuf, wbuf) + return (iter(vbuf), iter(wbuf)) return vs.union(ws).groupByKey(numPartitions).mapValues(dispatch) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index fb27863e07f55..99693a4bff75f 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1133,7 +1133,7 @@ def mergeCombiners(a, b): return a + b return self.combineByKey(createCombiner, mergeValue, mergeCombiners, - numPartitions) + numPartitions).mapValues(lambda x: iter(x)) # TODO: add tests def flatMapValues(self, f): @@ -1180,7 +1180,7 @@ def cogroup(self, other, numPartitions=None): >>> x = sc.parallelize([("a", 1), ("b", 4)]) >>> y = sc.parallelize([("a", 2)]) - >>> sorted(x.cogroup(y).collect()) + >>> sorted(list(x.cogroup(y).collect())) [('a', ([1], [2])), ('b', ([4], []))] """ return python_cogroup(self, other, numPartitions) From cd1e81cc242583ec3b353db9440e1593ee64771e Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 27 Mar 2014 19:53:43 -0700 Subject: [PATCH 27/39] Try and make pickling list iterators work --- python/pyspark/cloudpickle.py | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/python/pyspark/cloudpickle.py b/python/pyspark/cloudpickle.py index 6a7c23a069bf8..e24e9e962393d 100644 --- a/python/pyspark/cloudpickle.py +++ b/python/pyspark/cloudpickle.py @@ -204,6 +204,15 @@ def save_codeobject(self, obj, pack=struct.pack): self.save_reduce(types.CodeType, args, obj=obj) dispatch[types.CodeType] = save_codeobject #new type + def save_listiterator(self, obj, pack=struct.pack): + """ + Save a list iterator. Note: this reads the iterator... Sorry! + """ + pickle.PicklingError("mini sad panda") + contents = list(obj) + self.save_reduce(load_listr_itr, contents, obj=obj) + dispatch[type(iter(list()))] = save_listiterator + def save_function(self, obj, name=None, pack=struct.pack): """ Registered with the dispatch to handle all function types. @@ -836,6 +845,10 @@ def _restore_attr(obj, attr): setattr(obj, key, val) return obj +#hack to load the list iterator +def load_list_tr(args): + itr(args) + def _get_module_builtins(): return pickle.__builtins__ From 3bcd81dcb93d53def9764f671aaafb5897362fc5 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 27 Mar 2014 19:53:53 -0700 Subject: [PATCH 28/39] Revert "Try and make pickling list iterators work" This reverts commit f607138dfea8f04d7b302f49b76dba8756b17575. --- python/pyspark/cloudpickle.py | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/python/pyspark/cloudpickle.py b/python/pyspark/cloudpickle.py index e24e9e962393d..6a7c23a069bf8 100644 --- a/python/pyspark/cloudpickle.py +++ b/python/pyspark/cloudpickle.py @@ -204,15 +204,6 @@ def save_codeobject(self, obj, pack=struct.pack): self.save_reduce(types.CodeType, args, obj=obj) dispatch[types.CodeType] = save_codeobject #new type - def save_listiterator(self, obj, pack=struct.pack): - """ - Save a list iterator. Note: this reads the iterator... Sorry! - """ - pickle.PicklingError("mini sad panda") - contents = list(obj) - self.save_reduce(load_listr_itr, contents, obj=obj) - dispatch[type(iter(list()))] = save_listiterator - def save_function(self, obj, name=None, pack=struct.pack): """ Registered with the dispatch to handle all function types. @@ -845,10 +836,6 @@ def _restore_attr(obj, attr): setattr(obj, key, val) return obj -#hack to load the list iterator -def load_list_tr(args): - itr(args) - def _get_module_builtins(): return pickle.__builtins__ From d052c077360ee35af1dc7f395b1ccddae67ab5ef Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 27 Mar 2014 20:58:08 -0700 Subject: [PATCH 29/39] Python tests now pass with iterator pandas --- python/pyspark/join.py | 3 ++- python/pyspark/rdd.py | 10 +++++----- python/pyspark/resultitr.py | 39 +++++++++++++++++++++++++++++++++++++ 3 files changed, 46 insertions(+), 6 deletions(-) create mode 100644 python/pyspark/resultitr.py diff --git a/python/pyspark/join.py b/python/pyspark/join.py index 9feb4362dc469..febc223a645a4 100644 --- a/python/pyspark/join.py +++ b/python/pyspark/join.py @@ -31,6 +31,7 @@ OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. """ +from pyspark.resultitr import ResultItr def _do_python_join(rdd, other, numPartitions, dispatch): vs = rdd.map(lambda (k, v): (k, (1, v))) @@ -88,5 +89,5 @@ def dispatch(seq): vbuf.append(v) elif n == 2: wbuf.append(v) - return (iter(vbuf), iter(wbuf)) + return (ResultItr(vbuf), ResultItr(wbuf)) return vs.union(ws).groupByKey(numPartitions).mapValues(dispatch) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 99693a4bff75f..feb00731af8e9 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -38,6 +38,7 @@ from pyspark.statcounter import StatCounter from pyspark.rddsampler import RDDSampler from pyspark.storagelevel import StorageLevel +from pyspark.resultitr import ResultItr from py4j.java_collections import ListConverter, MapConverter @@ -1118,7 +1119,7 @@ def groupByKey(self, numPartitions=None): Hash-partitions the resulting RDD with into numPartitions partitions. >>> x = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) - >>> sorted(x.groupByKey().collect()) + >>> map((lambda (x,y): (x, list(y))), sorted(x.groupByKey().collect())) [('a', [1, 1]), ('b', [1])] """ @@ -1133,7 +1134,7 @@ def mergeCombiners(a, b): return a + b return self.combineByKey(createCombiner, mergeValue, mergeCombiners, - numPartitions).mapValues(lambda x: iter(x)) + numPartitions).mapValues(lambda x: ResultItr(x)) # TODO: add tests def flatMapValues(self, f): @@ -1180,7 +1181,7 @@ def cogroup(self, other, numPartitions=None): >>> x = sc.parallelize([("a", 1), ("b", 4)]) >>> y = sc.parallelize([("a", 2)]) - >>> sorted(list(x.cogroup(y).collect())) + >>> map((lambda (x,y): (x, (list(y[0]), list(y[1])))), sorted(list(x.cogroup(y).collect()))) [('a', ([1], [2])), ('b', ([4], []))] """ return python_cogroup(self, other, numPartitions) @@ -1217,7 +1218,7 @@ def keyBy(self, f): >>> x = sc.parallelize(range(0,3)).keyBy(lambda x: x*x) >>> y = sc.parallelize(zip(range(0,5), range(0,5))) - >>> sorted(x.cogroup(y).collect()) + >>> map((lambda (x,y): (x, (list(y[0]), (list(y[1]))))), sorted(x.cogroup(y).collect())) [(0, ([0], [0])), (1, ([1], [1])), (2, ([], [2])), (3, ([], [3])), (4, ([2], [4]))] """ return self.map(lambda x: (f(x), x)) @@ -1317,7 +1318,6 @@ def getStorageLevel(self): # keys in the pairs. This could be an expensive operation, since those # hashes aren't retained. - class PipelinedRDD(RDD): """ Pipelined maps: diff --git a/python/pyspark/resultitr.py b/python/pyspark/resultitr.py new file mode 100644 index 0000000000000..c0a07f64ecb78 --- /dev/null +++ b/python/pyspark/resultitr.py @@ -0,0 +1,39 @@ +# +# 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. +# + +__all__ = ["ResultItr"] + +import collections + +class ResultItr(collections.Iterator): + """ + A special result iterator. This is used because the standard iterator can not be pickled + """ + def __init__(self, data): + self.data = data + self.index = 0 + self.maxindex = len(data) + def next(self): + if index == maxindex: + raise StopIteration + v = self.data[0] + self.data = data[1:] + return v + def __iter__(self): + return iter(self.data) + def __len__(self): + return len(self.data) From 4ed579bd10624d3a76b54449aa5b14b566154f67 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Mon, 7 Apr 2014 12:53:49 -0700 Subject: [PATCH 30/39] Refactor from iterator to iterable --- .../scala/org/apache/spark/bagel/Bagel.scala | 5 ++- .../apache/spark/api/java/JavaPairRDD.scala | 39 ++++++++-------- .../apache/spark/api/java/JavaRDDLike.scala | 6 +-- .../apache/spark/rdd/PairRDDFunctions.scala | 32 +++++++------- .../main/scala/org/apache/spark/rdd/RDD.scala | 6 +-- .../apache/spark/examples/JavaPageRank.java | 22 +++++----- .../bagel/WikipediaPageRankStandalone.scala | 4 +- .../spark/mllib/recommendation/ALS.scala | 2 +- .../org/apache/spark/mllib/util/LAUtils.scala | 5 ++- python/pyspark/join.py | 4 +- python/pyspark/resultitr.py | 10 +---- .../streaming/api/java/JavaPairDStream.scala | 44 +++++++++---------- .../dstream/PairDStreamFunctions.scala | 28 ++++++------ .../streaming/dstream/StateDStream.scala | 9 ++-- 14 files changed, 109 insertions(+), 107 deletions(-) diff --git a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala index a389d5077bd47..70a99b33d753c 100644 --- a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala +++ b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala @@ -220,13 +220,14 @@ object Bagel extends Logging { */ private def comp[K: Manifest, V <: Vertex, M <: Message[K], C]( sc: SparkContext, - grouped: RDD[(K, (Iterator[C], Iterator[V]))], + grouped: RDD[(K, (Iterable[C], Iterable[V]))], compute: (V, Option[C]) => (V, Array[M]), storageLevel: StorageLevel ): (RDD[(K, (V, Array[M]))], Int, Int) = { var numMsgs = sc.accumulator(0) var numActiveVerts = sc.accumulator(0) - val processed = grouped.flatMapValues { + val processed = grouped.mapValues(x => (x._1.iterator, x._2.iterator)) + .flatMapValues { case (_, vs) if !vs.hasNext => None case (c, vs) => { val (newVert, newMsgs) = diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index fd82230f122c8..e6c5d85917678 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -17,7 +17,8 @@ package org.apache.spark.api.java -import java.util.{Comparator, List => JList, Iterator => JIterator} +import java.util.{Comparator, List => JList} +import java.lang.{Iterable => JIterable} import scala.collection.JavaConversions._ import scala.reflect.ClassTag @@ -250,14 +251,14 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * Group the values for each key in the RDD into a single sequence. Allows controlling the * partitioning of the resulting key-value pair RDD by passing a Partitioner. */ - def groupByKey(partitioner: Partitioner): JavaPairRDD[K, JIterator[V]] = + def groupByKey(partitioner: Partitioner): JavaPairRDD[K, JIterable[V]] = fromRDD(groupByResultToJava(rdd.groupByKey(partitioner))) /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with into `numPartitions` partitions. */ - def groupByKey(numPartitions: Int): JavaPairRDD[K, JIterator[V]] = + def groupByKey(numPartitions: Int): JavaPairRDD[K, JIterable[V]] = fromRDD(groupByResultToJava(rdd.groupByKey(numPartitions))) /** @@ -367,7 +368,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with the existing partitioner/parallelism level. */ - def groupByKey(): JavaPairRDD[K, JIterator[V]] = + def groupByKey(): JavaPairRDD[K, JIterable[V]] = fromRDD(groupByResultToJava(rdd.groupByKey())) /** @@ -462,7 +463,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * list of values for that key in `this` as well as `other`. */ def cogroup[W](other: JavaPairRDD[K, W], partitioner: Partitioner) - : JavaPairRDD[K, (JIterator[V], JIterator[W])] = + : JavaPairRDD[K, (JIterable[V], JIterable[W])] = fromRDD(cogroupResultToJava(rdd.cogroup(other, partitioner))) /** @@ -470,14 +471,14 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], - partitioner: Partitioner): JavaPairRDD[K, (JIterator[V], JIterator[W1], JIterator[W2])] = + partitioner: Partitioner): JavaPairRDD[K, (JIterable[V], JIterable[W1], JIterable[W2])] = fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, partitioner))) /** * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JIterator[V], JIterator[W])] = + def cogroup[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JIterable[V], JIterable[W])] = fromRDD(cogroupResultToJava(rdd.cogroup(other))) /** @@ -485,7 +486,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2]) - : JavaPairRDD[K, (JIterator[V], JIterator[W1], JIterator[W2])] = + : JavaPairRDD[K, (JIterable[V], JIterable[W1], JIterable[W2])] = fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2))) /** @@ -493,7 +494,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * list of values for that key in `this` as well as `other`. */ def cogroup[W](other: JavaPairRDD[K, W], numPartitions: Int) - : JavaPairRDD[K, (JIterator[V], JIterator[W])] = + : JavaPairRDD[K, (JIterable[V], JIterable[W])] = fromRDD(cogroupResultToJava(rdd.cogroup(other, numPartitions))) /** @@ -501,16 +502,16 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], numPartitions: Int) - : JavaPairRDD[K, (JIterator[V], JIterator[W1], JIterator[W2])] = + : JavaPairRDD[K, (JIterable[V], JIterable[W1], JIterable[W2])] = fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, numPartitions))) /** Alias for cogroup. */ - def groupWith[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JIterator[V], JIterator[W])] = + def groupWith[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JIterable[V], JIterable[W])] = fromRDD(cogroupResultToJava(rdd.groupWith(other))) /** Alias for cogroup. */ def groupWith[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2]) - : JavaPairRDD[K, (JIterator[V], JIterator[W1], JIterator[W2])] = + : JavaPairRDD[K, (JIterable[V], JIterable[W1], JIterable[W2])] = fromRDD(cogroupResult2ToJava(rdd.groupWith(other1, other2))) /** @@ -695,22 +696,22 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) object JavaPairRDD { private[spark] - def groupByResultToJava[K: ClassTag, T](rdd: RDD[(K, Iterator[T])]): RDD[(K, JIterator[T])] = { - rddToPairRDDFunctions(rdd).mapValues(asJavaIterator) + def groupByResultToJava[K: ClassTag, T](rdd: RDD[(K, Iterable[T])]): RDD[(K, JIterable[T])] = { + rddToPairRDDFunctions(rdd).mapValues(asJavaIterable) } private[spark] def cogroupResultToJava[K: ClassTag, V, W]( - rdd: RDD[(K, (Iterator[V], Iterator[W]))]): RDD[(K, (JIterator[V], JIterator[W]))] = { - rddToPairRDDFunctions(rdd).mapValues(x => (asJavaIterator(x._1), asJavaIterator(x._2))) + rdd: RDD[(K, (Iterable[V], Iterable[W]))]): RDD[(K, (JIterable[V], JIterable[W]))] = { + rddToPairRDDFunctions(rdd).mapValues(x => (asJavaIterable(x._1), asJavaIterable(x._2))) } private[spark] def cogroupResult2ToJava[K: ClassTag, V, W1, W2]( - rdd: RDD[(K, (Iterator[V], Iterator[W1], Iterator[W2]))]) - : RDD[(K, (JIterator[V], JIterator[W1], JIterator[W2]))] = { + rdd: RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))]) + : RDD[(K, (JIterable[V], JIterable[W1], JIterable[W2]))] = { rddToPairRDDFunctions(rdd) - .mapValues(x => (asJavaIterator(x._1), asJavaIterator(x._2), asJavaIterator(x._3))) + .mapValues(x => (asJavaIterable(x._1), asJavaIterable(x._2), asJavaIterable(x._3))) } def fromRDD[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]): JavaPairRDD[K, V] = { 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 fd74e8218f925..383aeda8a5738 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 @@ -17,7 +17,7 @@ package org.apache.spark.api.java -import java.util.{Comparator, Iterator => JIterator, List => JList} +import java.util.{Comparator, List => JList} import java.lang.{Iterable => JIterable} import scala.collection.JavaConversions._ @@ -204,7 +204,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ - def groupBy[K](f: JFunction[T, K]): JavaPairRDD[K, JIterator[T]] = { + def groupBy[K](f: JFunction[T, K]): JavaPairRDD[K, JIterable[T]] = { implicit val ctagK: ClassTag[K] = fakeClassTag implicit val ctagV: ClassTag[JList[T]] = fakeClassTag JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f)(fakeClassTag))) @@ -214,7 +214,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ - def groupBy[K](f: JFunction[T, K], numPartitions: Int): JavaPairRDD[K, JIterator[T]] = { + def groupBy[K](f: JFunction[T, K], numPartitions: Int): JavaPairRDD[K, JIterable[T]] = { implicit val ctagK: ClassTag[K] = fakeClassTag implicit val ctagV: ClassTag[JList[T]] = fakeClassTag JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f, numPartitions)(fakeClassTag[K]))) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 1b9af45c6bdbb..f2127b5384e94 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -261,7 +261,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * Group the values for each key in the RDD into a single sequence. Allows controlling the * partitioning of the resulting key-value pair RDD by passing a Partitioner. */ - def groupByKey(partitioner: Partitioner): RDD[(K, Iterator[V])] = { + def groupByKey(partitioner: Partitioner): RDD[(K, Iterable[V])] = { // groupByKey shouldn't use map side combine because map side combine does not // reduce the amount of data shuffled and requires all map side data be inserted // into a hash table, leading to more objects in the old gen. @@ -270,14 +270,14 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) def mergeCombiners(c1: ArrayBuffer[V], c2: ArrayBuffer[V]) = c1 ++ c2 val bufs = combineByKey[ArrayBuffer[V]]( createCombiner _, mergeValue _, mergeCombiners _, partitioner, mapSideCombine=false) - bufs.mapValues(_.toIterator) + bufs.mapValues(_.toIterable) } /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with into `numPartitions` partitions. */ - def groupByKey(numPartitions: Int): RDD[(K, Iterator[V])] = { + def groupByKey(numPartitions: Int): RDD[(K, Iterable[V])] = { groupByKey(new HashPartitioner(numPartitions)) } @@ -361,7 +361,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with the existing partitioner/parallelism level. */ - def groupByKey(): RDD[(K, Iterator[V])] = { + def groupByKey(): RDD[(K, Iterable[V])] = { groupByKey(defaultPartitioner(self)) } @@ -457,13 +457,13 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * list of values for that key in `this` as well as `other`. */ def cogroup[W](other: RDD[(K, W)], partitioner: Partitioner) - : RDD[(K, (Iterator[V], Iterator[W]))] = { + : RDD[(K, (Iterable[V], Iterable[W]))] = { if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { throw new SparkException("Default partitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K](Seq(self, other), partitioner) cg.mapValues { case Seq(vs, ws) => - (vs.asInstanceOf[Seq[V]].iterator, ws.asInstanceOf[Seq[W]].iterator) + (vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]]) } } @@ -472,15 +472,15 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], partitioner: Partitioner) - : RDD[(K, (Iterator[V], Iterator[W1], Iterator[W2]))] = { + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = { if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { throw new SparkException("Default partitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K](Seq(self, other1, other2), partitioner) cg.mapValues { case Seq(vs, w1s, w2s) => - (vs.asInstanceOf[Seq[V]].iterator, - w1s.asInstanceOf[Seq[W1]].iterator, - w2s.asInstanceOf[Seq[W2]].iterator) + (vs.asInstanceOf[Seq[V]], + w1s.asInstanceOf[Seq[W1]], + w2s.asInstanceOf[Seq[W2]]) } } @@ -488,7 +488,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: RDD[(K, W)]): RDD[(K, (Iterator[V], Iterator[W]))] = { + def cogroup[W](other: RDD[(K, W)]): RDD[(K, (Iterable[V], Iterable[W]))] = { cogroup(other, defaultPartitioner(self, other)) } @@ -497,7 +497,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) - : RDD[(K, (Iterator[V], Iterator[W1], Iterator[W2]))] = { + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = { cogroup(other1, other2, defaultPartitioner(self, other1, other2)) } @@ -505,7 +505,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Iterator[V], Iterator[W]))] = { + def cogroup[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Iterable[V], Iterable[W]))] = { cogroup(other, new HashPartitioner(numPartitions)) } @@ -514,18 +514,18 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numPartitions: Int) - : RDD[(K, (Iterator[V], Iterator[W1], Iterator[W2]))] = { + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = { cogroup(other1, other2, new HashPartitioner(numPartitions)) } /** Alias for cogroup. */ - def groupWith[W](other: RDD[(K, W)]): RDD[(K, (Iterator[V], Iterator[W]))] = { + def groupWith[W](other: RDD[(K, W)]): RDD[(K, (Iterable[V], Iterable[W]))] = { cogroup(other, defaultPartitioner(self, other)) } /** Alias for cogroup. */ def groupWith[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) - : RDD[(K, (Iterator[V], Iterator[W1], Iterator[W2]))] = { + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = { cogroup(other1, other2, defaultPartitioner(self, other1, other2)) } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index a702ef4348268..74fa2a4fcd401 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -438,20 +438,20 @@ abstract class RDD[T: ClassTag]( /** * Return an RDD of grouped items. */ - def groupBy[K: ClassTag](f: T => K): RDD[(K, Iterator[T])] = + def groupBy[K: ClassTag](f: T => K): RDD[(K, Iterable[T])] = groupBy[K](f, defaultPartitioner(this)) /** * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ - def groupBy[K: ClassTag](f: T => K, numPartitions: Int): RDD[(K, Iterator[T])] = + def groupBy[K: ClassTag](f: T => K, numPartitions: Int): RDD[(K, Iterable[T])] = groupBy(f, new HashPartitioner(numPartitions)) /** * Return an RDD of grouped items. */ - def groupBy[K: ClassTag](f: T => K, p: Partitioner): RDD[(K, Iterator[T])] = { + def groupBy[K: ClassTag](f: T => K, p: Partitioner): RDD[(K, Iterable[T])] = { val cleanF = sc.clean(f) this.map(t => (cleanF(t), t)).groupByKey(p) } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java index 7352160a55fec..cffc747b3c286 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -67,7 +67,7 @@ public static void main(String[] args) throws Exception { JavaRDD lines = ctx.textFile(args[1], 1); // Loads all URLs from input file and initialize their neighbors. - JavaPairRDD> links = lines.mapToPair(new PairFunction() { + JavaPairRDD> links = lines.mapToPair(new PairFunction() { @Override public Tuple2 call(String s) { String[] parts = SPACES.split(s); @@ -76,9 +76,9 @@ public Tuple2 call(String s) { }).distinct().groupByKey().cache(); // Loads all URLs with other URL(s) link to from input file and initialize ranks of them to one. - JavaPairRDD ranks = links.mapValues(new Function, Double>() { + JavaPairRDD ranks = links.mapValues(new Function, Double>() { @Override - public Double call(Iterator rs) { + public Double call(Iterable rs) { return 1.0; } }); @@ -87,16 +87,18 @@ public Double call(Iterator rs) { for (int current = 0; current < Integer.parseInt(args[2]); current++) { // Calculates URL contributions to the rank of other URLs. JavaPairRDD contribs = links.join(ranks).values() - .flatMapToPair(new PairFlatMapFunction, Double>, String, Double>() { + .flatMapToPair(new PairFlatMapFunction, Double>, String, Double>() { @Override - public Iterable> call(Tuple2, Double> s) { - List urls = new ArrayList(); - while (s._1.hasNext()) { - urls.add(s._1.next()); + public Iterable> call(Tuple2, Double> s) { + int urlCount = 0; + Iterator urls = s._1.iterator(); + while (urls.hasNext()) { + urls.next(); + urlCount++; } List> results = new ArrayList>(); - for (String n : urls) { - results.add(new Tuple2(n, s._2() / urls.size())); + for (String n : s._1) { + results.add(new Tuple2(n, s._2() / urlCount)); } return results; } diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala index 698c7cb2e85d1..7aac6a13597e6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala @@ -115,7 +115,9 @@ object WikipediaPageRankStandalone { var ranks = links.mapValues { edges => defaultRank } for (i <- 1 to numIterations) { val contribs = links.groupWith(ranks).flatMap { - case (id, (linksWrapper, rankWrapper)) => + case (id, (linksWrapperIterable, rankWrapperIterable)) => + val linksWrapper = linksWrapperIterable.iterator + val rankWrapper = rankWrapperIterable.iterator if (linksWrapper.hasNext) { val linksWrapperHead = linksWrapper.next if (rankWrapper.hasNext) { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index fc9a74ffbef86..08894e709907d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -421,7 +421,7 @@ class ALS private ( * Compute the new feature vectors for a block of the users matrix given the list of factors * it received from each product and its InLinkBlock. */ - private def updateBlock(messages: Iterator[(Int, Array[Array[Double]])], inLinkBlock: InLinkBlock, + private def updateBlock(messages: Iterable[(Int, Array[Array[Double]])], inLinkBlock: InLinkBlock, rank: Int, lambda: Double, alpha: Double, YtY: Option[Broadcast[DoubleMatrix]]) : Array[Array[Double]] = { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala index 8445afef12357..87aac347579c7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala @@ -38,8 +38,9 @@ object LAUtils { case (i, cols) => val rowArray = Array.ofDim[Double](n) var j = 0 - while (cols.hasNext) { - val element = cols.next + val colsItr = cols.iterator + while (colsItr.hasNext) { + val element = colsItr.next rowArray(element._1) = element._2 j += 1 } diff --git a/python/pyspark/join.py b/python/pyspark/join.py index febc223a645a4..df2a75154178b 100644 --- a/python/pyspark/join.py +++ b/python/pyspark/join.py @@ -36,7 +36,7 @@ def _do_python_join(rdd, other, numPartitions, dispatch): vs = rdd.map(lambda (k, v): (k, (1, v))) ws = other.map(lambda (k, v): (k, (2, v))) - return vs.union(ws).groupByKey(numPartitions).flatMapValues(dispatch) + return vs.union(ws).groupByKey(numPartitions).flatMapValues(lambda x : dispatch(x.__iter__())) def python_join(rdd, other, numPartitions): @@ -90,4 +90,4 @@ def dispatch(seq): elif n == 2: wbuf.append(v) return (ResultItr(vbuf), ResultItr(wbuf)) - return vs.union(ws).groupByKey(numPartitions).mapValues(dispatch) + return vs.union(ws).groupByKey(numPartitions).mapValues(lambda x : dispatch(x.__iter__())) diff --git a/python/pyspark/resultitr.py b/python/pyspark/resultitr.py index c0a07f64ecb78..083045f7e75e5 100644 --- a/python/pyspark/resultitr.py +++ b/python/pyspark/resultitr.py @@ -19,20 +19,14 @@ import collections -class ResultItr(collections.Iterator): +class ResultItr(collections.Iterable): """ - A special result iterator. This is used because the standard iterator can not be pickled + A special result iterable. This is used because the standard iterator can not be pickled """ def __init__(self, data): self.data = data self.index = 0 self.maxindex = len(data) - def next(self): - if index == maxindex: - raise StopIteration - v = self.data[0] - self.data = data[1:] - return v def __iter__(self): return iter(self.data) def __len__(self): diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index 4b29f3e2f93b2..2ac943d7bf781 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -17,8 +17,8 @@ package org.apache.spark.streaming.api.java -import java.lang.{Long => JLong} -import java.util.{List => JList, Iterator => JIterator} +import java.lang.{Long => JLong, Iterable => JIterable} +import java.util.{List => JList} import scala.collection.JavaConversions._ import scala.reflect.ClassTag @@ -115,15 +115,15 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with Spark's default number of partitions. */ - def groupByKey(): JavaPairDStream[K, JIterator[V]] = - dstream.groupByKey().mapValues(asJavaIterator _) + def groupByKey(): JavaPairDStream[K, JIterable[V]] = + dstream.groupByKey().mapValues(asJavaIterable _) /** * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with `numPartitions` partitions. */ - def groupByKey(numPartitions: Int): JavaPairDStream[K, JIterator[V]] = - dstream.groupByKey(numPartitions).mapValues(asJavaIterator _) + def groupByKey(numPartitions: Int): JavaPairDStream[K, JIterable[V]] = + dstream.groupByKey(numPartitions).mapValues(asJavaIterable _) /** * Return a new DStream by applying `groupByKey` on each RDD of `this` DStream. @@ -131,8 +131,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * single sequence to generate the RDDs of the new DStream. org.apache.spark.Partitioner * is used to control the partitioning of each RDD. */ - def groupByKey(partitioner: Partitioner): JavaPairDStream[K, JIterator[V]] = - dstream.groupByKey(partitioner).mapValues(asJavaIterator _) + def groupByKey(partitioner: Partitioner): JavaPairDStream[K, JIterable[V]] = + dstream.groupByKey(partitioner).mapValues(asJavaIterable _) /** * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are @@ -196,8 +196,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval */ - def groupByKeyAndWindow(windowDuration: Duration): JavaPairDStream[K, JIterator[V]] = { - dstream.groupByKeyAndWindow(windowDuration).mapValues(asJavaIterator _) + def groupByKeyAndWindow(windowDuration: Duration): JavaPairDStream[K, JIterable[V]] = { + dstream.groupByKeyAndWindow(windowDuration).mapValues(asJavaIterable _) } /** @@ -211,8 +211,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * DStream's batching interval */ def groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration) - : JavaPairDStream[K, JIterator[V]] = { - dstream.groupByKeyAndWindow(windowDuration, slideDuration).mapValues(asJavaIterator _) + : JavaPairDStream[K, JIterable[V]] = { + dstream.groupByKeyAndWindow(windowDuration, slideDuration).mapValues(asJavaIterable _) } /** @@ -227,9 +227,9 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * @param numPartitions Number of partitions of each RDD in the new DStream. */ def groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration, numPartitions: Int) - :JavaPairDStream[K, JIterator[V]] = { + :JavaPairDStream[K, JIterable[V]] = { dstream.groupByKeyAndWindow(windowDuration, slideDuration, numPartitions) - .mapValues(asJavaIterator _) + .mapValues(asJavaIterable _) } /** @@ -247,9 +247,9 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( windowDuration: Duration, slideDuration: Duration, partitioner: Partitioner - ):JavaPairDStream[K, JIterator[V]] = { + ):JavaPairDStream[K, JIterable[V]] = { dstream.groupByKeyAndWindow(windowDuration, slideDuration, partitioner) - .mapValues(asJavaIterator _) + .mapValues(asJavaIterable _) } /** @@ -518,9 +518,9 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * Hash partitioning is used to generate the RDDs with Spark's default number * of partitions. */ - def cogroup[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (JIterator[V], JIterator[W])] = { + def cogroup[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (JIterable[V], JIterable[W])] = { implicit val cm: ClassTag[W] = fakeClassTag - dstream.cogroup(other.dstream).mapValues(t => (asJavaIterator(t._1), asJavaIterator((t._2)))) + dstream.cogroup(other.dstream).mapValues(t => (asJavaIterable(t._1), asJavaIterable((t._2)))) } /** @@ -530,10 +530,10 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( def cogroup[W]( other: JavaPairDStream[K, W], numPartitions: Int - ): JavaPairDStream[K, (JIterator[V], JIterator[W])] = { + ): JavaPairDStream[K, (JIterable[V], JIterable[W])] = { implicit val cm: ClassTag[W] = fakeClassTag dstream.cogroup(other.dstream, numPartitions) - .mapValues(t => (asJavaIterator(t._1), asJavaIterator((t._2)))) + .mapValues(t => (asJavaIterable(t._1), asJavaIterable((t._2)))) } /** @@ -543,10 +543,10 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( def cogroup[W]( other: JavaPairDStream[K, W], partitioner: Partitioner - ): JavaPairDStream[K, (JIterator[V], JIterator[W])] = { + ): JavaPairDStream[K, (JIterable[V], JIterable[W])] = { implicit val cm: ClassTag[W] = fakeClassTag dstream.cogroup(other.dstream, partitioner) - .mapValues(t => (asJavaIterator(t._1), asJavaIterator((t._2)))) + .mapValues(t => (asJavaIterable(t._1), asJavaIterable((t._2)))) } /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala index fa269da89b905..354bc132dcdc0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala @@ -51,7 +51,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with Spark's default number of partitions. */ - def groupByKey(): DStream[(K, Iterator[V])] = { + def groupByKey(): DStream[(K, Iterable[V])] = { groupByKey(defaultPartitioner()) } @@ -59,7 +59,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with `numPartitions` partitions. */ - def groupByKey(numPartitions: Int): DStream[(K, Iterator[V])] = { + def groupByKey(numPartitions: Int): DStream[(K, Iterable[V])] = { groupByKey(defaultPartitioner(numPartitions)) } @@ -67,12 +67,12 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * Return a new DStream by applying `groupByKey` on each RDD. The supplied * org.apache.spark.Partitioner is used to control the partitioning of each RDD. */ - def groupByKey(partitioner: Partitioner): DStream[(K, Iterator[V])] = { + def groupByKey(partitioner: Partitioner): DStream[(K, Iterable[V])] = { val createCombiner = (v: V) => ArrayBuffer[V](v) val mergeValue = (c: ArrayBuffer[V], v: V) => (c += v) val mergeCombiner = (c1: ArrayBuffer[V], c2: ArrayBuffer[V]) => (c1 ++ c2) combineByKey(createCombiner, mergeValue, mergeCombiner, partitioner) - .asInstanceOf[DStream[(K, Seq[V])]].mapValues(_.toIterator) + .asInstanceOf[DStream[(K, Iterable[V])]] } /** @@ -126,7 +126,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval */ - def groupByKeyAndWindow(windowDuration: Duration): DStream[(K, Iterator[V])] = { + def groupByKeyAndWindow(windowDuration: Duration): DStream[(K, Iterable[V])] = { groupByKeyAndWindow(windowDuration, self.slideDuration, defaultPartitioner()) } @@ -141,7 +141,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * DStream's batching interval */ def groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration) - : DStream[(K, Iterator[V])] = + : DStream[(K, Iterable[V])] = { groupByKeyAndWindow(windowDuration, slideDuration, defaultPartitioner()) } @@ -162,7 +162,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) windowDuration: Duration, slideDuration: Duration, numPartitions: Int - ): DStream[(K, Iterator[V])] = { + ): DStream[(K, Iterable[V])] = { groupByKeyAndWindow(windowDuration, slideDuration, defaultPartitioner(numPartitions)) } @@ -181,14 +181,14 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) windowDuration: Duration, slideDuration: Duration, partitioner: Partitioner - ): DStream[(K, Iterator[V])] = { - val createCombiner = (v: Iterator[V]) => new ArrayBuffer[V] ++= v - val mergeValue = (buf: ArrayBuffer[V], v: Iterator[V]) => buf ++= v + ): DStream[(K, Iterable[V])] = { + val createCombiner = (v: Iterable[V]) => new ArrayBuffer[V] ++= v + val mergeValue = (buf: ArrayBuffer[V], v: Iterable[V]) => buf ++= v val mergeCombiner = (buf1: ArrayBuffer[V], buf2: ArrayBuffer[V]) => buf1 ++= buf2 self.groupByKey(partitioner) .window(windowDuration, slideDuration) .combineByKey[ArrayBuffer[V]](createCombiner, mergeValue, mergeCombiner, partitioner) - .asInstanceOf[DStream[(K, Seq[V])]].mapValues(_.toIterator) + .asInstanceOf[DStream[(K, Iterable[V])]] } /** @@ -439,7 +439,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * Hash partitioning is used to generate the RDDs with Spark's default number * of partitions. */ - def cogroup[W: ClassTag](other: DStream[(K, W)]): DStream[(K, (Iterator[V], Iterator[W]))] = { + def cogroup[W: ClassTag](other: DStream[(K, W)]): DStream[(K, (Iterable[V], Iterable[W]))] = { cogroup(other, defaultPartitioner()) } @@ -448,7 +448,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. */ def cogroup[W: ClassTag](other: DStream[(K, W)], numPartitions: Int) - : DStream[(K, (Iterator[V], Iterator[W]))] = { + : DStream[(K, (Iterable[V], Iterable[W]))] = { cogroup(other, defaultPartitioner(numPartitions)) } @@ -459,7 +459,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) def cogroup[W: ClassTag]( other: DStream[(K, W)], partitioner: Partitioner - ): DStream[(K, (Iterator[V], Iterator[W]))] = { + ): DStream[(K, (Iterable[V], Iterable[W]))] = { self.transformWith( other, (rdd1: RDD[(K, V)], rdd2: RDD[(K, W)]) => rdd1.cogroup(rdd2, partitioner) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala index 09ff4163985c9..7e22268767de7 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala @@ -56,10 +56,11 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag]( // first map the cogrouped tuple to tuples of required type, // and then apply the update function val updateFuncLocal = updateFunc - val finalFunc = (iterator: Iterator[(K, (Iterator[V], Iterator[S]))]) => { + val finalFunc = (iterator: Iterator[(K, (Iterable[V], Iterable[S]))]) => { val i = iterator.map(t => { - val headOption = t._2._2.hasNext match { - case true => Some(t._2._2.next()) + val itr = t._2._2.iterator + val headOption = itr.hasNext match { + case true => Some(itr.next()) case false => None } (t._1, t._2._1.toSeq, headOption) @@ -94,7 +95,7 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag]( // first map the grouped tuple to tuples of required type, // and then apply the update function val updateFuncLocal = updateFunc - val finalFunc = (iterator: Iterator[(K, Iterator[V])]) => { + val finalFunc = (iterator: Iterator[(K, Iterable[V])]) => { updateFuncLocal(iterator.map(tuple => (tuple._1, tuple._2.toSeq, None))) } From 66b583d7b6c8119389c44d86474d98b324b12b62 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Mon, 7 Apr 2014 14:27:13 -0700 Subject: [PATCH 31/39] Fix the core test suite to compile --- .../java/org/apache/spark/JavaAPISuite.java | 32 ++++++++++++------- .../scala/org/apache/spark/FailureSuite.scala | 2 +- .../org/apache/spark/PipedRDDSuite.scala | 2 +- .../apache/spark/streaming/JavaAPISuite.java | 16 +++++++--- 4 files changed, 34 insertions(+), 18 deletions(-) diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 980a6236d3cf3..81c4a0223a465 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -78,7 +78,7 @@ public int compare(Integer a, Integer b) { } } - public int iteratorSize(Iterator a) { + private int iteratorSize(Iterator a) { int size = 0; while (a.hasNext()) { size++; @@ -87,7 +87,12 @@ public int iteratorSize(Iterator a) { return size; } - public String iteratorStr(Iterator a) { + private int iterableSize(Iterable a) { + return iteratorSize(a.iterator()); + } + + + private String iteratorStr(Iterator a) { StringBuilder str = new StringBuilder(); str.append("["); while (a.hasNext()) { @@ -100,6 +105,11 @@ public String iteratorStr(Iterator a) { return str.toString(); } + private String iterableStr(Iterable a) { + return iteratorStr(a.iterator()); + } + + @SuppressWarnings("unchecked") @Test public void sparkContextUnion() { @@ -220,7 +230,7 @@ public void lookup() { new Tuple2("Oranges", "Citrus") )); Assert.assertEquals(2, categories.lookup("Oranges").size()); - Assert.assertEquals(2, iteratorSize(categories.groupByKey().lookup("Oranges").get(0))); + Assert.assertEquals(2, iterableSize(categories.groupByKey().lookup("Oranges").get(0))); } @Test @@ -232,15 +242,15 @@ public Boolean call(Integer x) { return x % 2 == 0; } }; - JavaPairRDD> oddsAndEvens = rdd.groupBy(isOdd); + JavaPairRDD> oddsAndEvens = rdd.groupBy(isOdd); Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, iteratorSize(oddsAndEvens.lookup(true).get(0))); // Evens - Assert.assertEquals(5, iteratorSize(oddsAndEvens.lookup(false).get(0))); // Odds + Assert.assertEquals(2, iterableSize(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, iterableSize(oddsAndEvens.lookup(false).get(0))); // Odds oddsAndEvens = rdd.groupBy(isOdd, 1); Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, iteratorSize(oddsAndEvens.lookup(true).get(0))); // Evens - Assert.assertEquals(5, iteratorSize(oddsAndEvens.lookup(false).get(0))); // Odds + Assert.assertEquals(2, iterableSize(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, iterableSize(oddsAndEvens.lookup(false).get(0))); // Odds } @SuppressWarnings("unchecked") @@ -255,9 +265,9 @@ public void cogroup() { new Tuple2("Oranges", 2), new Tuple2("Apples", 3) )); - JavaPairRDD, Iterator>> cogrouped = categories.cogroup(prices); - Assert.assertEquals("[Fruit, Citrus]", iteratorStr(cogrouped.lookup("Oranges").get(0)._1())); - Assert.assertEquals("[2]", iteratorStr(cogrouped.lookup("Oranges").get(0)._2())); + JavaPairRDD, Iterable>> cogrouped = categories.cogroup(prices); + Assert.assertEquals("[Fruit, Citrus]", iterableStr(cogrouped.lookup("Oranges").get(0)._1())); + Assert.assertEquals("[2]", iterableStr(cogrouped.lookup("Oranges").get(0)._2())); cogrouped.collect(); } diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index fa052a10fd326..d6ee130f9fd13 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -72,7 +72,7 @@ class FailureSuite extends FunSuite with LocalSparkContext { throw new Exception("Intentional task failure") } } - val vHead = v.next() + val vHead = v.iterator.next() (k, vHead * vHead) }.collect() FailureSuiteState.synchronized { diff --git a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala index d57b1d9c9dd81..867b28cc0d971 100644 --- a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala @@ -85,7 +85,7 @@ class PipedRDDSuite extends FunSuite with SharedSparkContext { (f: String => Unit) => { bl.value.map(f(_)); f("\u0001") }, - (i: Tuple2[String, Iterator[String]], f: String => Unit) => { + (i: Tuple2[String, Iterable[String]], f: String => Unit) => { for (e <- i._2) { f(e + "_") } diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index 0644202de3434..97bcf41e2f645 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -23,6 +23,7 @@ import org.junit.Test; import java.io.*; import java.util.*; +import java.lang.Iterable; import com.google.common.base.Optional; import com.google.common.collect.Lists; @@ -52,6 +53,11 @@ public void equalIterator(Iterator a, Iterator b) { Assert.assertEquals(a.hasNext(), b.hasNext()); } + public void equalIterable(Iterable a, Iterable b) { + equalIterator(a.iterator(), b.iterator()); + } + + @SuppressWarnings("unchecked") @Test public void testCount() { @@ -1023,9 +1029,9 @@ public void testPairGroupByKey() { JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream> grouped = pairStream.groupByKey(); + JavaPairDStream> grouped = pairStream.groupByKey(); JavaTestUtils.attachTestOutputStream(grouped); - List>>> result = JavaTestUtils.runStreams(ssc, 2, 2); + List>>> result = JavaTestUtils.runStreams(ssc, 2, 2); Assert.assertEquals(expected.size(), result.size()); Iterator>>> resultItr = result.iterator(); @@ -1148,7 +1154,7 @@ public void testGroupByKeyAndWindow() { JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream> groupWindowed = + JavaPairDStream> groupWindowed = pairStream.groupByKeyAndWindow(new Duration(2000), new Duration(1000)); JavaTestUtils.attachTestOutputStream(groupWindowed); List>>> result = JavaTestUtils.runStreams(ssc, 3, 3); @@ -1491,9 +1497,9 @@ public void testCoGroup() { ssc, stringStringKVStream2, 1); JavaPairDStream pairStream2 = JavaPairDStream.fromJavaDStream(stream2); - JavaPairDStream, Iterator>> grouped = pairStream1.cogroup(pairStream2); + JavaPairDStream, Iterable>> grouped = pairStream1.cogroup(pairStream2); JavaTestUtils.attachTestOutputStream(grouped); - List, Iterator>>>> result = JavaTestUtils.runStreams(ssc, 2, 2); + List, Iterable>>>> result = JavaTestUtils.runStreams(ssc, 2, 2); Assert.assertEquals(expected.size(), result.size()); Iterator, Iterator>>>> resultItr = result.iterator(); From 11e730c9fc84e8a539e4c7ffa6361e8c280a4669 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Mon, 7 Apr 2014 21:27:10 -0700 Subject: [PATCH 32/39] Fix streaming tests --- .../apache/spark/streaming/JavaAPISuite.java | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index 97bcf41e2f645..13fa64894b773 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -1034,16 +1034,16 @@ public void testPairGroupByKey() { List>>> result = JavaTestUtils.runStreams(ssc, 2, 2); Assert.assertEquals(expected.size(), result.size()); - Iterator>>> resultItr = result.iterator(); + Iterator>>> resultItr = result.iterator(); Iterator>>> expectedItr = expected.iterator(); while (resultItr.hasNext() && expectedItr.hasNext()) { - Iterator>> resultElements = resultItr.next().iterator(); + Iterator>> resultElements = resultItr.next().iterator(); Iterator>> expectedElements = expectedItr.next().iterator(); while (resultElements.hasNext() && expectedElements.hasNext()) { - Tuple2> resultElement = resultElements.next(); + Tuple2> resultElement = resultElements.next(); Tuple2> expectedElement = expectedElements.next(); Assert.assertEquals(expectedElement._1(), resultElement._1()); - equalIterator(expectedElement._2().iterator(), resultElement._2()); + equalIterable(expectedElement._2(), resultElement._2()); } Assert.assertEquals(resultElements.hasNext(), expectedElements.hasNext()); } @@ -1497,22 +1497,22 @@ public void testCoGroup() { ssc, stringStringKVStream2, 1); JavaPairDStream pairStream2 = JavaPairDStream.fromJavaDStream(stream2); - JavaPairDStream, Iterable>> grouped = pairStream1.cogroup(pairStream2); + JavaPairDStream, Iterable>> grouped = pairStream1.cogroup(pairStream2); JavaTestUtils.attachTestOutputStream(grouped); - List, Iterable>>>> result = JavaTestUtils.runStreams(ssc, 2, 2); + List, Iterable>>>> result = JavaTestUtils.runStreams(ssc, 2, 2); Assert.assertEquals(expected.size(), result.size()); - Iterator, Iterator>>>> resultItr = result.iterator(); + Iterator, Iterable>>>> resultItr = result.iterator(); Iterator, List>>>> expectedItr = expected.iterator(); while (resultItr.hasNext() && expectedItr.hasNext()) { - Iterator, Iterator>>> resultElements = resultItr.next().iterator(); + Iterator, Iterable>>> resultElements = resultItr.next().iterator(); Iterator, List>>> expectedElements = expectedItr.next().iterator(); while (resultElements.hasNext() && expectedElements.hasNext()) { - Tuple2, Iterator>> resultElement = resultElements.next(); + Tuple2, Iterable>> resultElement = resultElements.next(); Tuple2, List>> expectedElement = expectedElements.next(); Assert.assertEquals(expectedElement._1(), resultElement._1()); - equalIterator(expectedElement._2()._1().iterator(), resultElement._2()._1()); - equalIterator(expectedElement._2()._2().iterator(), resultElement._2()._2()); + equalIterable(expectedElement._2()._1(), resultElement._2()._1()); + equalIterable(expectedElement._2()._2(), resultElement._2()._2()); } Assert.assertEquals(resultElements.hasNext(), expectedElements.hasNext()); } From 2d06e101324f4759ff5e609aea56b67a0371f653 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Mon, 7 Apr 2014 21:38:10 -0700 Subject: [PATCH 33/39] Fix Java 8 cogroup tests for the new API --- .../java/org/apache/spark/Java8APISuite.java | 20 ++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java b/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java index f67251217ed4a..0e409f30a2881 100644 --- a/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java +++ b/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java @@ -60,6 +60,16 @@ public void tearDown() { System.clearProperty("spark.driver.port"); } + private int iterableSize(Iterable a) { + int count = 0; + Iterator aItr = a.iterator(); + while (aItr.hasNext()) { + aItr.next(); + count++; + } + return count; + } + @Test public void foreachWithAnonymousClass() { foreachCalls = 0; @@ -85,15 +95,15 @@ public void foreach() { public void groupBy() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); Function isOdd = x -> x % 2 == 0; - JavaPairRDD> oddsAndEvens = rdd.groupBy(isOdd); + JavaPairRDD> oddsAndEvens = rdd.groupBy(isOdd); Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, oddsAndEvens.lookup(true).get(0).size()); // Evens - Assert.assertEquals(5, oddsAndEvens.lookup(false).get(0).size()); // Odds + Assert.assertEquals(2, iterableSize(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, iterableSize(oddsAndEvens.lookup(false).get(0))); // Odds oddsAndEvens = rdd.groupBy(isOdd, 1); Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, oddsAndEvens.lookup(true).get(0).size()); // Evens - Assert.assertEquals(5, oddsAndEvens.lookup(false).get(0).size()); // Odds + Assert.assertEquals(2, iterableSize(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, iterableSize(oddsAndEvens.lookup(false).get(0))); // Odds } @Test From c5075aa5809358c16b6b6aafaf0294d1fb87cbb9 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 8 Apr 2014 05:09:24 -0700 Subject: [PATCH 34/39] If guava 14 had iterables --- .../apache/spark/rdd/PairRDDFunctions.scala | 9 +++---- .../java/org/apache/spark/JavaAPISuite.java | 26 +++---------------- .../scala/org/apache/spark/FailureSuite.scala | 3 +-- .../apache/spark/examples/JavaPageRank.java | 10 +++---- .../java/org/apache/spark/Java8APISuite.java | 19 ++++---------- 5 files changed, 16 insertions(+), 51 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index f2127b5384e94..a92a84b5342d1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -298,8 +298,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) */ def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => - val wlist = ws.toList - for (v <- vs; w <- wlist.iterator) yield (v, w) + for (v <- vs; w <- ws) yield (v, w) } } @@ -314,8 +313,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) if (ws.isEmpty) { vs.map(v => (v, None)) } else { - val wlist = ws.toList - for (v <- vs; w <- wlist.iterator) yield (v, Some(w)) + for (v <- vs; w <- ws) yield (v, Some(w)) } } } @@ -332,8 +330,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) if (vs.isEmpty) { ws.map(w => (None, w)) } else { - val wlist = ws.toList - for (v <- vs; w <- wlist) yield (Some(v), w) + for (v <- vs; w <- ws) yield (Some(v), w) } } } diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 81c4a0223a465..e82caaac8a042 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -24,6 +24,7 @@ import scala.Tuple2; import com.google.common.collect.Lists; +import com.google.collect.Iterables; import com.google.common.base.Optional; import com.google.common.base.Charsets; import com.google.common.io.Files; @@ -78,35 +79,14 @@ public int compare(Integer a, Integer b) { } } - private int iteratorSize(Iterator a) { - int size = 0; - while (a.hasNext()) { - size++; - a.next(); - } - return size; - } private int iterableSize(Iterable a) { - return iteratorSize(a.iterator()); + return Iterables.size(a.iterator()); } - private String iteratorStr(Iterator a) { - StringBuilder str = new StringBuilder(); - str.append("["); - while (a.hasNext()) { - str.append(a.next().toString()); - if (a.hasNext()) { - str.append(", "); - } - } - str.append("]"); - return str.toString(); - } - private String iterableStr(Iterable a) { - return iteratorStr(a.iterator()); + return Iterables.toString(a.iterator()); } diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index d6ee130f9fd13..12dbebcb28644 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -72,8 +72,7 @@ class FailureSuite extends FunSuite with LocalSparkContext { throw new Exception("Intentional task failure") } } - val vHead = v.iterator.next() - (k, vHead * vHead) + (k, v.head * v.head) }.collect() FailureSuiteState.synchronized { assert(FailureSuiteState.tasksRun === 4) diff --git a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java index cffc747b3c286..c0525f887e7cd 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -17,7 +17,10 @@ package org.apache.spark.examples; + import scala.Tuple2; + +import com.google.collect.Iterables; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -90,12 +93,7 @@ public Double call(Iterable rs) { .flatMapToPair(new PairFlatMapFunction, Double>, String, Double>() { @Override public Iterable> call(Tuple2, Double> s) { - int urlCount = 0; - Iterator urls = s._1.iterator(); - while (urls.hasNext()) { - urls.next(); - urlCount++; - } + int urlCount = Iterables.size(s._1); List> results = new ArrayList>(); for (String n : s._1) { results.add(new Tuple2(n, s._2() / urlCount)); diff --git a/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java b/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java index 0e409f30a2881..9c45863f8bae9 100644 --- a/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java +++ b/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java @@ -23,6 +23,7 @@ import scala.Tuple2; +import com.google.collections.Iterables; import com.google.common.base.Optional; import com.google.common.io.Files; import org.apache.hadoop.io.IntWritable; @@ -60,16 +61,6 @@ public void tearDown() { System.clearProperty("spark.driver.port"); } - private int iterableSize(Iterable a) { - int count = 0; - Iterator aItr = a.iterator(); - while (aItr.hasNext()) { - aItr.next(); - count++; - } - return count; - } - @Test public void foreachWithAnonymousClass() { foreachCalls = 0; @@ -97,13 +88,13 @@ public void groupBy() { Function isOdd = x -> x % 2 == 0; JavaPairRDD> oddsAndEvens = rdd.groupBy(isOdd); Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, iterableSize(oddsAndEvens.lookup(true).get(0))); // Evens - Assert.assertEquals(5, iterableSize(oddsAndEvens.lookup(false).get(0))); // Odds + Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds oddsAndEvens = rdd.groupBy(isOdd, 1); Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, iterableSize(oddsAndEvens.lookup(true).get(0))); // Evens - Assert.assertEquals(5, iterableSize(oddsAndEvens.lookup(false).get(0))); // Odds + Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds } @Test From eb06216fd7efb4c057debca011d0061c896ded25 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 8 Apr 2014 05:17:13 -0700 Subject: [PATCH 35/39] maybe I should have had a coffee first. use correct import for guava iterables --- .../java/org/apache/spark/JavaAPISuite.java | 27 ++++++------------- .../apache/spark/examples/JavaPageRank.java | 2 +- .../java/org/apache/spark/Java8APISuite.java | 2 +- 3 files changed, 10 insertions(+), 21 deletions(-) diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index e82caaac8a042..ab2fdac553349 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -23,8 +23,8 @@ import scala.Tuple2; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.google.collect.Iterables; import com.google.common.base.Optional; import com.google.common.base.Charsets; import com.google.common.io.Files; @@ -79,17 +79,6 @@ public int compare(Integer a, Integer b) { } } - - private int iterableSize(Iterable a) { - return Iterables.size(a.iterator()); - } - - - private String iterableStr(Iterable a) { - return Iterables.toString(a.iterator()); - } - - @SuppressWarnings("unchecked") @Test public void sparkContextUnion() { @@ -210,7 +199,7 @@ public void lookup() { new Tuple2("Oranges", "Citrus") )); Assert.assertEquals(2, categories.lookup("Oranges").size()); - Assert.assertEquals(2, iterableSize(categories.groupByKey().lookup("Oranges").get(0))); + Assert.assertEquals(2, Iterables.size(categories.groupByKey().lookup("Oranges").get(0))); } @Test @@ -224,13 +213,13 @@ public Boolean call(Integer x) { }; JavaPairRDD> oddsAndEvens = rdd.groupBy(isOdd); Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, iterableSize(oddsAndEvens.lookup(true).get(0))); // Evens - Assert.assertEquals(5, iterableSize(oddsAndEvens.lookup(false).get(0))); // Odds + Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds oddsAndEvens = rdd.groupBy(isOdd, 1); Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, iterableSize(oddsAndEvens.lookup(true).get(0))); // Evens - Assert.assertEquals(5, iterableSize(oddsAndEvens.lookup(false).get(0))); // Odds + Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds } @SuppressWarnings("unchecked") @@ -246,8 +235,8 @@ public void cogroup() { new Tuple2("Apples", 3) )); JavaPairRDD, Iterable>> cogrouped = categories.cogroup(prices); - Assert.assertEquals("[Fruit, Citrus]", iterableStr(cogrouped.lookup("Oranges").get(0)._1())); - Assert.assertEquals("[2]", iterableStr(cogrouped.lookup("Oranges").get(0)._2())); + Assert.assertEquals("[Fruit, Citrus]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._1())); + Assert.assertEquals("[2]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._2())); cogrouped.collect(); } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java index c0525f887e7cd..8513ba07e7705 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -20,7 +20,7 @@ import scala.Tuple2; -import com.google.collect.Iterables; +import com.google.common.collect.Iterables; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; diff --git a/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java b/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java index 9c45863f8bae9..7eb8b45fc3cf0 100644 --- a/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java +++ b/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java @@ -23,7 +23,7 @@ import scala.Tuple2; -import com.google.collections.Iterables; +import com.google.common.collections.Iterables; import com.google.common.base.Optional; import com.google.common.io.Files; import org.apache.hadoop.io.IntWritable; From 7a092a3074cb567c9f842e96f4eb124fe46d10fe Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 8 Apr 2014 10:56:46 -0700 Subject: [PATCH 36/39] switch resultitr to resultiterable --- python/pyspark/join.py | 6 +++--- python/pyspark/rdd.py | 4 ++-- python/pyspark/{resultitr.py => resultiterable.py} | 4 ++-- 3 files changed, 7 insertions(+), 7 deletions(-) rename python/pyspark/{resultitr.py => resultiterable.py} (93%) diff --git a/python/pyspark/join.py b/python/pyspark/join.py index df2a75154178b..6f94d26ef86a9 100644 --- a/python/pyspark/join.py +++ b/python/pyspark/join.py @@ -31,7 +31,7 @@ OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. """ -from pyspark.resultitr import ResultItr +from pyspark.resultiterable import ResultIterable def _do_python_join(rdd, other, numPartitions, dispatch): vs = rdd.map(lambda (k, v): (k, (1, v))) @@ -89,5 +89,5 @@ def dispatch(seq): vbuf.append(v) elif n == 2: wbuf.append(v) - return (ResultItr(vbuf), ResultItr(wbuf)) - return vs.union(ws).groupByKey(numPartitions).mapValues(lambda x : dispatch(x.__iter__())) + return (ResultIterable(vbuf), ResultIterable(wbuf)) + return vs.union(ws).groupByKey(numPartitions).mapValues(dispatch) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index feb00731af8e9..91fc7e637e2c6 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -38,7 +38,7 @@ from pyspark.statcounter import StatCounter from pyspark.rddsampler import RDDSampler from pyspark.storagelevel import StorageLevel -from pyspark.resultitr import ResultItr +from pyspark.resultiterable import ResultIterable from py4j.java_collections import ListConverter, MapConverter @@ -1134,7 +1134,7 @@ def mergeCombiners(a, b): return a + b return self.combineByKey(createCombiner, mergeValue, mergeCombiners, - numPartitions).mapValues(lambda x: ResultItr(x)) + numPartitions).mapValues(lambda x: ResultIterable(x)) # TODO: add tests def flatMapValues(self, f): diff --git a/python/pyspark/resultitr.py b/python/pyspark/resultiterable.py similarity index 93% rename from python/pyspark/resultitr.py rename to python/pyspark/resultiterable.py index 083045f7e75e5..fbb5832fd8f03 100644 --- a/python/pyspark/resultitr.py +++ b/python/pyspark/resultiterable.py @@ -15,11 +15,11 @@ # limitations under the License. # -__all__ = ["ResultItr"] +__all__ = ["ResultIterable"] import collections -class ResultItr(collections.Iterable): +class ResultIterable(collections.Iterator): """ A special result iterable. This is used because the standard iterator can not be pickled """ From d3fe9098e01271e274943e033a47390d23732ae2 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 8 Apr 2014 12:19:40 -0700 Subject: [PATCH 37/39] use toSeq instead --- mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala | 3 +-- .../main/scala/org/apache/spark/mllib/recommendation/ALS.scala | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala index d5b7d9f855f03..0d97b7d92f155 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala @@ -287,8 +287,7 @@ class SVD { val rows = data.map(entry => (entry.i, (entry.j, entry.mval))).groupByKey() val emits = rows.flatMap { - case (rowind, colsI) => - val cols = colsI.toList + case (rowind, cols) => cols.flatMap { case (colind1, mval1) => cols.map { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 08894e709907d..3124fac326d22 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -426,7 +426,7 @@ class ALS private ( : Array[Array[Double]] = { // Sort the incoming block factor messages by block ID and make them an array - val blockFactors = messages.toArray.sortBy(_._1).map(_._2) // Array[Array[Double]] + val blockFactors = messages.toSeq.sortBy(_._1).map(_._2).toArray // Array[Array[Double]] val numBlocks = blockFactors.length val numUsers = inLinkBlock.elementIds.length From 77048f8c8a8d33f9a4785bf6d336ae5cf20e3dfc Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 8 Apr 2014 14:48:49 -0700 Subject: [PATCH 38/39] Fix merge up to master --- 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 383aeda8a5738..ae577b500ccb4 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 @@ -17,7 +17,7 @@ package org.apache.spark.api.java -import java.util.{Comparator, List => JList} +import java.util.{Comparator, List => JList, Iterator => JIterator} import java.lang.{Iterable => JIterable} import scala.collection.JavaConversions._ From f289536d3ef2b1fa1fac00923fa6921e15b4a659 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 8 Apr 2014 16:00:46 -0700 Subject: [PATCH 39/39] Fix bad merge, should have been Iterable rather than Iterator --- python/pyspark/resultiterable.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/resultiterable.py b/python/pyspark/resultiterable.py index fbb5832fd8f03..7f418f8d2e29a 100644 --- a/python/pyspark/resultiterable.py +++ b/python/pyspark/resultiterable.py @@ -19,7 +19,7 @@ import collections -class ResultIterable(collections.Iterator): +class ResultIterable(collections.Iterable): """ A special result iterable. This is used because the standard iterator can not be pickled """