From f2b8dd3929ce2a8ccfea8bcf05b3c72a7acd63fd Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 13 Aug 2013 14:21:49 -0700 Subject: [PATCH 01/13] second indexedrdd design --- .../main/scala/spark/PairRDDFunctions.scala | 4 + .../src/main/scala/spark/rdd/IndexedRDD.scala | 132 ++++++++++++++++++ 2 files changed, 136 insertions(+) create mode 100644 core/src/main/scala/spark/rdd/IndexedRDD.scala diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 6b0cc2fbf1..28b46990f8 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -715,6 +715,10 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( */ def values: RDD[V] = self.map(_._2) + + def index(existingIndex: RDD[JHashMap[K,Int]] = null): IndexedRDD[K,V] = + IndexedRDD(self, existingIndex) + private[spark] def getKeyClass() = implicitly[ClassManifest[K]].erasure private[spark] def getValueClass() = implicitly[ClassManifest[V]].erasure diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala new file mode 100644 index 0000000000..55ca353618 --- /dev/null +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package spark.rdd + +import java.util.{HashMap => JHashMap, BitSet => JBitSet, HashSet => JHashSet} + +import scala.collection.JavaConversions._ +import scala.collection.mutable.ArrayBuffer + + +import spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext} + +import spark.PairRDDFunctions +import spark.SparkContext._ + + +// import java.io.{ObjectOutputStream, IOException} + + +/** + * An IndexedRDD is an RDD[(K,V)] where each K is unique. + * + * The IndexedRDD contains an index datastructure that can + * be used to accelerate join and aggregation operations. + */ +class IndexedRDD[K: ClassManifest, V: ClassManifest]( + sc: SparkContext, + val indexRDD: RDD[ JHashMap[K, Int] ], + val valuesRDD: RDD[ Array[Seq[V]] ]) + extends RDD[(K, V)](sc, + List(new OneToOneDependency(indexRDD), new OneToOneDependency(valuesRDD)) ) { + + + + val tuples = new ZippedRDD[ JHashMap[K, Int], Array[Seq[V]] ](sc, indexRDD, valuesRDD) + + + override val partitioner = indexRDD.partitioner + override def getPartitions: Array[Partition] = tuples.getPartitions + override def getPreferredLocations(s: Partition): Seq[String] = tuples.getPreferredLocations(s) + + + + + + /** + * Provide the RDD[(K,V)] equivalent output. + */ + override def compute(part: Partition, context: TaskContext): Iterator[(K, V)] = { + tuples.compute(part, context).flatMap { case (indexMap, values) => + // Walk the index to construct the key, value pairs + indexMap.iterator + // Extract rows with key value pairs and indicators + .map{ case (k, ind) => (k, values(ind)) } + // Remove tuples that aren't actually present in the array + .filter{ case (_, valar) => valar != null } + // Extract the pair (removing the indicator from the tuple) + .flatMap{ case (k, valar) => valar.map(v => (k,v))} + } + } +} + +object IndexedRDD { + def apply[K: ClassManifest, V: ClassManifest]( + tbl: RDD[(K,V)], + existingIndex: RDD[JHashMap[K,Int]] = null ): IndexedRDD[K, V] = { + + if(existingIndex == null) { + // build th index + val groups = tbl.groupByKey().mapPartitions( iter => { + val indexMap = new JHashMap[K, Int]() + val values = new ArrayBuffer[Seq[V]]() + for((k,ar) <- iter){ + val ind = values.size + indexMap.put(k, ind) + values.append(ar) + } + List((indexMap, values.toArray)).iterator + }, true).cache + // extract the index and the values + val index = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) + val values = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) + new IndexedRDD[K,V](tbl.context, index, values) + } else { + val index = existingIndex + // Shuffle the table according to the index (if necessary) + val shuffledTbl = + if (tbl.partitioner == Some(index.partitioner)) { + tbl + } else { + new ShuffledRDD[K,V](tbl, index.partitioner.get()) + } + + // Use the index to build the new values table + val values = index.zipPartitions[ (K, Seq[V]), Array[Seq[V]] ]( + (indexIter, tblIter) => { + // There is only one map + val index: JHashMap[K,Int] = iter.next() + assert(!iter.hasNext()) + val values = new Array[Seq[V]](index.size) + for((k,a) <- tblIter) { + assert(index.contains(k)) + values(index.get(k)) = a + } + values + }, shuffleTbl) + + new IndexedRDD[K,V](index, values) + } + } + +} + + + + + From b71d4febbc4ed4760a3e2ddeecf9e677b5742635 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Wed, 14 Aug 2013 15:25:56 -0700 Subject: [PATCH 02/13] Finished early prototype of IndexedRDD --- .../src/main/scala/spark/rdd/IndexedRDD.scala | 272 ++++++++++++++++-- 1 file changed, 251 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index 55ca353618..0f6e29ad58 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -27,11 +27,12 @@ import spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskConte import spark.PairRDDFunctions import spark.SparkContext._ +import spark.SparkException +import spark.Partitioner // import java.io.{ObjectOutputStream, IOException} - /** * An IndexedRDD is an RDD[(K,V)] where each K is unique. * @@ -39,22 +40,238 @@ import spark.SparkContext._ * be used to accelerate join and aggregation operations. */ class IndexedRDD[K: ClassManifest, V: ClassManifest]( - sc: SparkContext, - val indexRDD: RDD[ JHashMap[K, Int] ], - val valuesRDD: RDD[ Array[Seq[V]] ]) - extends RDD[(K, V)](sc, - List(new OneToOneDependency(indexRDD), new OneToOneDependency(valuesRDD)) ) { + val index: RDD[ JHashMap[K, Int] ], + val valuesRDD: RDD[ Seq[Seq[V]] ]) + extends RDD[(K, V)](index.context, + List(new OneToOneDependency(index), new OneToOneDependency(valuesRDD)) ) { - val tuples = new ZippedRDD[ JHashMap[K, Int], Array[Seq[V]] ](sc, indexRDD, valuesRDD) + val tuples = new ZippedRDD[JHashMap[K, Int], Seq[Seq[V]]](index.context, index, valuesRDD) - override val partitioner = indexRDD.partitioner + override val partitioner = index.partitioner override def getPartitions: Array[Partition] = tuples.getPartitions override def getPreferredLocations(s: Partition): Seq[String] = tuples.getPreferredLocations(s) + /** + * Construct a new IndexedRDD that is indexed by only the keys in the RDD + */ + def reindex(): IndexedRDD[K,V] = IndexedRDD(this) + + + /** + * Pass each value in the key-value pair RDD through a map function without changing the keys; + * this also retains the original RDD's partitioning. + */ + def mapValues[U: ClassManifest](f: V => U): IndexedRDD[K, U] = { + val cleanF = index.context.clean(f) + val newValues = valuesRDD.mapPartitions(_.map{ values => + values.map{_.map(x => f(x))} + }, true) + new IndexedRDD[K,U](index, newValues) + } + + + /** + * Pass each value in the key-value pair RDD through a flatMap function without changing the + * keys; this also retains the original RDD's partitioning. + */ + def flatMapValues[U: ClassManifest](f: V => TraversableOnce[U]): IndexedRDD[K,U] = { + val cleanF = index.context.clean(f) + val newValues = valuesRDD.mapPartitions(_.map{ values => + values.map{_.flatMap(x => f(x))} + }, true) + new IndexedRDD[K,U](index, newValues) + } + + + /** + * Generic function to combine the elements for each key using a custom set of aggregation + * functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], for a "combined type" C + * Note that V and C can be different -- for example, one might group an RDD of type + * (Int, Int) into an RDD of type (Int, Seq[Int]). Users provide three functions: + * + * - `createCombiner`, which turns a V into a C (e.g., creates a one-element list) + * - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list) + * - `mergeCombiners`, to combine two C's into a single one. + * + * In addition, users can control the partitioning of the output RDD, and whether to perform + * map-side aggregation (if a mapper can produce multiple items with the same key). + */ + def combineByKey[C: ClassManifest](createCombiner: V => C, + mergeValue: (C, V) => C, + mergeCombiners: (C, C) => C, + partitioner: Partitioner, + mapSideCombine: Boolean = true, + serializerClass: String = null): IndexedRDD[K, C] = { + val newValues = valuesRDD.mapPartitions( + _.map{ groups: Seq[Seq[V]] => + groups.map{ group: Seq[V] => + if(group != null && !group.isEmpty) { + val c: C = createCombiner(group.head) + val sum: C = group.tail.foldLeft(c)(mergeValue) + Seq(sum) + } else { + null + } + } + }, true) + new IndexedRDD[K,C](index, newValues) + } + + + + /** + * 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: ClassManifest](other: RDD[(K, W)], partitionerUnused: Partitioner): + IndexedRDD[K, (Seq[V], Seq[W])] = { + other match { + case other: IndexedRDD[_, _] if other.index == index => { + // if both RDDs share exactly the same index and therefore the same super set of keys + // then we simply merge the value RDDs. + // However it is possible that both RDDs are missing a value for a given key in + // which case the returned RDD should have a null value + val newValues = + valuesRDD.zipPartitions[ Seq[Seq[W]], Seq[Seq[(Seq[V], Seq[W])]] ]( + (thisIter, otherIter) => { + val thisValues: Seq[Seq[V]] = thisIter.next() + assert(!thisIter.hasNext()) + val otherValues: Seq[Seq[W]] = otherIter.next() + assert(!otherIter.hasNext()) + // Zip the values and if both arrays are null then the key is not present and + // so the resulting value must be null (not a tuple of empty sequences) + val tmp: Seq[Seq[(Seq[V], Seq[W])]] = thisValues.view.zip(otherValues).map{ + case (null, null) => null // The key is not present in either RDD + case (a, null) => Seq((a, Seq.empty[W])) + case (null, b) => Seq((Seq.empty[V], b)) + case (a, b) => Seq((a,b)) + }.toSeq + List(tmp).iterator + }, other.valuesRDD) + new IndexedRDD[K, (Seq[V], Seq[W])](index, newValues) + } + case other: IndexedRDD[_, _] if other.index.partitioner == index.partitioner => { + // If both RDDs are indexed using different indices but with the same partitioners + // then we we need to first merge the indicies and then use the merged index to + // merge the values. + val newIndex = + index.zipPartitions[JHashMap[K,Int], JHashMap[K,Int]]( (thisIter, otherIter) => { + val thisIndex = thisIter.next() + assert(!thisIter.hasNext()) + val otherIndex = otherIter.next() + assert(!otherIter.hasNext()) + val newIndex = new JHashMap[K, Int]() + // @todo Merge only the keys that correspond to non-null values + // Merge the keys + newIndex.putAll(thisIndex) + newIndex.putAll(otherIndex) + // We need to rekey the index + var ctr = 0 + for(e <- newIndex.entrySet) { + e.setValue(ctr) + ctr += 1 + } + List(newIndex).iterator + }, other.index).cache() + // Use the new index along with the this and the other indices to merge the values + val newValues = + newIndex.zipPartitions[ + (JHashMap[K, Int], Seq[Seq[V]]), + (JHashMap[K, Int], Seq[Seq[W]]), + Seq[Seq[(Seq[V],Seq[W])]] ]( + (newIndexIter, thisTuplesIter, otherTuplesIter) => { + // Get the new index for this partition + val newIndex = newIndexIter.next() + assert(!newIndexIter.hasNext()) + // Get the corresponding indicies and values for this and the other IndexedRDD + val (thisIndex, thisValues) = thisTuplesIter.next() + assert(!thisTuplesIter.hasNext()) + val (otherIndex, otherValues) = otherTuplesIter.next() + assert(!otherTuplesIter.hasNext()) + // Preallocate the new Values array + val newValues = new Array[Seq[(Seq[V],Seq[W])]](newIndex.size) + // Lookup the sequences in both submaps + for((k,ind) <- newIndex) { + val thisSeq = if(thisIndex.contains(k)) thisValues(thisIndex.get(k)) else null + val otherSeq = if(otherIndex.contains(k)) otherValues(otherIndex.get(k)) else null + // if either of the sequences is not null then the key was in one of the two tables + // and so the value should appear in the returned table + newValues(ind) = (thisSeq, otherSeq) match { + case (null, null) => null + case (a, null) => Seq( (a, Seq.empty[W]) ) + case (null, b) => Seq( (Seq.empty[V], b) ) + case (a, b) => Seq( (a,b) ) + } + } + List(newValues.toSeq).iterator + }, tuples, other.tuples) + new IndexedRDD(newIndex, newValues) + } + case _ => { + // Get the partitioner from the index + val partitioner = index.partitioner match { + case Some(p) => p + case None => throw new SparkException("An index must have a partitioner.") + } + // Shuffle the other RDD using the partitioner for this index + val otherShuffled = + if (other.partitioner == Some(partitioner)) { + other + } else { + new ShuffledRDD[K,W](other, partitioner) + } + // Join the other RDD with this RDD building a new valueset and new index on the fly + val groups = tuples.zipPartitions[(K, W), (JHashMap[K, Int], Seq[Seq[(Seq[V],Seq[W])]]) ]( + (thisTuplesIter, otherTuplesIter) => { + // Get the corresponding indicies and values for this IndexedRDD + val (thisIndex, thisValues) = thisTuplesIter.next() + assert(!thisTuplesIter.hasNext()) + // Construct a new index + val newIndex = thisIndex.clone().asInstanceOf[JHashMap[K, Int]] + // Construct a new array Buffer to store the values + val newValues = new ArrayBuffer[(Seq[V], ArrayBuffer[W])](thisValues.size) + // populate the newValues with the values in this IndexedRDD + for((k,i) <- thisIndex) { + if(thisValues(i) != null) { + newValues(i) = (thisValues(i), new ArrayBuffer[W]()) + } + } + // Now iterate through the other tuples updating the map + for((k,w) <- otherTuplesIter){ + if(!newIndex.contains(k)) { + // update the index + val ind = newIndex.size + newIndex.put(k, ind) + // Create the buffer for w + val wbuffer = new ArrayBuffer[W]() + wbuffer.append(w) + // Update the values + newValues.append( (Seq.empty[V], wbuffer) ) + } else { + val ind = newIndex.get(k) + newValues(ind)._2.append(w) + } + } + // Finalize the new values array + val newValuesArray: Seq[Seq[(Seq[V],Seq[W])]] = + newValues.view.map{ case (s, ab) => Seq((s, ab.toSeq)) }.toSeq + List( (newIndex, newValuesArray) ).iterator + }, otherShuffled).cache() + + // Extract the index and values from the above RDD + val newIndex = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) + val newValues = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) + + new IndexedRDD[K, (Seq[V], Seq[W])](newIndex, newValues) + + } + } + } + @@ -68,12 +285,16 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( // Extract rows with key value pairs and indicators .map{ case (k, ind) => (k, values(ind)) } // Remove tuples that aren't actually present in the array - .filter{ case (_, valar) => valar != null } + .filter{ case (_, valar) => valar != null && !valar.isEmpty()} // Extract the pair (removing the indicator from the tuple) .flatMap{ case (k, valar) => valar.map(v => (k,v))} } } -} + +} // End of IndexedRDD + + + object IndexedRDD { def apply[K: ClassManifest, V: ClassManifest]( @@ -90,35 +311,44 @@ object IndexedRDD { indexMap.put(k, ind) values.append(ar) } - List((indexMap, values.toArray)).iterator + List((indexMap, values.toSeq)).iterator }, true).cache // extract the index and the values val index = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) val values = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) - new IndexedRDD[K,V](tbl.context, index, values) + new IndexedRDD[K,V](index, values) } else { val index = existingIndex + val partitioner = index.partitioner match { + case Some(p) => p + case None => throw new SparkException("An index must have a partitioner.") + } + // Shuffle the table according to the index (if necessary) val shuffledTbl = - if (tbl.partitioner == Some(index.partitioner)) { + if (tbl.partitioner == Some(partitioner)) { tbl } else { - new ShuffledRDD[K,V](tbl, index.partitioner.get()) + new ShuffledRDD[K,V](tbl, partitioner) } // Use the index to build the new values table - val values = index.zipPartitions[ (K, Seq[V]), Array[Seq[V]] ]( + val values = index.zipPartitions[ (K, V), Seq[Seq[V]] ]( (indexIter, tblIter) => { // There is only one map - val index: JHashMap[K,Int] = iter.next() - assert(!iter.hasNext()) + val index: JHashMap[K,Int] = indexIter.next() + assert(!indexIter.hasNext()) val values = new Array[Seq[V]](index.size) - for((k,a) <- tblIter) { + for((k,v) <- tblIter) { assert(index.contains(k)) - values(index.get(k)) = a + val ind = index(k) + if(values(ind) == null){ + values(ind) = new ArrayBuffer[V]() + } + values(ind).asInstanceOf[ArrayBuffer[V]].append(v) } - values - }, shuffleTbl) + List(values.toSeq).iterator + }, shuffledTbl) new IndexedRDD[K,V](index, values) } From 54b54903c316096b0ef1cda48670d32adeb746e1 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Wed, 14 Aug 2013 16:35:20 -0700 Subject: [PATCH 03/13] Adding testing code for indexedrdd --- .../src/main/scala/spark/rdd/IndexedRDD.scala | 24 +- .../test/scala/spark/IndexedRDDSuite.scala | 354 ++++++++++++++++++ 2 files changed, 372 insertions(+), 6 deletions(-) create mode 100644 core/src/test/scala/spark/IndexedRDDSuite.scala diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index 0f6e29ad58..a6852f3f8a 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -44,6 +44,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( val valuesRDD: RDD[ Seq[Seq[V]] ]) extends RDD[(K, V)](index.context, List(new OneToOneDependency(index), new OneToOneDependency(valuesRDD)) ) { + //with PairRDDFunctions[K,V] { @@ -129,6 +130,8 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( */ def cogroup[W: ClassManifest](other: RDD[(K, W)], partitionerUnused: Partitioner): IndexedRDD[K, (Seq[V], Seq[W])] = { + //RDD[(K, (Seq[V], Seq[W]))] = { + assert(false) other match { case other: IndexedRDD[_, _] if other.index == index => { // if both RDDs share exactly the same index and therefore the same super set of keys @@ -302,14 +305,23 @@ object IndexedRDD { existingIndex: RDD[JHashMap[K,Int]] = null ): IndexedRDD[K, V] = { if(existingIndex == null) { - // build th index - val groups = tbl.groupByKey().mapPartitions( iter => { + // Shuffle the table (if necessary) + val shuffledTbl = + if (tbl.partitioner.isEmpty) { + new ShuffledRDD[K,V](tbl, Partitioner.defaultPartitioner(tbl)) + } else { tbl } + + val groups = shuffledTbl.mapPartitions( iter => { val indexMap = new JHashMap[K, Int]() val values = new ArrayBuffer[Seq[V]]() - for((k,ar) <- iter){ - val ind = values.size - indexMap.put(k, ind) - values.append(ar) + for((k,v) <- iter){ + if(!indexMap.contains(k)) { + val ind = indexMap.size + indexMap.put(k, ind) + values.append(new ArrayBuffer[V]()) + } + val ind = indexMap.get(k) + values(ind).asInstanceOf[ArrayBuffer[V]].append(v) } List((indexMap, values.toSeq)).iterator }, true).cache diff --git a/core/src/test/scala/spark/IndexedRDDSuite.scala b/core/src/test/scala/spark/IndexedRDDSuite.scala new file mode 100644 index 0000000000..aacb6423ee --- /dev/null +++ b/core/src/test/scala/spark/IndexedRDDSuite.scala @@ -0,0 +1,354 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package spark + + +import org.scalatest.FunSuite +import org.scalatest.prop.Checkers +import org.scalacheck.Arbitrary._ +import org.scalacheck.Gen +import org.scalacheck.Prop._ + +import com.google.common.io.Files + +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashSet + +import spark.rdd.ShuffledRDD +import spark.SparkContext._ + +class IndexedRDDSuite extends FunSuite with SharedSparkContext { + + def lineage(rdd: RDD[_]): collection.mutable.HashSet[RDD[_]] = { + val set = new collection.mutable.HashSet[RDD[_]] + def visit(rdd: RDD[_]) { + for (dep <- rdd.dependencies) { + set += dep.rdd + visit(dep.rdd) + } + } + visit(rdd) + set + } + + test("groupByKey") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))).index() + val groups = pairs.groupByKey().collect() + assert(groups.size === 2) + val valuesFor1 = groups.find(_._1 == 1).get._2 + assert(valuesFor1.toList.sorted === List(1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } + + test("groupByKey with duplicates") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).index() + val groups = pairs.groupByKey().collect() + assert(groups.size === 2) + val valuesFor1 = groups.find(_._1 == 1).get._2 + assert(valuesFor1.toList.sorted === List(1, 1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } + + test("groupByKey with negative key hash codes") { + val pairs = sc.parallelize(Array((-1, 1), (-1, 2), (-1, 3), (2, 1))).index() + val groups = pairs.groupByKey().collect() + assert(groups.size === 2) + val valuesForMinus1 = groups.find(_._1 == -1).get._2 + assert(valuesForMinus1.toList.sorted === List(1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } + + test("groupByKey with many output partitions") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))).index() + val groups = pairs.groupByKey(10).collect() + assert(groups.size === 2) + val valuesFor1 = groups.find(_._1 == 1).get._2 + assert(valuesFor1.toList.sorted === List(1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } + + test("reduceByKey") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).index() + val sums = pairs.reduceByKey(_+_).collect() + assert(sums.toSet === Set((1, 7), (2, 1))) + } + + test("reduceByKey with collectAsMap") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).index() + val sums = pairs.reduceByKey(_+_).collectAsMap() + assert(sums.size === 2) + assert(sums(1) === 7) + assert(sums(2) === 1) + } + + test("reduceByKey with many output partitons") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).index() + val sums = pairs.reduceByKey(_+_, 10).collect() + assert(sums.toSet === Set((1, 7), (2, 1))) + } + + test("reduceByKey with partitioner") { + val p = new Partitioner() { + def numPartitions = 2 + def getPartition(key: Any) = key.asInstanceOf[Int] + } + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 1), (0, 1))).partitionBy(p).index() + val sums = pairs.reduceByKey(_+_) + assert(sums.collect().toSet === Set((1, 4), (0, 1))) + assert(sums.partitioner === Some(p)) + // count the dependencies to make sure there is only 1 ShuffledRDD + val deps = lineage(sums) + + assert(deps.filter(_.isInstanceOf[ShuffledRDD[_,_]]).size === 1) // ShuffledRDD, ParallelCollection + } + + test("join") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (2, 'x')), + (2, (1, 'y')), + (2, (1, 'z')) + )) + } + + + test("joinIndexVsPair") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).index() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (2, 'x')), + (2, (1, 'y')), + (2, (1, 'z')) + )) + } + + test("joinIndexVsIndex") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).index() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).index() + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (2, 'x')), + (2, (1, 'y')), + (2, (1, 'z')) + )) + } + + test("joinSharedIndex") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1), (4,-4), (4, 4) )).index() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).index(rdd1.index) + val joined = rdd1.join(rdd2).collect() + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (2, 'x')), + (2, (1, 'y')), + (2, (1, 'z')), + (4, (-4, 'w')), + (4, (4, 'w')) + )) + } + + + 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'))) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 6) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (1, 'y')), + (1, (2, 'x')), + (1, (2, 'y')), + (1, (3, 'x')), + (1, (3, 'y')) + )) + } + + test("leftOuterJoin") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.leftOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (1, Some('x'))), + (1, (2, Some('x'))), + (2, (1, Some('y'))), + (2, (1, Some('z'))), + (3, (1, None)) + )) + } + + test("rightOuterJoin") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.rightOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (Some(1), 'x')), + (1, (Some(2), 'x')), + (2, (Some(1), 'y')), + (2, (Some(1), 'z')), + (4, (None, 'w')) + )) + } + + test("join with no matches") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((4, 'x'), (5, 'y'), (5, 'z'), (6, 'w'))) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 0) + } + + test("join with many output partitions") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.join(rdd2, 10).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (2, 'x')), + (2, (1, 'y')), + (2, (1, 'z')) + )) + } + + test("groupWith") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + 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'))) + )) + } + + test("zero-partition RDD") { + val emptyDir = Files.createTempDir() + val file = sc.textFile(emptyDir.getAbsolutePath) + assert(file.partitions.size == 0) + assert(file.collect().toList === Nil) + // Test that a shuffle on the file works, because this used to be a bug + assert(file.map(line => (line, 1)).reduceByKey(_ + _).collect().toList === Nil) + } + + test("keys and values") { + val rdd = sc.parallelize(Array((1, "a"), (2, "b"))) + assert(rdd.keys.collect().toList === List(1, 2)) + assert(rdd.values.collect().toList === List("a", "b")) + } + + test("default partitioner uses partition size") { + // specify 2000 partitions + val a = sc.makeRDD(Array(1, 2, 3, 4), 2000) + // do a map, which loses the partitioner + val b = a.map(a => (a, (a * 2).toString)) + // then a group by, and see we didn't revert to 2 partitions + val c = b.groupByKey() + assert(c.partitions.size === 2000) + } + + test("default partitioner uses largest partitioner") { + val a = sc.makeRDD(Array((1, "a"), (2, "b")), 2) + val b = sc.makeRDD(Array((1, "a"), (2, "b")), 2000) + val c = a.join(b) + assert(c.partitions.size === 2000) + } + + test("subtract") { + val a = sc.parallelize(Array(1, 2, 3), 2) + val b = sc.parallelize(Array(2, 3, 4), 4) + val c = a.subtract(b) + assert(c.collect().toSet === Set(1)) + assert(c.partitions.size === a.partitions.size) + } + + test("subtract with narrow dependency") { + // use a deterministic partitioner + val p = new Partitioner() { + def numPartitions = 5 + def getPartition(key: Any) = key.asInstanceOf[Int] + } + // partitionBy so we have a narrow dependency + val a = sc.parallelize(Array((1, "a"), (2, "b"), (3, "c"))).partitionBy(p) + // more partitions/no partitioner so a shuffle dependency + val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) + val c = a.subtract(b) + assert(c.collect().toSet === Set((1, "a"), (3, "c"))) + // Ideally we could keep the original partitioner... + assert(c.partitioner === None) + } + + test("subtractByKey") { + val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 2) + val b = sc.parallelize(Array((2, 20), (3, 30), (4, 40)), 4) + val c = a.subtractByKey(b) + assert(c.collect().toSet === Set((1, "a"), (1, "a"))) + assert(c.partitions.size === a.partitions.size) + } + + test("subtractByKey with narrow dependency") { + // use a deterministic partitioner + val p = new Partitioner() { + def numPartitions = 5 + def getPartition(key: Any) = key.asInstanceOf[Int] + } + // partitionBy so we have a narrow dependency + val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c"))).partitionBy(p) + // more partitions/no partitioner so a shuffle dependency + val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) + val c = a.subtractByKey(b) + assert(c.collect().toSet === Set((1, "a"), (1, "a"))) + assert(c.partitioner.get === p) + } + + test("foldByKey") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val sums = pairs.foldByKey(0)(_+_).collect() + assert(sums.toSet === Set((1, 7), (2, 1))) + } + + test("foldByKey with mutable result type") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val bufs = pairs.mapValues(v => ArrayBuffer(v)).cache() + // Fold the values using in-place mutation + val sums = bufs.foldByKey(new ArrayBuffer[Int])(_ ++= _).collect() + assert(sums.toSet === Set((1, ArrayBuffer(1, 2, 3, 1)), (2, ArrayBuffer(1)))) + // Check that the mutable objects in the original RDD were not changed + assert(bufs.collect().toSet === Set( + (1, ArrayBuffer(1)), + (1, ArrayBuffer(2)), + (1, ArrayBuffer(3)), + (1, ArrayBuffer(1)), + (2, ArrayBuffer(1)))) + } +} From 61281756f225902386e8f41d12a3c58f06da3977 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 15 Aug 2013 14:20:59 -0700 Subject: [PATCH 04/13] IndexedRDD passes all PairRDD Function tests --- .../src/main/scala/spark/rdd/IndexedRDD.scala | 337 ++++++++++++++++-- .../test/scala/spark/IndexedRDDSuite.scala | 40 ++- 2 files changed, 338 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index a6852f3f8a..b65efa4447 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -17,18 +17,21 @@ package spark.rdd +import java.nio.ByteBuffer + + import java.util.{HashMap => JHashMap, BitSet => JBitSet, HashSet => JHashSet} import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer -import spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext} - -import spark.PairRDDFunctions +import spark._ +import spark.rdd._ import spark.SparkContext._ -import spark.SparkException -import spark.Partitioner +import spark.Partitioner._ + + // import java.io.{ObjectOutputStream, IOException} @@ -110,7 +113,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( val newValues = valuesRDD.mapPartitions( _.map{ groups: Seq[Seq[V]] => groups.map{ group: Seq[V] => - if(group != null && !group.isEmpty) { + if (group != null && !group.isEmpty) { val c: C = createCombiner(group.head) val sum: C = group.tail.foldLeft(c)(mergeValue) Seq(sum) @@ -123,15 +126,129 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } + /** + * Simplified version of combineByKey that hash-partitions the output RDD. + */ + def combineByKey[C: ClassManifest](createCombiner: V => C, + mergeValue: (C, V) => C, + mergeCombiners: (C, C) => C, + numPartitions: Int): IndexedRDD[K, C] = { + combineByKey(createCombiner, mergeValue, mergeCombiners, new HashPartitioner(numPartitions)) + } + + + /** + * Simplified version of combineByKey that hash-partitions the resulting RDD using the + * existing partitioner/parallelism level. + */ + def combineByKey[C: ClassManifest](createCombiner: V => C, mergeValue: (C, V) => C, + mergeCombiners: (C, C) => C) + : IndexedRDD[K, C] = { + combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(index)) + } + + + /** + * Merge the values for each key using an associative function and a neutral "zero value" which may + * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for + * list concatenation, 0 for addition, or 1 for multiplication.). + */ + def foldByKey(zeroValue: V, partitioner: Partitioner)(func: (V, V) => V): IndexedRDD[K, V] = { + // Serialize the zero value to a byte array so that we can get a new clone of it on each key + val zeroBuffer = SparkEnv.get.closureSerializer.newInstance().serialize(zeroValue) + val zeroArray = new Array[Byte](zeroBuffer.limit) + zeroBuffer.get(zeroArray) + + // When deserializing, use a lazy val to create just one instance of the serializer per task + lazy val cachedSerializer = SparkEnv.get.closureSerializer.newInstance() + def createZero() = cachedSerializer.deserialize[V](ByteBuffer.wrap(zeroArray)) + combineByKey[V]((v: V) => func(createZero(), v), func, func, partitioner) + } + + /** + * Merge the values for each key using an associative function and a neutral "zero value" which may + * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for + * list concatenation, 0 for addition, or 1 for multiplication.). + */ + def foldByKey(zeroValue: V, numPartitions: Int)(func: (V, V) => V): IndexedRDD[K, V] = { + foldByKey(zeroValue, new HashPartitioner(numPartitions))(func) + } + + + /** + * Merge the values for each key using an associative function and a neutral "zero value" which may + * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for + * list concatenation, 0 for addition, or 1 for multiplication.). + */ + def foldByKey(zeroValue: V)(func: (V, V) => V): IndexedRDD[K, V] = { + foldByKey(zeroValue, defaultPartitioner(index))(func) + } + + + /** + * Merge the values for each key using an associative reduce function. This will also perform + * the merging locally on each mapper before sending results to a reducer, similarly to a + * "combiner" in MapReduce. + */ + def reduceByKey(partitioner: Partitioner, func: (V, V) => V): IndexedRDD[K, V] = { + combineByKey[V]((v: V) => v, func, func, partitioner) + } + + /** + * Merge the values for each key using an associative reduce function. This will also perform + * the merging locally on each mapper before sending results to a reducer, similarly to a + * "combiner" in MapReduce. Output will be hash-partitioned with numPartitions partitions. + */ + def reduceByKey(func: (V, V) => V, numPartitions: Int): IndexedRDD[K, V] = { + reduceByKey(new HashPartitioner(numPartitions), func) + } + + /** + * Merge the values for each key using an associative reduce function. This will also perform + * the merging locally on each mapper before sending results to a reducer, similarly to a + * "combiner" in MapReduce. Output will be hash-partitioned with the existing partitioner/ + * parallelism level. + */ + def reduceByKey(func: (V, V) => V): IndexedRDD[K, V] = { + reduceByKey(defaultPartitioner(index), func) + } + + + /** + * 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): IndexedRDD[K, Seq[V]] = { + val newValues = valuesRDD.mapPartitions(_.map{ar => ar.map{s => Seq(s)} }, true) + new IndexedRDD[K, Seq[V]](index, newValues) + } + + + /** + * 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): IndexedRDD[K, Seq[V]] = { + groupByKey(new HashPartitioner(numPartitions)) + } + + + /** + * 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(): IndexedRDD[K, Seq[V]] = { + groupByKey(defaultPartitioner(index)) + } + /** * 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: ClassManifest](other: RDD[(K, W)], partitionerUnused: Partitioner): - IndexedRDD[K, (Seq[V], Seq[W])] = { + def cogroup[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): + IndexedRDD[K, (Seq[V], Seq[W])] = { //RDD[(K, (Seq[V], Seq[W]))] = { - assert(false) other match { case other: IndexedRDD[_, _] if other.index == index => { // if both RDDs share exactly the same index and therefore the same super set of keys @@ -174,7 +291,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( newIndex.putAll(otherIndex) // We need to rekey the index var ctr = 0 - for(e <- newIndex.entrySet) { + for (e <- newIndex.entrySet) { e.setValue(ctr) ctr += 1 } @@ -198,9 +315,9 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( // Preallocate the new Values array val newValues = new Array[Seq[(Seq[V],Seq[W])]](newIndex.size) // Lookup the sequences in both submaps - for((k,ind) <- newIndex) { - val thisSeq = if(thisIndex.contains(k)) thisValues(thisIndex.get(k)) else null - val otherSeq = if(otherIndex.contains(k)) otherValues(otherIndex.get(k)) else null + for ((k,ind) <- newIndex) { + val thisSeq = if (thisIndex.contains(k)) thisValues(thisIndex.get(k)) else null + val otherSeq = if (otherIndex.contains(k)) otherValues(otherIndex.get(k)) else null // if either of the sequences is not null then the key was in one of the two tables // and so the value should appear in the returned table newValues(ind) = (thisSeq, otherSeq) match { @@ -233,27 +350,27 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( // Get the corresponding indicies and values for this IndexedRDD val (thisIndex, thisValues) = thisTuplesIter.next() assert(!thisTuplesIter.hasNext()) + assert(thisIndex.size == thisValues.size) // Construct a new index val newIndex = thisIndex.clone().asInstanceOf[JHashMap[K, Int]] + assert(thisIndex.size == newIndex.size) // Construct a new array Buffer to store the values - val newValues = new ArrayBuffer[(Seq[V], ArrayBuffer[W])](thisValues.size) + val newValues = ArrayBuffer.fill[(Seq[V], ArrayBuffer[W])](thisValues.size)(null) // populate the newValues with the values in this IndexedRDD - for((k,i) <- thisIndex) { - if(thisValues(i) != null) { - newValues(i) = (thisValues(i), new ArrayBuffer[W]()) + for ((k,i) <- thisIndex) { + assert(i < thisValues.size) + if (thisValues(i) != null) { + newValues(i) = (thisValues(i), ArrayBuffer.empty[W]) } } // Now iterate through the other tuples updating the map - for((k,w) <- otherTuplesIter){ - if(!newIndex.contains(k)) { + for ((k,w) <- otherTuplesIter){ + if (!newIndex.contains(k)) { // update the index val ind = newIndex.size newIndex.put(k, ind) - // Create the buffer for w - val wbuffer = new ArrayBuffer[W]() - wbuffer.append(w) // Update the values - newValues.append( (Seq.empty[V], wbuffer) ) + newValues.append( (Seq.empty[V], ArrayBuffer(w) ) ) } else { val ind = newIndex.get(k) newValues(ind)._2.append(w) @@ -265,17 +382,167 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( List( (newIndex, newValuesArray) ).iterator }, otherShuffled).cache() - // Extract the index and values from the above RDD - val newIndex = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) - val newValues = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) + // Extract the index and values from the above RDD + val newIndex = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) + val newValues = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) - new IndexedRDD[K, (Seq[V], Seq[W])](newIndex, newValues) - - } + new IndexedRDD[K, (Seq[V], Seq[W])](newIndex, newValues) } } + } + /** + * 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: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Seq[V], Seq[W])] = { + cogroup(other, defaultPartitioner(this, other)) + } + + // /** + // * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a + // * 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)]) + // : IndexedRDD[K, (Seq[V], Seq[W1], Seq[W2])] = { + // cogroup(other1, other2, defaultPartitioner(this, other1, other2)) + // } + + /** + * 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: ClassManifest](other: RDD[(K, W)], numPartitions: Int): IndexedRDD[K, (Seq[V], Seq[W])] = { + cogroup(other, new HashPartitioner(numPartitions)) + } + + // /** + // * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a + // * 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]))] = { + // cogroup(other1, other2, new HashPartitioner(numPartitions)) + // } + + /** Alias for cogroup. */ + def groupWith[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Seq[V], Seq[W])] = { + cogroup(other, defaultPartitioner(this, other)) + } + + // /** Alias for cogroup. */ + // def groupWith[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) + // : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { + // cogroup(other1, other2, defaultPartitioner(self, other1, other2)) + // } + + /** + * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each + * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and + * (k, v2) is in `other`. Uses the given Partitioner to partition the output RDD. + */ + def join[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): IndexedRDD[K, (V, W)] = { + cogroup(other, partitioner).flatMapValues { + case (vs, ws) => + for (v <- vs.iterator; w <- ws.iterator) yield (v, w) + } + } + + /** + * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the + * pair (k, (v, None)) if no elements in `other` have key k. Uses the given Partitioner to + * partition the output RDD. + */ + def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): IndexedRDD[K, (V, Option[W])] = { + cogroup(other, partitioner).flatMapValues { + case (vs, ws) => + if (ws.isEmpty) { + vs.iterator.map(v => (v, None)) + } else { + for (v <- vs.iterator; w <- ws.iterator) yield (v, Some(w)) + } + } + } + + /** + * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the + * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the + * pair (k, (None, w)) if no elements in `this` have key k. Uses the given Partitioner to + * partition the output RDD. + */ + def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner) + : IndexedRDD[K, (Option[V], W)] = { + cogroup(other, partitioner).flatMapValues { + case (vs, ws) => + if (vs.isEmpty) { + ws.iterator.map(w => (None, w)) + } else { + for (v <- vs.iterator; w <- ws.iterator) yield (Some(v), w) + } + } + } + + /** + * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each + * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and + * (k, v2) is in `other`. Performs a hash join across the cluster. + */ + def join[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (V, W)] = { + join(other, defaultPartitioner(this, other)) + } + + /** + * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each + * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and + * (k, v2) is in `other`. Performs a hash join across the cluster. + */ + def join[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): IndexedRDD[K, (V, W)] = { + join(other, new HashPartitioner(numPartitions)) + } + + /** + * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the + * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output + * using the existing partitioner/parallelism level. + */ + def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (V, Option[W])] = { + leftOuterJoin(other, defaultPartitioner(this, other)) + } + + /** + * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the + * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output + * into `numPartitions` partitions. + */ + def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): IndexedRDD[K, (V, Option[W])] = { + leftOuterJoin(other, new HashPartitioner(numPartitions)) + } + + /** + * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the + * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the + * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting + * RDD using the existing partitioner/parallelism level. + */ + def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Option[V], W)] = { + rightOuterJoin(other, defaultPartitioner(this, other)) + } + + /** + * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the + * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the + * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting + * RDD into the given number of partitions. + */ + def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): IndexedRDD[K, (Option[V], W)] = { + rightOuterJoin(other, new HashPartitioner(numPartitions)) + } + + /** @@ -304,7 +571,7 @@ object IndexedRDD { tbl: RDD[(K,V)], existingIndex: RDD[JHashMap[K,Int]] = null ): IndexedRDD[K, V] = { - if(existingIndex == null) { + if (existingIndex == null) { // Shuffle the table (if necessary) val shuffledTbl = if (tbl.partitioner.isEmpty) { @@ -314,11 +581,11 @@ object IndexedRDD { val groups = shuffledTbl.mapPartitions( iter => { val indexMap = new JHashMap[K, Int]() val values = new ArrayBuffer[Seq[V]]() - for((k,v) <- iter){ + for ((k,v) <- iter){ if(!indexMap.contains(k)) { val ind = indexMap.size indexMap.put(k, ind) - values.append(new ArrayBuffer[V]()) + values.append(ArrayBuffer.empty[V]) } val ind = indexMap.get(k) values(ind).asInstanceOf[ArrayBuffer[V]].append(v) @@ -351,11 +618,11 @@ object IndexedRDD { val index: JHashMap[K,Int] = indexIter.next() assert(!indexIter.hasNext()) val values = new Array[Seq[V]](index.size) - for((k,v) <- tblIter) { + for ((k,v) <- tblIter) { assert(index.contains(k)) val ind = index(k) - if(values(ind) == null){ - values(ind) = new ArrayBuffer[V]() + if (values(ind) == null) { + values(ind) = ArrayBuffer.empty[V] } values(ind).asInstanceOf[ArrayBuffer[V]].append(v) } diff --git a/core/src/test/scala/spark/IndexedRDDSuite.scala b/core/src/test/scala/spark/IndexedRDDSuite.scala index aacb6423ee..f30e1f57fa 100644 --- a/core/src/test/scala/spark/IndexedRDDSuite.scala +++ b/core/src/test/scala/spark/IndexedRDDSuite.scala @@ -31,6 +31,8 @@ import scala.collection.mutable.HashSet import spark.rdd.ShuffledRDD import spark.SparkContext._ +import spark._ + class IndexedRDDSuite extends FunSuite with SharedSparkContext { @@ -165,6 +167,7 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1), (4,-4), (4, 4) )).index() val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).index(rdd1.index) val joined = rdd1.join(rdd2).collect() + assert(joined.size === 6) assert(joined.toSet === Set( (1, (1, 'x')), (1, (2, 'x')), @@ -177,8 +180,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { 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'))) + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (1, 3))).index() + val rdd2 = sc.parallelize(Array((1, 'x'), (1, 'y'))).index(rdd1.index) val joined = rdd1.join(rdd2).collect() assert(joined.size === 6) assert(joined.toSet === Set( @@ -191,8 +194,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { )) } - test("leftOuterJoin") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + test("leftOuterJoinIndex") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).index() val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) val joined = rdd1.leftOuterJoin(rdd2).collect() assert(joined.size === 5) @@ -205,6 +208,35 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { )) } + test("leftOuterJoinIndextoIndex") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).index() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).index() + val joined = rdd1.leftOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (1, Some('x'))), + (1, (2, Some('x'))), + (2, (1, Some('y'))), + (2, (1, Some('z'))), + (3, (1, None)) + )) + } + + test("leftOuterJoinIndextoSharedIndex") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1), (4, -4))).index() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).index(rdd1.index) + val joined = rdd1.leftOuterJoin(rdd2).collect() + assert(joined.size === 6) + assert(joined.toSet === Set( + (1, (1, Some('x'))), + (1, (2, Some('x'))), + (2, (1, Some('y'))), + (2, (1, Some('z'))), + (4, (-4, Some('w'))), + (3, (1, None)) + )) + } + test("rightOuterJoin") { val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) From 3bb6e019d4eee5cc80b6b506951ce9efc4f21ed2 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 15 Aug 2013 14:29:48 -0700 Subject: [PATCH 05/13] adding better error handling when indexing an RDD --- core/src/main/scala/spark/rdd/IndexedRDD.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index b65efa4447..7d8e0d3938 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -350,15 +350,12 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( // Get the corresponding indicies and values for this IndexedRDD val (thisIndex, thisValues) = thisTuplesIter.next() assert(!thisTuplesIter.hasNext()) - assert(thisIndex.size == thisValues.size) // Construct a new index val newIndex = thisIndex.clone().asInstanceOf[JHashMap[K, Int]] - assert(thisIndex.size == newIndex.size) // Construct a new array Buffer to store the values val newValues = ArrayBuffer.fill[(Seq[V], ArrayBuffer[W])](thisValues.size)(null) // populate the newValues with the values in this IndexedRDD for ((k,i) <- thisIndex) { - assert(i < thisValues.size) if (thisValues(i) != null) { newValues(i) = (thisValues(i), ArrayBuffer.empty[W]) } @@ -619,7 +616,10 @@ object IndexedRDD { assert(!indexIter.hasNext()) val values = new Array[Seq[V]](index.size) for ((k,v) <- tblIter) { - assert(index.contains(k)) + if (!index.contains(k)) { + throw new SparkException("Error: Try to bind an external index " + + "to an RDD which contains keys that are not in the index.") + } val ind = index(k) if (values(ind) == null) { values(ind) = ArrayBuffer.empty[V] From 327a4db9f7546c58d93e0ccd38a6d131aee98c78 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 15 Aug 2013 16:36:26 -0700 Subject: [PATCH 06/13] changing caching behavior on indexedrdds --- core/src/main/scala/spark/rdd/IndexedRDD.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index 7d8e0d3938..6b1cb7608b 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -58,6 +58,12 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( override def getPartitions: Array[Partition] = tuples.getPartitions override def getPreferredLocations(s: Partition): Seq[String] = tuples.getPreferredLocations(s) + override def cache: IndexedRDD[K,V] = { + index.cache + valuesRDD.cache + return this + } + /** * Construct a new IndexedRDD that is indexed by only the keys in the RDD From 8fd37adf832a7cd234dca64ea7679668b89fafa8 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sun, 18 Aug 2013 10:57:35 -0700 Subject: [PATCH 07/13] Merged with changes to zipPartitions --- .../src/main/scala/spark/rdd/IndexedRDD.scala | 25 +++++++++---------- 1 file changed, 12 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index 6b1cb7608b..37ff11e639 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -262,7 +262,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( // However it is possible that both RDDs are missing a value for a given key in // which case the returned RDD should have a null value val newValues = - valuesRDD.zipPartitions[ Seq[Seq[W]], Seq[Seq[(Seq[V], Seq[W])]] ]( + valuesRDD.zipPartitions(other.valuesRDD)( (thisIter, otherIter) => { val thisValues: Seq[Seq[V]] = thisIter.next() assert(!thisIter.hasNext()) @@ -277,7 +277,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( case (a, b) => Seq((a,b)) }.toSeq List(tmp).iterator - }, other.valuesRDD) + }) new IndexedRDD[K, (Seq[V], Seq[W])](index, newValues) } case other: IndexedRDD[_, _] if other.index.partitioner == index.partitioner => { @@ -285,7 +285,8 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( // then we we need to first merge the indicies and then use the merged index to // merge the values. val newIndex = - index.zipPartitions[JHashMap[K,Int], JHashMap[K,Int]]( (thisIter, otherIter) => { + index.zipPartitions(other.index)( + (thisIter, otherIter) => { val thisIndex = thisIter.next() assert(!thisIter.hasNext()) val otherIndex = otherIter.next() @@ -302,13 +303,10 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( ctr += 1 } List(newIndex).iterator - }, other.index).cache() + }).cache() // Use the new index along with the this and the other indices to merge the values val newValues = - newIndex.zipPartitions[ - (JHashMap[K, Int], Seq[Seq[V]]), - (JHashMap[K, Int], Seq[Seq[W]]), - Seq[Seq[(Seq[V],Seq[W])]] ]( + newIndex.zipPartitions(tuples, other.tuples)( (newIndexIter, thisTuplesIter, otherTuplesIter) => { // Get the new index for this partition val newIndex = newIndexIter.next() @@ -334,7 +332,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } } List(newValues.toSeq).iterator - }, tuples, other.tuples) + }) new IndexedRDD(newIndex, newValues) } case _ => { @@ -351,7 +349,8 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( new ShuffledRDD[K,W](other, partitioner) } // Join the other RDD with this RDD building a new valueset and new index on the fly - val groups = tuples.zipPartitions[(K, W), (JHashMap[K, Int], Seq[Seq[(Seq[V],Seq[W])]]) ]( + val groups = + tuples.zipPartitions(otherShuffled)( (thisTuplesIter, otherTuplesIter) => { // Get the corresponding indicies and values for this IndexedRDD val (thisIndex, thisValues) = thisTuplesIter.next() @@ -383,7 +382,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( val newValuesArray: Seq[Seq[(Seq[V],Seq[W])]] = newValues.view.map{ case (s, ab) => Seq((s, ab.toSeq)) }.toSeq List( (newIndex, newValuesArray) ).iterator - }, otherShuffled).cache() + }).cache() // Extract the index and values from the above RDD val newIndex = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) @@ -615,7 +614,7 @@ object IndexedRDD { } // Use the index to build the new values table - val values = index.zipPartitions[ (K, V), Seq[Seq[V]] ]( + val values = index.zipPartitions(shuffledTbl)( (indexIter, tblIter) => { // There is only one map val index: JHashMap[K,Int] = indexIter.next() @@ -633,7 +632,7 @@ object IndexedRDD { values(ind).asInstanceOf[ArrayBuffer[V]].append(v) } List(values.toSeq).iterator - }, shuffledTbl) + }) new IndexedRDD[K,V](index, values) } From 630281bf7671ffec65a4672361d3d9570cfe7a39 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sun, 18 Aug 2013 17:16:45 -0700 Subject: [PATCH 08/13] Corrected all indexed RDD tests. There appears to be an issue with subtract by key tests that needs to be investigated further. --- .../main/scala/spark/PairRDDFunctions.scala | 9 +- core/src/main/scala/spark/RDD.scala | 9 + .../src/main/scala/spark/rdd/IndexedRDD.scala | 316 ++++++------------ .../test/scala/spark/IndexedRDDSuite.scala | 206 ++++++++---- 4 files changed, 266 insertions(+), 274 deletions(-) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 469e870409..6751da72af 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -699,7 +699,14 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( def values: RDD[V] = self.map(_._2) - def index(existingIndex: RDD[JHashMap[K,Int]] = null): IndexedRDD[K,V] = + def indexed(numPartitions: Int): IndexedRDD[K,V] = + IndexedRDD(self.partitionBy(new HashPartitioner(numPartitions))) + + def indexed(partitioner: Partitioner): IndexedRDD[K,V] = + IndexedRDD(self.partitionBy(partitioner)) + + + def indexed(existingIndex: RDD[BlockIndex[K]] = null): IndexedRDD[K,V] = IndexedRDD(self, existingIndex) private[spark] def getKeyClass() = implicitly[ClassManifest[K]].erasure diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 503ea6ccbf..1c5095fa86 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -54,6 +54,9 @@ import spark.rdd.ZippedRDD import spark.rdd.ZippedPartitionsRDD2 import spark.rdd.ZippedPartitionsRDD3 import spark.rdd.ZippedPartitionsRDD4 + +import spark.rdd.{IndexedRDD, BlockIndex} + import spark.storage.StorageLevel import spark.util.BoundedPriorityQueue @@ -770,6 +773,12 @@ abstract class RDD[T: ClassManifest]( return buf.toArray } + + def makeIndex(partitioner: Option[Partitioner] = None): RDD[BlockIndex[T]] = + IndexedRDD.makeIndex(this, partitioner) + + + /** * Return the first element in this RDD. */ diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index 37ff11e639..b2a2180b8f 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -34,7 +34,13 @@ import spark.Partitioner._ -// import java.io.{ObjectOutputStream, IOException} +/** + * And index RDD + */ +class BlockIndex[@specialized K: ClassManifest] extends JHashMap[K,Int] + +//type BlockIndex[@specialized K: ClassManifest] = JHashMap[K,Int] + /** * An IndexedRDD is an RDD[(K,V)] where each K is unique. @@ -43,20 +49,26 @@ import spark.Partitioner._ * be used to accelerate join and aggregation operations. */ class IndexedRDD[K: ClassManifest, V: ClassManifest]( - val index: RDD[ JHashMap[K, Int] ], + val index: RDD[ BlockIndex[K] ], val valuesRDD: RDD[ Seq[Seq[V]] ]) extends RDD[(K, V)](index.context, List(new OneToOneDependency(index), new OneToOneDependency(valuesRDD)) ) { - //with PairRDDFunctions[K,V] { - - - val tuples = new ZippedRDD[JHashMap[K, Int], Seq[Seq[V]]](index.context, index, valuesRDD) + /** + * An internal representation of the maps and block managers + */ + protected val tuples = new ZippedRDD(index.context, index, valuesRDD) override val partitioner = index.partitioner + + override def getPartitions: Array[Partition] = tuples.getPartitions - override def getPreferredLocations(s: Partition): Seq[String] = tuples.getPreferredLocations(s) + + + override def getPreferredLocations(s: Partition): Seq[String] = + tuples.getPreferredLocations(s) + override def cache: IndexedRDD[K,V] = { index.cache @@ -77,9 +89,10 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( */ def mapValues[U: ClassManifest](f: V => U): IndexedRDD[K, U] = { val cleanF = index.context.clean(f) - val newValues = valuesRDD.mapPartitions(_.map{ values => - values.map{_.map(x => f(x))} - }, true) + val newValues = valuesRDD.mapPartitions(_.map(values => values.map{ + case null => null + case row => row.map(x => f(x)) + }), true) new IndexedRDD[K,U](index, newValues) } @@ -90,9 +103,10 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( */ def flatMapValues[U: ClassManifest](f: V => TraversableOnce[U]): IndexedRDD[K,U] = { val cleanF = index.context.clean(f) - val newValues = valuesRDD.mapPartitions(_.map{ values => - values.map{_.flatMap(x => f(x))} - }, true) + val newValues = valuesRDD.mapPartitions(_.map(values => values.map{ + case null => null + case row => row.flatMap(x => f(x)) + }), true) new IndexedRDD[K,U](index, newValues) } @@ -106,15 +120,10 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( * - `createCombiner`, which turns a V into a C (e.g., creates a one-element list) * - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list) * - `mergeCombiners`, to combine two C's into a single one. - * - * In addition, users can control the partitioning of the output RDD, and whether to perform - * map-side aggregation (if a mapper can produce multiple items with the same key). */ def combineByKey[C: ClassManifest](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, - partitioner: Partitioner, - mapSideCombine: Boolean = true, serializerClass: String = null): IndexedRDD[K, C] = { val newValues = valuesRDD.mapPartitions( _.map{ groups: Seq[Seq[V]] => @@ -131,35 +140,12 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( new IndexedRDD[K,C](index, newValues) } - - /** - * Simplified version of combineByKey that hash-partitions the output RDD. - */ - def combineByKey[C: ClassManifest](createCombiner: V => C, - mergeValue: (C, V) => C, - mergeCombiners: (C, C) => C, - numPartitions: Int): IndexedRDD[K, C] = { - combineByKey(createCombiner, mergeValue, mergeCombiners, new HashPartitioner(numPartitions)) - } - - - /** - * Simplified version of combineByKey that hash-partitions the resulting RDD using the - * existing partitioner/parallelism level. - */ - def combineByKey[C: ClassManifest](createCombiner: V => C, mergeValue: (C, V) => C, - mergeCombiners: (C, C) => C) - : IndexedRDD[K, C] = { - combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(index)) - } - - /** * Merge the values for each key using an associative function and a neutral "zero value" which may * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for * list concatenation, 0 for addition, or 1 for multiplication.). */ - def foldByKey(zeroValue: V, partitioner: Partitioner)(func: (V, V) => V): IndexedRDD[K, V] = { + def foldByKey(zeroValue: V)(func: (V, V) => V): IndexedRDD[K, V] = { // Serialize the zero value to a byte array so that we can get a new clone of it on each key val zeroBuffer = SparkEnv.get.closureSerializer.newInstance().serialize(zeroValue) val zeroArray = new Array[Byte](zeroBuffer.limit) @@ -168,45 +154,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( // When deserializing, use a lazy val to create just one instance of the serializer per task lazy val cachedSerializer = SparkEnv.get.closureSerializer.newInstance() def createZero() = cachedSerializer.deserialize[V](ByteBuffer.wrap(zeroArray)) - combineByKey[V]((v: V) => func(createZero(), v), func, func, partitioner) - } - - /** - * Merge the values for each key using an associative function and a neutral "zero value" which may - * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for - * list concatenation, 0 for addition, or 1 for multiplication.). - */ - def foldByKey(zeroValue: V, numPartitions: Int)(func: (V, V) => V): IndexedRDD[K, V] = { - foldByKey(zeroValue, new HashPartitioner(numPartitions))(func) - } - - - /** - * Merge the values for each key using an associative function and a neutral "zero value" which may - * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for - * list concatenation, 0 for addition, or 1 for multiplication.). - */ - def foldByKey(zeroValue: V)(func: (V, V) => V): IndexedRDD[K, V] = { - foldByKey(zeroValue, defaultPartitioner(index))(func) - } - - - /** - * Merge the values for each key using an associative reduce function. This will also perform - * the merging locally on each mapper before sending results to a reducer, similarly to a - * "combiner" in MapReduce. - */ - def reduceByKey(partitioner: Partitioner, func: (V, V) => V): IndexedRDD[K, V] = { - combineByKey[V]((v: V) => v, func, func, partitioner) - } - - /** - * Merge the values for each key using an associative reduce function. This will also perform - * the merging locally on each mapper before sending results to a reducer, similarly to a - * "combiner" in MapReduce. Output will be hash-partitioned with numPartitions partitions. - */ - def reduceByKey(func: (V, V) => V, numPartitions: Int): IndexedRDD[K, V] = { - reduceByKey(new HashPartitioner(numPartitions), func) + combineByKey[V]((v: V) => func(createZero(), v), func, func) } /** @@ -216,26 +164,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( * parallelism level. */ def reduceByKey(func: (V, V) => V): IndexedRDD[K, V] = { - reduceByKey(defaultPartitioner(index), func) - } - - - /** - * 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): IndexedRDD[K, Seq[V]] = { - val newValues = valuesRDD.mapPartitions(_.map{ar => ar.map{s => Seq(s)} }, true) - new IndexedRDD[K, Seq[V]](index, newValues) - } - - - /** - * 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): IndexedRDD[K, Seq[V]] = { - groupByKey(new HashPartitioner(numPartitions)) + combineByKey[V]((v: V) => v, func, func) } @@ -244,7 +173,8 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( * resulting RDD with the existing partitioner/parallelism level. */ def groupByKey(): IndexedRDD[K, Seq[V]] = { - groupByKey(defaultPartitioner(index)) + val newValues = valuesRDD.mapPartitions(_.map{ar => ar.map{s => Seq(s)} }, true) + new IndexedRDD[K, Seq[V]](index, newValues) } @@ -252,8 +182,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( * 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: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): - IndexedRDD[K, (Seq[V], Seq[W])] = { + def cogroup[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Seq[V], Seq[W])] = { //RDD[(K, (Seq[V], Seq[W]))] = { other match { case other: IndexedRDD[_, _] if other.index == index => { @@ -291,7 +220,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( assert(!thisIter.hasNext()) val otherIndex = otherIter.next() assert(!otherIter.hasNext()) - val newIndex = new JHashMap[K, Int]() + val newIndex = new BlockIndex[K]() // @todo Merge only the keys that correspond to non-null values // Merge the keys newIndex.putAll(thisIndex) @@ -356,9 +285,9 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( val (thisIndex, thisValues) = thisTuplesIter.next() assert(!thisTuplesIter.hasNext()) // Construct a new index - val newIndex = thisIndex.clone().asInstanceOf[JHashMap[K, Int]] + val newIndex = thisIndex.clone().asInstanceOf[BlockIndex[K]] // Construct a new array Buffer to store the values - val newValues = ArrayBuffer.fill[(Seq[V], ArrayBuffer[W])](thisValues.size)(null) + val newValues = ArrayBuffer.fill[(Seq[V], Seq[W])](thisValues.size)(null) // populate the newValues with the values in this IndexedRDD for ((k,i) <- thisIndex) { if (thisValues(i) != null) { @@ -375,12 +304,21 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( newValues.append( (Seq.empty[V], ArrayBuffer(w) ) ) } else { val ind = newIndex.get(k) - newValues(ind)._2.append(w) + if(newValues(ind) == null) { + // If the other key was in the index but not in the values + // of this indexed RDD then create a new values entry for it + newValues(ind) = (Seq.empty[V], ArrayBuffer(w)) + } else { + newValues(ind)._2.asInstanceOf[ArrayBuffer[W]].append(w) + } } } // Finalize the new values array val newValuesArray: Seq[Seq[(Seq[V],Seq[W])]] = - newValues.view.map{ case (s, ab) => Seq((s, ab.toSeq)) }.toSeq + newValues.view.map{ + case null => null + case (s, ab) => Seq((s, ab.toSeq)) + }.toSeq List( (newIndex, newValuesArray) ).iterator }).cache() @@ -394,13 +332,6 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } - /** - * 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: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Seq[V], Seq[W])] = { - cogroup(other, defaultPartitioner(this, other)) - } // /** // * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a @@ -411,14 +342,6 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( // cogroup(other1, other2, defaultPartitioner(this, other1, other2)) // } - /** - * 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: ClassManifest](other: RDD[(K, W)], numPartitions: Int): IndexedRDD[K, (Seq[V], Seq[W])] = { - cogroup(other, new HashPartitioner(numPartitions)) - } - // /** // * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a // * tuple with the list of values for that key in `this`, `other1` and `other2`. @@ -430,7 +353,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( /** Alias for cogroup. */ def groupWith[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Seq[V], Seq[W])] = { - cogroup(other, defaultPartitioner(this, other)) + cogroup(other) } // /** Alias for cogroup. */ @@ -439,52 +362,6 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( // cogroup(other1, other2, defaultPartitioner(self, other1, other2)) // } - /** - * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each - * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and - * (k, v2) is in `other`. Uses the given Partitioner to partition the output RDD. - */ - def join[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): IndexedRDD[K, (V, W)] = { - cogroup(other, partitioner).flatMapValues { - case (vs, ws) => - for (v <- vs.iterator; w <- ws.iterator) yield (v, w) - } - } - - /** - * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the - * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the - * pair (k, (v, None)) if no elements in `other` have key k. Uses the given Partitioner to - * partition the output RDD. - */ - def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): IndexedRDD[K, (V, Option[W])] = { - cogroup(other, partitioner).flatMapValues { - case (vs, ws) => - if (ws.isEmpty) { - vs.iterator.map(v => (v, None)) - } else { - for (v <- vs.iterator; w <- ws.iterator) yield (v, Some(w)) - } - } - } - - /** - * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the - * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the - * pair (k, (None, w)) if no elements in `this` have key k. Uses the given Partitioner to - * partition the output RDD. - */ - def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner) - : IndexedRDD[K, (Option[V], W)] = { - cogroup(other, partitioner).flatMapValues { - case (vs, ws) => - if (vs.isEmpty) { - ws.iterator.map(w => (None, w)) - } else { - for (v <- vs.iterator; w <- ws.iterator) yield (Some(v), w) - } - } - } /** * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each @@ -492,17 +369,12 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( * (k, v2) is in `other`. Performs a hash join across the cluster. */ def join[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (V, W)] = { - join(other, defaultPartitioner(this, other)) + cogroup(other).flatMapValues { + case (vs, ws) => + for (v <- vs.iterator; w <- ws.iterator) yield (v, w) + } } - /** - * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each - * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and - * (k, v2) is in `other`. Performs a hash join across the cluster. - */ - def join[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): IndexedRDD[K, (V, W)] = { - join(other, new HashPartitioner(numPartitions)) - } /** * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the @@ -511,18 +383,17 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( * using the existing partitioner/parallelism level. */ def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (V, Option[W])] = { - leftOuterJoin(other, defaultPartitioner(this, other)) + cogroup(other).flatMapValues { + case (vs, ws) => + if (ws.isEmpty) { + vs.iterator.map(v => (v, None)) + } else { + for (v <- vs.iterator; w <- ws.iterator) yield (v, Some(w)) + } + } + } - /** - * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the - * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the - * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output - * into `numPartitions` partitions. - */ - def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): IndexedRDD[K, (V, Option[W])] = { - leftOuterJoin(other, new HashPartitioner(numPartitions)) - } /** * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the @@ -531,21 +402,17 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( * RDD using the existing partitioner/parallelism level. */ def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Option[V], W)] = { - rightOuterJoin(other, defaultPartitioner(this, other)) + cogroup(other).flatMapValues { + case (vs, ws) => + if (vs.isEmpty) { + ws.iterator.map(w => (None, w)) + } else { + for (v <- vs.iterator; w <- ws.iterator) yield (Some(v), w) + } + } + } - /** - * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the - * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the - * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting - * RDD into the given number of partitions. - */ - def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): IndexedRDD[K, (Option[V], W)] = { - rightOuterJoin(other, new HashPartitioner(numPartitions)) - } - - - /** * Provide the RDD[(K,V)] equivalent output. @@ -571,7 +438,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( object IndexedRDD { def apply[K: ClassManifest, V: ClassManifest]( tbl: RDD[(K,V)], - existingIndex: RDD[JHashMap[K,Int]] = null ): IndexedRDD[K, V] = { + existingIndex: RDD[BlockIndex[K]] = null ): IndexedRDD[K, V] = { if (existingIndex == null) { // Shuffle the table (if necessary) @@ -581,7 +448,7 @@ object IndexedRDD { } else { tbl } val groups = shuffledTbl.mapPartitions( iter => { - val indexMap = new JHashMap[K, Int]() + val indexMap = new BlockIndex[K]() val values = new ArrayBuffer[Seq[V]]() for ((k,v) <- iter){ if(!indexMap.contains(k)) { @@ -617,12 +484,12 @@ object IndexedRDD { val values = index.zipPartitions(shuffledTbl)( (indexIter, tblIter) => { // There is only one map - val index: JHashMap[K,Int] = indexIter.next() + val index = indexIter.next() assert(!indexIter.hasNext()) val values = new Array[Seq[V]](index.size) for ((k,v) <- tblIter) { if (!index.contains(k)) { - throw new SparkException("Error: Try to bind an external index " + + throw new SparkException("Error: Trying to bind an external index " + "to an RDD which contains keys that are not in the index.") } val ind = index(k) @@ -638,6 +505,41 @@ object IndexedRDD { } } + /** + * Construct and index of the unique values in a given RDD. + */ + def makeIndex[K: ClassManifest](keys: RDD[K], + partitioner: Option[Partitioner] = None): RDD[BlockIndex[K]] = { + + + // Ugly hack :-(. In order to partition the keys they must have values. + val tbl = keys.mapPartitions(_.map(k => (k, false)), true) + // Shuffle the table (if necessary) + val shuffledTbl = partitioner match { + case None => { + if (tbl.partitioner.isEmpty) { + new ShuffledRDD[K, Boolean](tbl, Partitioner.defaultPartitioner(tbl)) + } else { tbl } + } + case Some(partitioner) => + tbl.partitionBy(partitioner) +// new ShuffledRDD[K, Boolean](tbl, partitioner) + } + + + val index = shuffledTbl.mapPartitions( iter => { + val indexMap = new BlockIndex[K]() + for ( (k,_) <- iter ){ + if(!indexMap.contains(k)){ + val ind = indexMap.size + indexMap.put(k, ind) + } + } + List(indexMap).iterator + }, true).cache + index + } + } diff --git a/core/src/test/scala/spark/IndexedRDDSuite.scala b/core/src/test/scala/spark/IndexedRDDSuite.scala index f30e1f57fa..c1433c5095 100644 --- a/core/src/test/scala/spark/IndexedRDDSuite.scala +++ b/core/src/test/scala/spark/IndexedRDDSuite.scala @@ -30,10 +30,13 @@ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashSet import spark.rdd.ShuffledRDD +import spark.rdd.IndexedRDD + import spark.SparkContext._ import spark._ + class IndexedRDDSuite extends FunSuite with SharedSparkContext { def lineage(rdd: RDD[_]): collection.mutable.HashSet[RDD[_]] = { @@ -49,7 +52,7 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("groupByKey") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))).index() + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))).indexed() val groups = pairs.groupByKey().collect() assert(groups.size === 2) val valuesFor1 = groups.find(_._1 == 1).get._2 @@ -59,7 +62,7 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("groupByKey with duplicates") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).index() + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed() val groups = pairs.groupByKey().collect() assert(groups.size === 2) val valuesFor1 = groups.find(_._1 == 1).get._2 @@ -69,7 +72,7 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("groupByKey with negative key hash codes") { - val pairs = sc.parallelize(Array((-1, 1), (-1, 2), (-1, 3), (2, 1))).index() + val pairs = sc.parallelize(Array((-1, 1), (-1, 2), (-1, 3), (2, 1))).indexed() val groups = pairs.groupByKey().collect() assert(groups.size === 2) val valuesForMinus1 = groups.find(_._1 == -1).get._2 @@ -79,8 +82,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("groupByKey with many output partitions") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))).index() - val groups = pairs.groupByKey(10).collect() + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))).indexed(10) + val groups = pairs.groupByKey().collect() assert(groups.size === 2) val valuesFor1 = groups.find(_._1 == 1).get._2 assert(valuesFor1.toList.sorted === List(1, 2, 3)) @@ -89,13 +92,13 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("reduceByKey") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).index() + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed() val sums = pairs.reduceByKey(_+_).collect() assert(sums.toSet === Set((1, 7), (2, 1))) } test("reduceByKey with collectAsMap") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).index() + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed() val sums = pairs.reduceByKey(_+_).collectAsMap() assert(sums.size === 2) assert(sums(1) === 7) @@ -103,8 +106,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("reduceByKey with many output partitons") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).index() - val sums = pairs.reduceByKey(_+_, 10).collect() + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed(10) + val sums = pairs.reduceByKey(_+_).collect() assert(sums.toSet === Set((1, 7), (2, 1))) } @@ -113,7 +116,7 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { def numPartitions = 2 def getPartition(key: Any) = key.asInstanceOf[Int] } - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 1), (0, 1))).partitionBy(p).index() + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 1), (0, 1))).indexed(p) val sums = pairs.reduceByKey(_+_) assert(sums.collect().toSet === Set((1, 4), (0, 1))) assert(sums.partitioner === Some(p)) @@ -123,22 +126,10 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { assert(deps.filter(_.isInstanceOf[ShuffledRDD[_,_]]).size === 1) // ShuffledRDD, ParallelCollection } - test("join") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) - val joined = rdd1.join(rdd2).collect() - assert(joined.size === 4) - assert(joined.toSet === Set( - (1, (1, 'x')), - (1, (2, 'x')), - (2, (1, 'y')), - (2, (1, 'z')) - )) - } test("joinIndexVsPair") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).index() + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed() val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) val joined = rdd1.join(rdd2).collect() assert(joined.size === 4) @@ -151,8 +142,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("joinIndexVsIndex") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).index() - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).index() + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed() val joined = rdd1.join(rdd2).collect() assert(joined.size === 4) assert(joined.toSet === Set( @@ -164,8 +155,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("joinSharedIndex") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1), (4,-4), (4, 4) )).index() - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).index(rdd1.index) + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1), (4,-4), (4, 4) )).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(rdd1.index) val joined = rdd1.join(rdd2).collect() assert(joined.size === 6) assert(joined.toSet === Set( @@ -180,8 +171,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { test("join all-to-all") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (1, 3))).index() - val rdd2 = sc.parallelize(Array((1, 'x'), (1, 'y'))).index(rdd1.index) + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (1, 3))).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (1, 'y'))).indexed(rdd1.index) val joined = rdd1.join(rdd2).collect() assert(joined.size === 6) assert(joined.toSet === Set( @@ -195,7 +186,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("leftOuterJoinIndex") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).index() + val index = sc.parallelize( 1 to 6 ).makeIndex() + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) val joined = rdd1.leftOuterJoin(rdd2).collect() assert(joined.size === 5) @@ -209,8 +201,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("leftOuterJoinIndextoIndex") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).index() - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).index() + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed() val joined = rdd1.leftOuterJoin(rdd2).collect() assert(joined.size === 5) assert(joined.toSet === Set( @@ -223,8 +215,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("leftOuterJoinIndextoSharedIndex") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1), (4, -4))).index() - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).index(rdd1.index) + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1), (4, -4))).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(rdd1.index) val joined = rdd1.leftOuterJoin(rdd2).collect() assert(joined.size === 6) assert(joined.toSet === Set( @@ -237,8 +229,25 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { )) } +test("leftOuterJoinIndextoIndexExternal") { + val index = sc.parallelize( 1 to 6 ).makeIndex() + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(index) + val joined = rdd1.leftOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (1, Some('x'))), + (1, (2, Some('x'))), + (2, (1, Some('y'))), + (2, (1, Some('z'))), + (3, (1, None)) + )) + } + + test("rightOuterJoin") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val index = sc.parallelize( 1 to 6 ).makeIndex() + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) val joined = rdd1.rightOuterJoin(rdd2).collect() assert(joined.size === 5) @@ -251,17 +260,58 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { )) } - test("join with no matches") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + test("rightOuterJoinIndex2Index") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed() + val joined = rdd1.rightOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (Some(1), 'x')), + (1, (Some(2), 'x')), + (2, (Some(1), 'y')), + (2, (Some(1), 'z')), + (4, (None, 'w')) + )) + } + + + test("rightOuterJoinIndex2Indexshared") { + val index = sc.parallelize( 1 to 6 ).makeIndex() + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(index) + val joined = rdd1.rightOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (Some(1), 'x')), + (1, (Some(2), 'x')), + (2, (Some(1), 'y')), + (2, (Some(1), 'z')), + (4, (None, 'w')) + )) + } + + + test("join with no matches index") { + val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) val rdd2 = sc.parallelize(Array((4, 'x'), (5, 'y'), (5, 'z'), (6, 'w'))) val joined = rdd1.join(rdd2).collect() assert(joined.size === 0) } + test("join with no matches shared index") { + val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) + val rdd2 = sc.parallelize(Array((4, 'x'), (5, 'y'), (5, 'z'), (6, 'w'))).indexed(index) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 0) + } + + test("join with many output partitions") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(10) val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) - val joined = rdd1.join(rdd2, 10).collect() + val joined = rdd1.join(rdd2).collect() assert(joined.size === 4) assert(joined.toSet === Set( (1, (1, 'x')), @@ -271,9 +321,25 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { )) } + test("join with many output partitions and two indices") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(10) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(20) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (2, 'x')), + (2, (1, 'y')), + (2, (1, 'z')) + )) + } + + test("groupWith") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) + + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(index) val joined = rdd1.groupWith(rdd2).collect() assert(joined.size === 4) assert(joined.toSet === Set( @@ -294,7 +360,7 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("keys and values") { - val rdd = sc.parallelize(Array((1, "a"), (2, "b"))) + val rdd = sc.parallelize(Array((1, "a"), (2, "b"))).indexed() assert(rdd.keys.collect().toList === List(1, 2)) assert(rdd.values.collect().toList === List("a", "b")) } @@ -309,12 +375,14 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { assert(c.partitions.size === 2000) } - test("default partitioner uses largest partitioner") { - val a = sc.makeRDD(Array((1, "a"), (2, "b")), 2) - val b = sc.makeRDD(Array((1, "a"), (2, "b")), 2000) - val c = a.join(b) - assert(c.partitions.size === 2000) - } + // test("default partitioner uses largest partitioner indexed to indexed") { + // val a = sc.makeRDD(Array((1, "a"), (2, "b")), 2).indexed() + // val b = sc.makeRDD(Array((1, "a"), (2, "b")), 2000).indexed() + // val c = a.join(b) + // assert(c.partitions.size === 2000) + // } + + test("subtract") { val a = sc.parallelize(Array(1, 2, 3), 2) @@ -331,7 +399,7 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { def getPartition(key: Any) = key.asInstanceOf[Int] } // partitionBy so we have a narrow dependency - val a = sc.parallelize(Array((1, "a"), (2, "b"), (3, "c"))).partitionBy(p) + val a = sc.parallelize(Array((1, "a"), (2, "b"), (3, "c"))).indexed(p) // more partitions/no partitioner so a shuffle dependency val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) val c = a.subtract(b) @@ -341,36 +409,42 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("subtractByKey") { - val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 2) + + val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 2).indexed() val b = sc.parallelize(Array((2, 20), (3, 30), (4, 40)), 4) val c = a.subtractByKey(b) assert(c.collect().toSet === Set((1, "a"), (1, "a"))) assert(c.partitions.size === a.partitions.size) } - test("subtractByKey with narrow dependency") { - // use a deterministic partitioner - val p = new Partitioner() { - def numPartitions = 5 - def getPartition(key: Any) = key.asInstanceOf[Int] - } - // partitionBy so we have a narrow dependency - val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c"))).partitionBy(p) - // more partitions/no partitioner so a shuffle dependency - val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) - val c = a.subtractByKey(b) - assert(c.collect().toSet === Set((1, "a"), (1, "a"))) - assert(c.partitioner.get === p) - } + // test("subtractByKey with narrow dependency") { + // // use a deterministic partitioner + // val p = new Partitioner() { + // def numPartitions = 5 + // def getPartition(key: Any) = key.asInstanceOf[Int] + // } + + // val index = sc.parallelize( 1 to 6 ).makeIndex(Some(p)) + // // partitionBy so we have a narrow dependency + // val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c"))).indexed(index) + // // more partitions/no partitioner so a shuffle dependency + // val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4).indexed(index) + // val c = a.subtractByKey(b) + // assert(c.collect().toSet === Set((1, "a"), (1, "a"))) + // assert(c.partitioner.get === p) + // } test("foldByKey") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed(index) val sums = pairs.foldByKey(0)(_+_).collect() assert(sums.toSet === Set((1, 7), (2, 1))) } test("foldByKey with mutable result type") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) + + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed(index) val bufs = pairs.mapValues(v => ArrayBuffer(v)).cache() // Fold the values using in-place mutation val sums = bufs.foldByKey(new ArrayBuffer[Int])(_ ++= _).collect() From 023702c90da7c421e1e339e14878bccebbb26e24 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Mon, 19 Aug 2013 13:32:54 -0700 Subject: [PATCH 09/13] Updating documentation. --- core/src/main/scala/spark/rdd/IndexedRDD.scala | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index b2a2180b8f..d19d220ec2 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -35,18 +35,20 @@ import spark.Partitioner._ /** - * And index RDD + * The BlockIndex is the internal map structure used inside the index + * of the IndexedRDD. */ class BlockIndex[@specialized K: ClassManifest] extends JHashMap[K,Int] -//type BlockIndex[@specialized K: ClassManifest] = JHashMap[K,Int] /** - * An IndexedRDD is an RDD[(K,V)] where each K is unique. + * An IndexedRDD[K,V] extends the RDD[(K,V)] by pre-indexing the keys and + * organizing the values to enable faster join operations. + * + * In addition to providing the basic RDD[(K,V)] functionality the IndexedRDD + * exposes an index member which can be used to "key" other IndexedRDDs * - * The IndexedRDD contains an index datastructure that can - * be used to accelerate join and aggregation operations. */ class IndexedRDD[K: ClassManifest, V: ClassManifest]( val index: RDD[ BlockIndex[K] ], From 55c6e73bfbd29358226d7fb6b7f753d1b05cd9bc Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Mon, 19 Aug 2013 17:35:51 -0700 Subject: [PATCH 10/13] Wrapping the index in an opaque type. --- .../main/scala/spark/PairRDDFunctions.scala | 2 +- core/src/main/scala/spark/RDD.scala | 4 +- .../src/main/scala/spark/rdd/IndexedRDD.scala | 75 +++++++++++++------ 3 files changed, 55 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 6751da72af..03f25040a1 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -706,7 +706,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( IndexedRDD(self.partitionBy(partitioner)) - def indexed(existingIndex: RDD[BlockIndex[K]] = null): IndexedRDD[K,V] = + def indexed(existingIndex: RDDIndex[K] = null): IndexedRDD[K,V] = IndexedRDD(self, existingIndex) private[spark] def getKeyClass() = implicitly[ClassManifest[K]].erasure diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 1c5095fa86..1664471524 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -55,7 +55,7 @@ import spark.rdd.ZippedPartitionsRDD2 import spark.rdd.ZippedPartitionsRDD3 import spark.rdd.ZippedPartitionsRDD4 -import spark.rdd.{IndexedRDD, BlockIndex} +import spark.rdd.{IndexedRDD, RDDIndex} import spark.storage.StorageLevel import spark.util.BoundedPriorityQueue @@ -774,7 +774,7 @@ abstract class RDD[T: ClassManifest]( } - def makeIndex(partitioner: Option[Partitioner] = None): RDD[BlockIndex[T]] = + def makeIndex(partitioner: Option[Partitioner] = None): RDDIndex[T] = IndexedRDD.makeIndex(this, partitioner) diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index d19d220ec2..ab1e460aeb 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -31,6 +31,10 @@ import spark.rdd._ import spark.SparkContext._ import spark.Partitioner._ +import spark.storage.StorageLevel + + + @@ -41,6 +45,20 @@ import spark.Partitioner._ class BlockIndex[@specialized K: ClassManifest] extends JHashMap[K,Int] +/** + * The RDDIndex is an opaque type used to represent the organization + * of values in an RDD + */ +class RDDIndex[@specialized K: ClassManifest](private[spark] val rdd: RDD[BlockIndex[K]]) { + def persist(newLevel: StorageLevel): RDDIndex[K] = { + rdd.persist(newLevel) + return this + } +} + + + + /** * An IndexedRDD[K,V] extends the RDD[(K,V)] by pre-indexing the keys and @@ -51,30 +69,41 @@ class BlockIndex[@specialized K: ClassManifest] extends JHashMap[K,Int] * */ class IndexedRDD[K: ClassManifest, V: ClassManifest]( - val index: RDD[ BlockIndex[K] ], + val index: RDDIndex[K], val valuesRDD: RDD[ Seq[Seq[V]] ]) - extends RDD[(K, V)](index.context, - List(new OneToOneDependency(index), new OneToOneDependency(valuesRDD)) ) { + extends RDD[(K, V)](index.rdd.context, + List(new OneToOneDependency(index.rdd), new OneToOneDependency(valuesRDD)) ) { /** - * An internal representation of the maps and block managers + * An internal representation which joins the block indices with the values */ - protected val tuples = new ZippedRDD(index.context, index, valuesRDD) + protected val tuples = new ZippedRDD(index.rdd.context, index.rdd, valuesRDD) - override val partitioner = index.partitioner + /** + * The partitioner is defined by the index + */ + override val partitioner = index.rdd.partitioner + /** + * The actual partitions are defined by the tuples. + */ override def getPartitions: Array[Partition] = tuples.getPartitions - + /** + * The preferred locations are computed based on the preferred locations of the tuples. + */ override def getPreferredLocations(s: Partition): Seq[String] = tuples.getPreferredLocations(s) - override def cache: IndexedRDD[K,V] = { - index.cache - valuesRDD.cache + /** + * Caching an IndexedRDD causes the index and values to be cached separately. + */ + override def persist(newLevel: StorageLevel): RDD[(K,V)] = { + index.persist(newLevel) + valuesRDD.persist(newLevel) return this } @@ -90,7 +119,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( * this also retains the original RDD's partitioning. */ def mapValues[U: ClassManifest](f: V => U): IndexedRDD[K, U] = { - val cleanF = index.context.clean(f) + val cleanF = index.rdd.context.clean(f) val newValues = valuesRDD.mapPartitions(_.map(values => values.map{ case null => null case row => row.map(x => f(x)) @@ -104,7 +133,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( * keys; this also retains the original RDD's partitioning. */ def flatMapValues[U: ClassManifest](f: V => TraversableOnce[U]): IndexedRDD[K,U] = { - val cleanF = index.context.clean(f) + val cleanF = index.rdd.context.clean(f) val newValues = valuesRDD.mapPartitions(_.map(values => values.map{ case null => null case row => row.flatMap(x => f(x)) @@ -211,12 +240,12 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( }) new IndexedRDD[K, (Seq[V], Seq[W])](index, newValues) } - case other: IndexedRDD[_, _] if other.index.partitioner == index.partitioner => { + case other: IndexedRDD[_, _] if other.index.rdd.partitioner == index.rdd.partitioner => { // If both RDDs are indexed using different indices but with the same partitioners // then we we need to first merge the indicies and then use the merged index to // merge the values. val newIndex = - index.zipPartitions(other.index)( + index.rdd.zipPartitions(other.index.rdd)( (thisIter, otherIter) => { val thisIndex = thisIter.next() assert(!thisIter.hasNext()) @@ -264,11 +293,11 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } List(newValues.toSeq).iterator }) - new IndexedRDD(newIndex, newValues) + new IndexedRDD(new RDDIndex(newIndex), newValues) } case _ => { // Get the partitioner from the index - val partitioner = index.partitioner match { + val partitioner = index.rdd.partitioner match { case Some(p) => p case None => throw new SparkException("An index must have a partitioner.") } @@ -328,7 +357,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( val newIndex = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) val newValues = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) - new IndexedRDD[K, (Seq[V], Seq[W])](newIndex, newValues) + new IndexedRDD[K, (Seq[V], Seq[W])](new RDDIndex(newIndex), newValues) } } } @@ -440,7 +469,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( object IndexedRDD { def apply[K: ClassManifest, V: ClassManifest]( tbl: RDD[(K,V)], - existingIndex: RDD[BlockIndex[K]] = null ): IndexedRDD[K, V] = { + existingIndex: RDDIndex[K] = null ): IndexedRDD[K, V] = { if (existingIndex == null) { // Shuffle the table (if necessary) @@ -466,10 +495,10 @@ object IndexedRDD { // extract the index and the values val index = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) val values = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) - new IndexedRDD[K,V](index, values) + new IndexedRDD[K,V](new RDDIndex(index), values) } else { val index = existingIndex - val partitioner = index.partitioner match { + val partitioner = index.rdd.partitioner match { case Some(p) => p case None => throw new SparkException("An index must have a partitioner.") } @@ -483,7 +512,7 @@ object IndexedRDD { } // Use the index to build the new values table - val values = index.zipPartitions(shuffledTbl)( + val values = index.rdd.zipPartitions(shuffledTbl)( (indexIter, tblIter) => { // There is only one map val index = indexIter.next() @@ -511,7 +540,7 @@ object IndexedRDD { * Construct and index of the unique values in a given RDD. */ def makeIndex[K: ClassManifest](keys: RDD[K], - partitioner: Option[Partitioner] = None): RDD[BlockIndex[K]] = { + partitioner: Option[Partitioner] = None): RDDIndex[K] = { // Ugly hack :-(. In order to partition the keys they must have values. @@ -539,7 +568,7 @@ object IndexedRDD { } List(indexMap).iterator }, true).cache - index + new RDDIndex(index) } } From 93503a7054e07cec71d8cac006e8585acb0eceb0 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 27 Aug 2013 18:16:19 -0700 Subject: [PATCH 11/13] Allowing RDD to select its implementation of PairRDDFunctions --- .../scala/spark/IndexedRDDFunctions.scala | 273 ++++++++++++++ .../main/scala/spark/PairRDDFunctions.scala | 44 +-- core/src/main/scala/spark/RDD.scala | 13 + core/src/main/scala/spark/SparkContext.scala | 2 +- .../scala/spark/api/java/JavaPairRDD.scala | 78 +++- .../src/main/scala/spark/rdd/IndexedRDD.scala | 340 +----------------- 6 files changed, 380 insertions(+), 370 deletions(-) create mode 100644 core/src/main/scala/spark/IndexedRDDFunctions.scala diff --git a/core/src/main/scala/spark/IndexedRDDFunctions.scala b/core/src/main/scala/spark/IndexedRDDFunctions.scala new file mode 100644 index 0000000000..8bfe9d75e1 --- /dev/null +++ b/core/src/main/scala/spark/IndexedRDDFunctions.scala @@ -0,0 +1,273 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package spark + +import java.util.{HashMap => JHashMap, BitSet => JBitSet, HashSet => JHashSet} + +import scala.collection.JavaConversions._ +import scala.collection.mutable.ArrayBuffer + +import spark._ + +import spark.rdd.ShuffledRDD +import spark.rdd.IndexedRDD +import spark.rdd.BlockIndex +import spark.rdd.RDDIndex + + +class IndexedRDDFunctions[K: ClassManifest, V: ClassManifest](self: IndexedRDD[K,V]) + extends PairRDDFunctions[K,V](self) { + + /** + * Construct a new IndexedRDD that is indexed by only the keys in the RDD + */ + def reindex(): IndexedRDD[K,V] = IndexedRDD(self) + + + /** + * Pass each value in the key-value pair RDD through a map function without changing the keys; + * this also retains the original RDD's partitioning. + */ + override def mapValues[U: ClassManifest](f: V => U): RDD[(K, U)] = { + val cleanF = self.index.rdd.context.clean(f) + val newValues = self.valuesRDD.mapPartitions(_.map(values => values.map{ + case null => null + case row => row.map(x => f(x)) + }), true) + new IndexedRDD[K,U](self.index, newValues) + } + + + /** + * Pass each value in the key-value pair RDD through a flatMap function without changing the + * keys; this also retains the original RDD's partitioning. + */ + override def flatMapValues[U: ClassManifest](f: V => TraversableOnce[U]): RDD[(K,U)] = { + val cleanF = self.index.rdd.context.clean(f) + val newValues = self.valuesRDD.mapPartitions(_.map(values => values.map{ + case null => null + case row => row.flatMap(x => f(x)) + }), true) + new IndexedRDD[K,U](self.index, newValues) + } + + + /** + * Generic function to combine the elements for each key using a custom set of aggregation + * functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], for a "combined type" C + * Note that V and C can be different -- for example, one might group an RDD of type + * (Int, Int) into an RDD of type (Int, Seq[Int]). Users provide three functions: + * + * - `createCombiner`, which turns a V into a C (e.g., creates a one-element list) + * - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list) + * - `mergeCombiners`, to combine two C's into a single one. + */ + override def combineByKey[C: ClassManifest](createCombiner: V => C, + mergeValue: (C, V) => C, + mergeCombiners: (C, C) => C, + partitioner: Partitioner, + mapSideCombine: Boolean = true, + serializerClass: String = null): RDD[(K, C)] = { + val newValues = self.valuesRDD.mapPartitions( + _.map{ groups: Seq[Seq[V]] => + groups.map{ group: Seq[V] => + if (group != null && !group.isEmpty) { + val c: C = createCombiner(group.head) + val sum: C = group.tail.foldLeft(c)(mergeValue) + Seq(sum) + } else { + null + } + } + }, true) + new IndexedRDD[K,C](self.index, newValues) + } + + + + /** + * Group the values for each key in the RDD into a single sequence. Hash-partitions the + * resulting RDD with the existing partitioner/parallelism level. + */ + override def groupByKey(partitioner: Partitioner): RDD[(K, Seq[V])] = { + val newValues = self.valuesRDD.mapPartitions(_.map{ar => ar.map{s => Seq(s)} }, true) + new IndexedRDD[K, Seq[V]](self.index, newValues) + } + + + /** + * 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`. + */ + override def cogroup[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): + IndexedRDD[K, (Seq[V], Seq[W])] = { + //RDD[(K, (Seq[V], Seq[W]))] = { + other match { + case other: IndexedRDD[_, _] if self.index == other.index => { + // if both RDDs share exactly the same index and therefore the same super set of keys + // then we simply merge the value RDDs. + // However it is possible that both RDDs are missing a value for a given key in + // which case the returned RDD should have a null value + val newValues = + self.valuesRDD.zipPartitions(other.valuesRDD)( + (thisIter, otherIter) => { + val thisValues: Seq[Seq[V]] = thisIter.next() + assert(!thisIter.hasNext) + val otherValues: Seq[Seq[W]] = otherIter.next() + assert(!otherIter.hasNext) + // Zip the values and if both arrays are null then the key is not present and + // so the resulting value must be null (not a tuple of empty sequences) + val tmp: Seq[Seq[(Seq[V], Seq[W])]] = thisValues.view.zip(otherValues).map{ + case (null, null) => null // The key is not present in either RDD + case (a, null) => Seq((a, Seq.empty[W])) + case (null, b) => Seq((Seq.empty[V], b)) + case (a, b) => Seq((a,b)) + }.toSeq + List(tmp).iterator + }) + new IndexedRDD[K, (Seq[V], Seq[W])](self.index, newValues) + } + case other: IndexedRDD[_, _] + if self.index.rdd.partitioner == other.index.rdd.partitioner => { + // If both RDDs are indexed using different indices but with the same partitioners + // then we we need to first merge the indicies and then use the merged index to + // merge the values. + val newIndex = + self.index.rdd.zipPartitions(other.index.rdd)( + (thisIter, otherIter) => { + val thisIndex = thisIter.next() + assert(!thisIter.hasNext) + val otherIndex = otherIter.next() + assert(!otherIter.hasNext) + val newIndex = new BlockIndex[K]() + // @todo Merge only the keys that correspond to non-null values + // Merge the keys + newIndex.putAll(thisIndex) + newIndex.putAll(otherIndex) + // We need to rekey the index + var ctr = 0 + for (e <- newIndex.entrySet) { + e.setValue(ctr) + ctr += 1 + } + List(newIndex).iterator + }).cache() + // Use the new index along with the this and the other indices to merge the values + val newValues = + newIndex.zipPartitions(self.tuples, other.tuples)( + (newIndexIter, thisTuplesIter, otherTuplesIter) => { + // Get the new index for this partition + val newIndex = newIndexIter.next() + assert(!newIndexIter.hasNext) + // Get the corresponding indicies and values for this and the other IndexedRDD + val (thisIndex, thisValues) = thisTuplesIter.next() + assert(!thisTuplesIter.hasNext) + val (otherIndex, otherValues) = otherTuplesIter.next() + assert(!otherTuplesIter.hasNext) + // Preallocate the new Values array + val newValues = new Array[Seq[(Seq[V],Seq[W])]](newIndex.size) + // Lookup the sequences in both submaps + for ((k,ind) <- newIndex) { + val thisSeq = if (thisIndex.contains(k)) thisValues(thisIndex.get(k)) else null + val otherSeq = if (otherIndex.contains(k)) otherValues(otherIndex.get(k)) else null + // if either of the sequences is not null then the key was in one of the two tables + // and so the value should appear in the returned table + newValues(ind) = (thisSeq, otherSeq) match { + case (null, null) => null + case (a, null) => Seq( (a, Seq.empty[W]) ) + case (null, b) => Seq( (Seq.empty[V], b) ) + case (a, b) => Seq( (a,b) ) + } + } + List(newValues.toSeq).iterator + }) + new IndexedRDD(new RDDIndex(newIndex), newValues) + } + case _ => { + // Get the partitioner from the index + val partitioner = self.index.rdd.partitioner match { + case Some(p) => p + case None => throw new SparkException("An index must have a partitioner.") + } + // Shuffle the other RDD using the partitioner for this index + val otherShuffled = + if (other.partitioner == Some(partitioner)) { + other + } else { + new ShuffledRDD[K,W](other, partitioner) + } + // Join the other RDD with this RDD building a new valueset and new index on the fly + val groups = + self.tuples.zipPartitions(otherShuffled)( + (thisTuplesIter, otherTuplesIter) => { + // Get the corresponding indicies and values for this IndexedRDD + val (thisIndex, thisValues) = thisTuplesIter.next() + assert(!thisTuplesIter.hasNext()) + // Construct a new index + val newIndex = thisIndex.clone().asInstanceOf[BlockIndex[K]] + // Construct a new array Buffer to store the values + val newValues = ArrayBuffer.fill[(Seq[V], Seq[W])](thisValues.size)(null) + // populate the newValues with the values in this IndexedRDD + for ((k,i) <- thisIndex) { + if (thisValues(i) != null) { + newValues(i) = (thisValues(i), ArrayBuffer.empty[W]) + } + } + // Now iterate through the other tuples updating the map + for ((k,w) <- otherTuplesIter){ + if (!newIndex.contains(k)) { + // update the index + val ind = newIndex.size + newIndex.put(k, ind) + // Update the values + newValues.append( (Seq.empty[V], ArrayBuffer(w) ) ) + } else { + val ind = newIndex.get(k) + if(newValues(ind) == null) { + // If the other key was in the index but not in the values + // of this indexed RDD then create a new values entry for it + newValues(ind) = (Seq.empty[V], ArrayBuffer(w)) + } else { + newValues(ind)._2.asInstanceOf[ArrayBuffer[W]].append(w) + } + } + } + // Finalize the new values array + val newValuesArray: Seq[Seq[(Seq[V],Seq[W])]] = + newValues.view.map{ + case null => null + case (s, ab) => Seq((s, ab.toSeq)) + }.toSeq + List( (newIndex, newValuesArray) ).iterator + }).cache() + + // Extract the index and values from the above RDD + val newIndex = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) + val newValues = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) + + new IndexedRDD[K, (Seq[V], Seq[W])](new RDDIndex(newIndex), newValues) + } + } + } + + +} + +//(self: IndexedRDD[K, V]) extends PairRDDFunctions(self) { } + + diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 03f25040a1..ca42980b46 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -69,7 +69,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * In addition, users can control the partitioning of the output RDD, and whether to perform * map-side aggregation (if a mapper can produce multiple items with the same key). */ - def combineByKey[C](createCombiner: V => C, + def combineByKey[C: ClassManifest](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, partitioner: Partitioner, @@ -103,7 +103,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( /** * Simplified version of combineByKey that hash-partitions the output RDD. */ - def combineByKey[C](createCombiner: V => C, + def combineByKey[C: ClassManifest](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, numPartitions: Int): RDD[(K, C)] = { @@ -248,7 +248,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Uses the given Partitioner to partition the output RDD. */ - def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = { + def join[W: ClassManifest](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) @@ -261,7 +261,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * pair (k, (v, None)) if no elements in `other` have key k. Uses the given Partitioner to * partition the output RDD. */ - def leftOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, Option[W]))] = { + def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, Option[W]))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => if (ws.isEmpty) { @@ -278,7 +278,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * pair (k, (None, w)) if no elements in `this` have key k. Uses the given Partitioner to * partition the output RDD. */ - def rightOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner) + def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner) : RDD[(K, (Option[V], W))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => @@ -294,7 +294,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * Simplified version of combineByKey that hash-partitions the resulting RDD using the * existing partitioner/parallelism level. */ - def combineByKey[C](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) + def combineByKey[C: ClassManifest](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) : RDD[(K, C)] = { combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(self)) } @@ -322,7 +322,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Performs a hash join across the cluster. */ - def join[W](other: RDD[(K, W)]): RDD[(K, (V, W))] = { + def join[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, (V, W))] = { join(other, defaultPartitioner(self, other)) } @@ -331,7 +331,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Performs a hash join across the cluster. */ - def join[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, W))] = { + def join[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, W))] = { join(other, new HashPartitioner(numPartitions)) } @@ -341,7 +341,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output * using the existing partitioner/parallelism level. */ - def leftOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (V, Option[W]))] = { + def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, (V, Option[W]))] = { leftOuterJoin(other, defaultPartitioner(self, other)) } @@ -351,7 +351,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output * into `numPartitions` partitions. */ - def leftOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, Option[W]))] = { + def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, Option[W]))] = { leftOuterJoin(other, new HashPartitioner(numPartitions)) } @@ -361,7 +361,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting * RDD using the existing partitioner/parallelism level. */ - def rightOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (Option[V], W))] = { + def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, (Option[V], W))] = { rightOuterJoin(other, defaultPartitioner(self, other)) } @@ -371,7 +371,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting * RDD into the given number of partitions. */ - def rightOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Option[V], W))] = { + def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Option[V], W))] = { rightOuterJoin(other, new HashPartitioner(numPartitions)) } @@ -384,7 +384,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * Pass each value in the key-value pair RDD through a map function without changing the keys; * this also retains the original RDD's partitioning. */ - def mapValues[U](f: V => U): RDD[(K, U)] = { + def mapValues[U: ClassManifest](f: V => U): RDD[(K, U)] = { val cleanF = self.context.clean(f) new MappedValuesRDD(self, cleanF) } @@ -393,7 +393,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * Pass each value in the key-value pair RDD through a flatMap function without changing the * keys; this also retains the original RDD's partitioning. */ - def flatMapValues[U](f: V => TraversableOnce[U]): RDD[(K, U)] = { + def flatMapValues[U: ClassManifest](f: V => TraversableOnce[U]): RDD[(K, U)] = { val cleanF = self.context.clean(f) new FlatMappedValuesRDD(self, cleanF) } @@ -402,7 +402,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * 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: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (Seq[V], Seq[W]))] = { if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { throw new SparkException("Default partitioner cannot partition array keys.") } @@ -420,7 +420,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * 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) + def cogroup[W1: ClassManifest, W2: ClassManifest](other1: RDD[(K, W1)], other2: RDD[(K, W2)], partitioner: Partitioner) : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { throw new SparkException("Default partitioner cannot partition array keys.") @@ -441,7 +441,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * 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: ClassManifest](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { cogroup(other, defaultPartitioner(self, other)) } @@ -449,7 +449,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * 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)]) + def cogroup[W1: ClassManifest, W2: ClassManifest](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { cogroup(other1, other2, defaultPartitioner(self, other1, other2)) } @@ -458,7 +458,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * 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: ClassManifest](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Seq[V], Seq[W]))] = { cogroup(other, new HashPartitioner(numPartitions)) } @@ -466,18 +466,18 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * 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) + def cogroup[W1: ClassManifest, W2: ClassManifest](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numPartitions: Int) : RDD[(K, (Seq[V], Seq[W1], Seq[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: ClassManifest](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { cogroup(other, defaultPartitioner(self, other)) } /** Alias for cogroup. */ - def groupWith[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) + def groupWith[W1: ClassManifest, W2: ClassManifest](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { cogroup(other1, other2, defaultPartitioner(self, other1, other2)) } diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 1664471524..95adbda43f 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -774,6 +774,19 @@ abstract class RDD[T: ClassManifest]( } + + // def pairRDDFunctions[K: ClassManifest, V,](implicit t: T <:< (K, V), k: ClassManifest[K], v: ClassManifest[V]): + // PairRDDFunctions[K, V] = { + // new PairRDDFunctions(this.asInstanceOf[RDD[(K,V)]]) + // } + + def pairRDDFunctions[K, V]( + implicit t: T <:< (K, V), k: ClassManifest[K], v: ClassManifest[V]): + PairRDDFunctions[K, V] = { + new PairRDDFunctions(this.asInstanceOf[RDD[(K,V)]]) + } + + def makeIndex(partitioner: Option[Partitioner] = None): RDDIndex[T] = IndexedRDD.makeIndex(this, partitioner) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 80c65dfebd..7394730786 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -878,7 +878,7 @@ object SparkContext { // TODO: Add AccumulatorParams for other types, e.g. lists and strings implicit def rddToPairRDDFunctions[K: ClassManifest, V: ClassManifest](rdd: RDD[(K, V)]) = - new PairRDDFunctions(rdd) + rdd.pairRDDFunctions implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable: ClassManifest]( rdd: RDD[(K, V)]) = diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala index c2995b836a..0b444c5a7e 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala @@ -263,8 +263,11 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * the merging locally on each mapper before sending results to a reducer, similarly to a * "combiner" in MapReduce. */ - def join[W](other: JavaPairRDD[K, W], partitioner: Partitioner): JavaPairRDD[K, (V, W)] = + def join[W](other: JavaPairRDD[K, W], partitioner: Partitioner): JavaPairRDD[K, (V, W)] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] fromRDD(rdd.join(other, partitioner)) + } /** * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the @@ -274,6 +277,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif */ def leftOuterJoin[W](other: JavaPairRDD[K, W], partitioner: Partitioner) : JavaPairRDD[K, (V, Optional[W])] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.leftOuterJoin(other, partitioner) fromRDD(joinResult.mapValues{case (v, w) => (v, JavaUtils.optionToOptional(w))}) } @@ -286,6 +291,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif */ def rightOuterJoin[W](other: JavaPairRDD[K, W], partitioner: Partitioner) : JavaPairRDD[K, (Optional[V], W)] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.rightOuterJoin(other, partitioner) fromRDD(joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}) } @@ -324,16 +331,22 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Performs a hash join across the cluster. */ - def join[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (V, W)] = + def join[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (V, W)] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] fromRDD(rdd.join(other)) + } /** * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Performs a hash join across the cluster. */ - def join[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (V, W)] = + def join[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (V, W)] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] fromRDD(rdd.join(other, numPartitions)) + } /** * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the @@ -342,6 +355,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * using the existing partitioner/parallelism level. */ def leftOuterJoin[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (V, Optional[W])] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.leftOuterJoin(other) fromRDD(joinResult.mapValues{case (v, w) => (v, JavaUtils.optionToOptional(w))}) } @@ -353,6 +368,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * into `numPartitions` partitions. */ def leftOuterJoin[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (V, Optional[W])] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.leftOuterJoin(other, numPartitions) fromRDD(joinResult.mapValues{case (v, w) => (v, JavaUtils.optionToOptional(w))}) } @@ -364,6 +381,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * RDD using the existing partitioner/parallelism level. */ def rightOuterJoin[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (Optional[V], W)] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.rightOuterJoin(other) fromRDD(joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}) } @@ -375,6 +394,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * RDD into the given number of partitions. */ def rightOuterJoin[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (Optional[V], W)] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.rightOuterJoin(other, numPartitions) fromRDD(joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}) } @@ -411,55 +432,86 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * 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, (JList[V], JList[W])] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] fromRDD(cogroupResultToJava(rdd.cogroup(other, partitioner))) + } /** * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * 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])] = + : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = { + implicit val w1m: ClassManifest[W1] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W1]] + implicit val w2m: ClassManifest[W2] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[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, (JList[V], JList[W])] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] fromRDD(cogroupResultToJava(rdd.cogroup(other))) + } /** * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * 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, (JList[V], JList[W1], JList[W2])] = { + implicit val w1m: ClassManifest[W1] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W1]] + implicit val w2m: ClassManifest[W2] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W2]] fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2))) + } /** * 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], numPartitions: Int): JavaPairRDD[K, (JList[V], JList[W])] - = fromRDD(cogroupResultToJava(rdd.cogroup(other, numPartitions))) - + def cogroup[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (JList[V], JList[W])] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] + fromRDD(cogroupResultToJava(rdd.cogroup(other, numPartitions))) + } /** * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * 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, (JList[V], JList[W1], JList[W2])] = { + implicit val w1m: ClassManifest[W1] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W1]] + implicit val w2m: ClassManifest[W2] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[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, (JList[V], JList[W])] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[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, (JList[V], JList[W1], JList[W2])] = { + implicit val w1m: ClassManifest[W1] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W1]] + implicit val w2m: ClassManifest[W2] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W2]] fromRDD(cogroupResult2ToJava(rdd.groupWith(other1, other2))) + } /** * Return the list of values in the RDD for key `key`. This operation is done efficiently if the diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index ab1e460aeb..ac80a06c8a 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -25,7 +25,6 @@ import java.util.{HashMap => JHashMap, BitSet => JBitSet, HashSet => JHashSet} import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer - import spark._ import spark.rdd._ import spark.SparkContext._ @@ -77,7 +76,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( /** * An internal representation which joins the block indices with the values */ - protected val tuples = new ZippedRDD(index.rdd.context, index.rdd, valuesRDD) + protected[spark] val tuples = new ZippedRDD(index.rdd.context, index.rdd, valuesRDD) /** @@ -108,341 +107,14 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } - /** - * Construct a new IndexedRDD that is indexed by only the keys in the RDD - */ - def reindex(): IndexedRDD[K,V] = IndexedRDD(this) - - - /** - * Pass each value in the key-value pair RDD through a map function without changing the keys; - * this also retains the original RDD's partitioning. - */ - def mapValues[U: ClassManifest](f: V => U): IndexedRDD[K, U] = { - val cleanF = index.rdd.context.clean(f) - val newValues = valuesRDD.mapPartitions(_.map(values => values.map{ - case null => null - case row => row.map(x => f(x)) - }), true) - new IndexedRDD[K,U](index, newValues) + override def pairRDDFunctions[K1, V1]( + implicit t: (K, V) <:< (K1,V1), k: ClassManifest[K1], v: ClassManifest[V1]): + PairRDDFunctions[K1, V1] = { + new IndexedRDDFunctions[K1,V1](this.asInstanceOf[IndexedRDD[K1,V1]]) } - /** - * Pass each value in the key-value pair RDD through a flatMap function without changing the - * keys; this also retains the original RDD's partitioning. - */ - def flatMapValues[U: ClassManifest](f: V => TraversableOnce[U]): IndexedRDD[K,U] = { - val cleanF = index.rdd.context.clean(f) - val newValues = valuesRDD.mapPartitions(_.map(values => values.map{ - case null => null - case row => row.flatMap(x => f(x)) - }), true) - new IndexedRDD[K,U](index, newValues) - } - - - /** - * Generic function to combine the elements for each key using a custom set of aggregation - * functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], for a "combined type" C - * Note that V and C can be different -- for example, one might group an RDD of type - * (Int, Int) into an RDD of type (Int, Seq[Int]). Users provide three functions: - * - * - `createCombiner`, which turns a V into a C (e.g., creates a one-element list) - * - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list) - * - `mergeCombiners`, to combine two C's into a single one. - */ - def combineByKey[C: ClassManifest](createCombiner: V => C, - mergeValue: (C, V) => C, - mergeCombiners: (C, C) => C, - serializerClass: String = null): IndexedRDD[K, C] = { - val newValues = valuesRDD.mapPartitions( - _.map{ groups: Seq[Seq[V]] => - groups.map{ group: Seq[V] => - if (group != null && !group.isEmpty) { - val c: C = createCombiner(group.head) - val sum: C = group.tail.foldLeft(c)(mergeValue) - Seq(sum) - } else { - null - } - } - }, true) - new IndexedRDD[K,C](index, newValues) - } - - /** - * Merge the values for each key using an associative function and a neutral "zero value" which may - * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for - * list concatenation, 0 for addition, or 1 for multiplication.). - */ - def foldByKey(zeroValue: V)(func: (V, V) => V): IndexedRDD[K, V] = { - // Serialize the zero value to a byte array so that we can get a new clone of it on each key - val zeroBuffer = SparkEnv.get.closureSerializer.newInstance().serialize(zeroValue) - val zeroArray = new Array[Byte](zeroBuffer.limit) - zeroBuffer.get(zeroArray) - - // When deserializing, use a lazy val to create just one instance of the serializer per task - lazy val cachedSerializer = SparkEnv.get.closureSerializer.newInstance() - def createZero() = cachedSerializer.deserialize[V](ByteBuffer.wrap(zeroArray)) - combineByKey[V]((v: V) => func(createZero(), v), func, func) - } - - /** - * Merge the values for each key using an associative reduce function. This will also perform - * the merging locally on each mapper before sending results to a reducer, similarly to a - * "combiner" in MapReduce. Output will be hash-partitioned with the existing partitioner/ - * parallelism level. - */ - def reduceByKey(func: (V, V) => V): IndexedRDD[K, V] = { - combineByKey[V]((v: V) => v, func, func) - } - - - /** - * 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(): IndexedRDD[K, Seq[V]] = { - val newValues = valuesRDD.mapPartitions(_.map{ar => ar.map{s => Seq(s)} }, true) - new IndexedRDD[K, Seq[V]](index, newValues) - } - - - /** - * 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: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Seq[V], Seq[W])] = { - //RDD[(K, (Seq[V], Seq[W]))] = { - other match { - case other: IndexedRDD[_, _] if other.index == index => { - // if both RDDs share exactly the same index and therefore the same super set of keys - // then we simply merge the value RDDs. - // However it is possible that both RDDs are missing a value for a given key in - // which case the returned RDD should have a null value - val newValues = - valuesRDD.zipPartitions(other.valuesRDD)( - (thisIter, otherIter) => { - val thisValues: Seq[Seq[V]] = thisIter.next() - assert(!thisIter.hasNext()) - val otherValues: Seq[Seq[W]] = otherIter.next() - assert(!otherIter.hasNext()) - // Zip the values and if both arrays are null then the key is not present and - // so the resulting value must be null (not a tuple of empty sequences) - val tmp: Seq[Seq[(Seq[V], Seq[W])]] = thisValues.view.zip(otherValues).map{ - case (null, null) => null // The key is not present in either RDD - case (a, null) => Seq((a, Seq.empty[W])) - case (null, b) => Seq((Seq.empty[V], b)) - case (a, b) => Seq((a,b)) - }.toSeq - List(tmp).iterator - }) - new IndexedRDD[K, (Seq[V], Seq[W])](index, newValues) - } - case other: IndexedRDD[_, _] if other.index.rdd.partitioner == index.rdd.partitioner => { - // If both RDDs are indexed using different indices but with the same partitioners - // then we we need to first merge the indicies and then use the merged index to - // merge the values. - val newIndex = - index.rdd.zipPartitions(other.index.rdd)( - (thisIter, otherIter) => { - val thisIndex = thisIter.next() - assert(!thisIter.hasNext()) - val otherIndex = otherIter.next() - assert(!otherIter.hasNext()) - val newIndex = new BlockIndex[K]() - // @todo Merge only the keys that correspond to non-null values - // Merge the keys - newIndex.putAll(thisIndex) - newIndex.putAll(otherIndex) - // We need to rekey the index - var ctr = 0 - for (e <- newIndex.entrySet) { - e.setValue(ctr) - ctr += 1 - } - List(newIndex).iterator - }).cache() - // Use the new index along with the this and the other indices to merge the values - val newValues = - newIndex.zipPartitions(tuples, other.tuples)( - (newIndexIter, thisTuplesIter, otherTuplesIter) => { - // Get the new index for this partition - val newIndex = newIndexIter.next() - assert(!newIndexIter.hasNext()) - // Get the corresponding indicies and values for this and the other IndexedRDD - val (thisIndex, thisValues) = thisTuplesIter.next() - assert(!thisTuplesIter.hasNext()) - val (otherIndex, otherValues) = otherTuplesIter.next() - assert(!otherTuplesIter.hasNext()) - // Preallocate the new Values array - val newValues = new Array[Seq[(Seq[V],Seq[W])]](newIndex.size) - // Lookup the sequences in both submaps - for ((k,ind) <- newIndex) { - val thisSeq = if (thisIndex.contains(k)) thisValues(thisIndex.get(k)) else null - val otherSeq = if (otherIndex.contains(k)) otherValues(otherIndex.get(k)) else null - // if either of the sequences is not null then the key was in one of the two tables - // and so the value should appear in the returned table - newValues(ind) = (thisSeq, otherSeq) match { - case (null, null) => null - case (a, null) => Seq( (a, Seq.empty[W]) ) - case (null, b) => Seq( (Seq.empty[V], b) ) - case (a, b) => Seq( (a,b) ) - } - } - List(newValues.toSeq).iterator - }) - new IndexedRDD(new RDDIndex(newIndex), newValues) - } - case _ => { - // Get the partitioner from the index - val partitioner = index.rdd.partitioner match { - case Some(p) => p - case None => throw new SparkException("An index must have a partitioner.") - } - // Shuffle the other RDD using the partitioner for this index - val otherShuffled = - if (other.partitioner == Some(partitioner)) { - other - } else { - new ShuffledRDD[K,W](other, partitioner) - } - // Join the other RDD with this RDD building a new valueset and new index on the fly - val groups = - tuples.zipPartitions(otherShuffled)( - (thisTuplesIter, otherTuplesIter) => { - // Get the corresponding indicies and values for this IndexedRDD - val (thisIndex, thisValues) = thisTuplesIter.next() - assert(!thisTuplesIter.hasNext()) - // Construct a new index - val newIndex = thisIndex.clone().asInstanceOf[BlockIndex[K]] - // Construct a new array Buffer to store the values - val newValues = ArrayBuffer.fill[(Seq[V], Seq[W])](thisValues.size)(null) - // populate the newValues with the values in this IndexedRDD - for ((k,i) <- thisIndex) { - if (thisValues(i) != null) { - newValues(i) = (thisValues(i), ArrayBuffer.empty[W]) - } - } - // Now iterate through the other tuples updating the map - for ((k,w) <- otherTuplesIter){ - if (!newIndex.contains(k)) { - // update the index - val ind = newIndex.size - newIndex.put(k, ind) - // Update the values - newValues.append( (Seq.empty[V], ArrayBuffer(w) ) ) - } else { - val ind = newIndex.get(k) - if(newValues(ind) == null) { - // If the other key was in the index but not in the values - // of this indexed RDD then create a new values entry for it - newValues(ind) = (Seq.empty[V], ArrayBuffer(w)) - } else { - newValues(ind)._2.asInstanceOf[ArrayBuffer[W]].append(w) - } - } - } - // Finalize the new values array - val newValuesArray: Seq[Seq[(Seq[V],Seq[W])]] = - newValues.view.map{ - case null => null - case (s, ab) => Seq((s, ab.toSeq)) - }.toSeq - List( (newIndex, newValuesArray) ).iterator - }).cache() - - // Extract the index and values from the above RDD - val newIndex = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) - val newValues = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) - - new IndexedRDD[K, (Seq[V], Seq[W])](new RDDIndex(newIndex), newValues) - } - } - } - - - - // /** - // * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a - // * 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)]) - // : IndexedRDD[K, (Seq[V], Seq[W1], Seq[W2])] = { - // cogroup(other1, other2, defaultPartitioner(this, other1, other2)) - // } - - // /** - // * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a - // * 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]))] = { - // cogroup(other1, other2, new HashPartitioner(numPartitions)) - // } - - /** Alias for cogroup. */ - def groupWith[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Seq[V], Seq[W])] = { - cogroup(other) - } - - // /** Alias for cogroup. */ - // def groupWith[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) - // : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { - // cogroup(other1, other2, defaultPartitioner(self, other1, other2)) - // } - - - /** - * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each - * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and - * (k, v2) is in `other`. Performs a hash join across the cluster. - */ - def join[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (V, W)] = { - cogroup(other).flatMapValues { - case (vs, ws) => - for (v <- vs.iterator; w <- ws.iterator) yield (v, w) - } - } - - - /** - * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the - * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the - * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output - * using the existing partitioner/parallelism level. - */ - def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (V, Option[W])] = { - cogroup(other).flatMapValues { - case (vs, ws) => - if (ws.isEmpty) { - vs.iterator.map(v => (v, None)) - } else { - for (v <- vs.iterator; w <- ws.iterator) yield (v, Some(w)) - } - } - - } - - - /** - * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the - * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the - * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting - * RDD using the existing partitioner/parallelism level. - */ - def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Option[V], W)] = { - cogroup(other).flatMapValues { - case (vs, ws) => - if (vs.isEmpty) { - ws.iterator.map(w => (None, w)) - } else { - for (v <- vs.iterator; w <- ws.iterator) yield (Some(v), w) - } - } - - } + /** From 766b6fd380341da06cb1c97de71c74e0eb6c36d0 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 27 Aug 2013 18:54:26 -0700 Subject: [PATCH 12/13] Fixing IndexedRDD unit tests. --- core/src/main/scala/spark/rdd/IndexedRDD.scala | 4 ++-- core/src/test/scala/spark/IndexedRDDSuite.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index 5ddfe5db2c..8536bb4a4b 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -68,8 +68,8 @@ class RDDIndex[@specialized K: ClassManifest](private[spark] val rdd: RDD[BlockI * */ class IndexedRDD[K: ClassManifest, V: ClassManifest]( - val index: RDDIndex[K], - val valuesRDD: RDD[ Seq[Seq[V]] ]) + @transient val index: RDDIndex[K], + @transient val valuesRDD: RDD[ Seq[Seq[V]] ]) extends RDD[(K, V)](index.rdd.context, List(new OneToOneDependency(index.rdd), new OneToOneDependency(valuesRDD)) ) { diff --git a/core/src/test/scala/spark/IndexedRDDSuite.scala b/core/src/test/scala/spark/IndexedRDDSuite.scala index c1433c5095..1a17f7c0d3 100644 --- a/core/src/test/scala/spark/IndexedRDDSuite.scala +++ b/core/src/test/scala/spark/IndexedRDDSuite.scala @@ -123,7 +123,7 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { // count the dependencies to make sure there is only 1 ShuffledRDD val deps = lineage(sums) - assert(deps.filter(_.isInstanceOf[ShuffledRDD[_,_]]).size === 1) // ShuffledRDD, ParallelCollection + assert(deps.filter(_.isInstanceOf[ShuffledRDD[_,_,_]]).size === 1) // ShuffledRDD, ParallelCollection } From 33250835527c1894d9cecdc35b25017bddc68670 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Wed, 28 Aug 2013 14:57:12 -0700 Subject: [PATCH 13/13] Removing commented out test code --- core/src/main/scala/spark/RDD.scala | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 6a2d3eb78b..b308d2f62d 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -777,11 +777,9 @@ abstract class RDD[T: ClassManifest]( - // def pairRDDFunctions[K: ClassManifest, V,](implicit t: T <:< (K, V), k: ClassManifest[K], v: ClassManifest[V]): - // PairRDDFunctions[K, V] = { - // new PairRDDFunctions(this.asInstanceOf[RDD[(K,V)]]) - // } - + /** + * For RDD[(K,V)] this function returns a pair-functions object for this RDD + */ def pairRDDFunctions[K, V]( implicit t: T <:< (K, V), k: ClassManifest[K], v: ClassManifest[V]): PairRDDFunctions[K, V] = {