From 97fbb3ec52785883a0eee8644f9f4603c4c9df21 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 23 Dec 2013 22:50:15 -0800 Subject: Working ExternalAppendOnlyMap for Aggregator, but not for CoGroupedRDD --- .../main/scala/org/apache/spark/Aggregator.scala | 13 +- .../scala/org/apache/spark/rdd/CoGroupedRDD.scala | 43 ++++--- .../org/apache/spark/rdd/PairRDDFunctions.scala | 11 +- .../apache/spark/util/ExternalAppendOnlyMap.scala | 136 +++++++++++++++++++++ 4 files changed, 182 insertions(+), 21 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index 1a2ec55876..ae16242718 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -17,7 +17,7 @@ package org.apache.spark -import org.apache.spark.util.AppendOnlyMap +import org.apache.spark.util.{AppendOnlyMap, ExternalAppendOnlyMap} /** * A set of functions used to aggregate data. @@ -32,7 +32,9 @@ case class Aggregator[K, V, C] ( mergeCombiners: (C, C) => C) { def combineValuesByKey(iter: Iterator[_ <: Product2[K, V]]) : Iterator[(K, C)] = { - val combiners = new AppendOnlyMap[K, C] + println("Combining values by key!!") + //val combiners = new AppendOnlyMap[K, C] + val combiners = new ExternalAppendOnlyMap[K, C](mergeCombiners) var kv: Product2[K, V] = null val update = (hadValue: Boolean, oldValue: C) => { if (hadValue) mergeValue(oldValue, kv._2) else createCombiner(kv._2) @@ -45,7 +47,9 @@ case class Aggregator[K, V, C] ( } def combineCombinersByKey(iter: Iterator[(K, C)]) : Iterator[(K, C)] = { - val combiners = new AppendOnlyMap[K, C] + println("Combining combiners by key!!") + //val combiners = new AppendOnlyMap[K, C] + val combiners = new ExternalAppendOnlyMap[K, C](mergeCombiners) var kc: (K, C) = null val update = (hadValue: Boolean, oldValue: C) => { if (hadValue) mergeCombiners(oldValue, kc._2) else kc._2 @@ -56,5 +60,4 @@ case class Aggregator[K, V, C] ( } combiners.iterator } -} - +} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 911a002884..6283686322 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -23,7 +23,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv, TaskContext} import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency} -import org.apache.spark.util.AppendOnlyMap +import org.apache.spark.util.{AppendOnlyMap, ExternalAppendOnlyMap} private[spark] sealed trait CoGroupSplitDep extends Serializable @@ -101,36 +101,49 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: override val partitioner = Some(part) override def compute(s: Partition, context: TaskContext): Iterator[(K, Seq[Seq[_]])] = { + println("Computing in CoGroupedRDD!") + // e.g. for `(k, a) cogroup (k, b)`, K -> Seq(ArrayBuffer as, ArrayBuffer bs) val split = s.asInstanceOf[CoGroupPartition] val numRdds = split.deps.size - // e.g. for `(k, a) cogroup (k, b)`, K -> Seq(ArrayBuffer as, ArrayBuffer bs) - val map = new AppendOnlyMap[K, Seq[ArrayBuffer[Any]]] - - val update: (Boolean, Seq[ArrayBuffer[Any]]) => Seq[ArrayBuffer[Any]] = (hadVal, oldVal) => { - if (hadVal) oldVal else Array.fill(numRdds)(new ArrayBuffer[Any]) - } - - val getSeq = (k: K) => { - map.changeValue(k, update) - } + val combineFunction: (Seq[ArrayBuffer[Any]], Seq[ArrayBuffer[Any]]) => Seq[ArrayBuffer[Any]] = + (x, y) => { x ++ y } + //val map = new AppendOnlyMap[K, Seq[ArrayBuffer[Any]]] + val map = new ExternalAppendOnlyMap[K, Seq[ArrayBuffer[Any]]](combineFunction) val ser = SparkEnv.get.serializerManager.get(serializerClass) for ((dep, depNum) <- split.deps.zipWithIndex) dep match { case NarrowCoGroupSplitDep(rdd, _, itsSplit) => { // Read them from the parent - rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, Any]]].foreach { kv => - getSeq(kv._1)(depNum) += kv._2 + rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, Any]]].foreach { + kv => addToMap(kv._1, kv._2, depNum) } } case ShuffleCoGroupSplitDep(shuffleId) => { // Read map outputs of shuffle val fetcher = SparkEnv.get.shuffleFetcher fetcher.fetch[Product2[K, Any]](shuffleId, split.index, context, ser).foreach { - kv => getSeq(kv._1)(depNum) += kv._2 + kv => addToMap(kv._1, kv._2, depNum) } } } - new InterruptibleIterator(context, map.iterator) + + def addToMap(key: K, value: Any, depNum: Int) { + val updateFunction: (Boolean, Seq[ArrayBuffer[Any]]) => Seq[ArrayBuffer[Any]] = + (hadVal, oldVal) => { + var newVal = oldVal + if (!hadVal){ + newVal = Array.fill(numRdds)(new ArrayBuffer[Any]) + } + newVal(depNum) += value + newVal + } + map.changeValue(key, updateFunction) + } + + println("About to construct CoGroupedRDD iterator!") + val theIterator = map.iterator + println("Returning CoGroupedRDD iterator!") + new InterruptibleIterator(context, theIterator) } override def clearDependencies() { diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 48168e152e..6849703f05 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -85,10 +85,12 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) } val aggregator = new Aggregator[K, V, C](createCombiner, mergeValue, mergeCombiners) if (self.partitioner == Some(partitioner)) { + println("Partitioner is some partitioner! In fact, it is " + self.partitioner.toString()) self.mapPartitionsWithContext((context, iter) => { new InterruptibleIterator(context, aggregator.combineValuesByKey(iter)) }, preservesPartitioning = true) } else if (mapSideCombine) { + println("Otherwise, combining on map side.") val combined = self.mapPartitions(aggregator.combineValuesByKey, preservesPartitioning = true) val partitioned = new ShuffledRDD[K, C, (K, C)](combined, partitioner) .setSerializer(serializerClass) @@ -96,6 +98,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) new InterruptibleIterator(context, aggregator.combineCombinersByKey(iter)) }, preservesPartitioning = true) } else { + println("Else. No combining on map side!") // Don't apply map-side combiner. // A sanity check to make sure mergeCombiners is not defined. assert(mergeCombiners == null) @@ -647,6 +650,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * MapReduce job. */ def saveAsHadoopDataset(conf: JobConf) { + println("SAVE AS HADOOP DATASET") val outputFormatClass = conf.getOutputFormat val keyClass = conf.getOutputKeyClass val valueClass = conf.getOutputValueClass @@ -666,6 +670,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) writer.preSetup() def writeToFile(context: TaskContext, iter: Iterator[(K, V)]) { + println("WRITE TO FILE") // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it // around by taking a mod. We expect that no task will be attempted 2 billion times. val attemptNumber = (context.attemptId % Int.MaxValue).toInt @@ -673,13 +678,17 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) writer.setup(context.stageId, context.partitionId, attemptNumber) writer.open() + println("START LOOP\n\n\n") var count = 0 while(iter.hasNext) { + println("Before next()") val record = iter.next() count += 1 + println("Before write. Record = "+record) writer.write(record._1.asInstanceOf[AnyRef], record._2.asInstanceOf[AnyRef]) + println("After write. Record = "+record) } - + println("ALL DONE! Woohoo.") writer.close() writer.commit() } diff --git a/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala new file mode 100644 index 0000000000..28a3b7ed64 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.io._ +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable + +/** + * A simple map that spills sorted content to disk when the memory threshold is exceeded. A combiner + * function must be specified to merge values back into memory during read. + */ +class ExternalAppendOnlyMap[K,V](combinerFunction: (V, V) => V, + memoryThresholdMB: Int = 1024) + extends Iterable[(K,V)] with Serializable { + + var currentMap = new AppendOnlyMap[K,V] + var oldMaps = new ArrayBuffer[DiskKVIterator] + + def changeValue(key: K, updateFunc: (Boolean, V) => V): Unit = { + currentMap.changeValue(key, updateFunc) + val mapSize = SizeEstimator.estimate(currentMap) + //if (mapSize > memoryThresholdMB * math.pow(1024, 2)) { + if (mapSize > 1024 * 10) { + spill() + } + } + + def spill(): Unit = { + println("SPILL") + val file = File.createTempFile("external_append_only_map", "") // Add spill location + val out = new ObjectOutputStream(new FileOutputStream(file)) + val sortedMap = currentMap.iterator.toList.sortBy(kv => kv._1.hashCode()) + sortedMap foreach { + out.writeObject( _ ) + } + out.close() + currentMap = new AppendOnlyMap[K,V] + oldMaps.append(new DiskKVIterator(file)) + } + + override def iterator: Iterator[(K,V)] = new ExternalIterator() + + /** + * An iterator that merges KV pairs from memory and disk in sorted order + */ + class ExternalIterator extends Iterator[(K, V)] { + + // Order by increasing key hash value + implicit object KVOrdering extends Ordering[KVITuple] { + def compare(a:KVITuple, b:KVITuple) = -a.key.hashCode().compareTo(b.key.hashCode()) + } + val pq = mutable.PriorityQueue[KVITuple]() + val inputStreams = Seq(new MemoryKVIterator(currentMap)) ++ oldMaps + inputStreams foreach { readFromIterator } + + override def hasNext: Boolean = !pq.isEmpty + + override def next(): (K,V) = { + println("ExternalIterator.next - How many left? "+pq.length) + val minKVI = pq.dequeue() + var (minKey, minValue, minIter) = (minKVI.key, minKVI.value, minKVI.iter) +// println("Min key = "+minKey) + readFromIterator(minIter) + while (!pq.isEmpty && pq.head.key == minKey) { + val newKVI = pq.dequeue() + val (newValue, newIter) = (newKVI.value, newKVI.iter) +// println("\tfound new value to merge! "+newValue) +// println("\tcombinerFunction("+minValue+" <====> "+newValue+")") + minValue = combinerFunction(minValue, newValue) +// println("\tCombine complete! New value = "+minValue) + readFromIterator(newIter) + } + println("Returning minKey = "+minKey+", minValue = "+minValue) + (minKey, minValue) + } + + def readFromIterator(iter: Iterator[(K,V)]): Unit = { + if (iter.hasNext) { + val (k, v) = iter.next() + pq.enqueue(KVITuple(k, v, iter)) + } + } + + case class KVITuple(key:K, value:V, iter:Iterator[(K,V)]) + } + + class MemoryKVIterator(map: AppendOnlyMap[K,V]) extends Iterator[(K,V)] { + val sortedMap = currentMap.iterator.toList.sortBy(kv => kv._1.hashCode()) + val it = sortedMap.iterator + override def hasNext: Boolean = it.hasNext + override def next(): (K,V) = it.next() + } + + class DiskKVIterator(file: File) extends Iterator[(K,V)] { + val in = new ObjectInputStream(new FileInputStream(file)) + var nextItem:(K,V) = _ + var eof = false + + override def hasNext: Boolean = { + if (eof) { + return false + } + try { + nextItem = in.readObject().asInstanceOf[(K,V)] + } catch { + case e: EOFException => + eof = true + return false + } + true + } + + override def next(): (K,V) = { + if (eof) { + throw new NoSuchElementException + } + nextItem + } + } +} -- cgit v1.2.3 From 6a45ec1972d6fc053a10fbd2373f43e0c7562aa5 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 24 Dec 2013 14:27:20 -0800 Subject: Working ExternalAppendOnlyMap for both CoGroupedRDDs and Aggregator --- .../main/scala/org/apache/spark/Aggregator.scala | 2 - .../scala/org/apache/spark/rdd/CoGroupedRDD.scala | 32 +++++---- .../org/apache/spark/rdd/PairRDDFunctions.scala | 16 +---- .../apache/spark/util/ExternalAppendOnlyMap.scala | 77 ++++++++++++---------- 4 files changed, 61 insertions(+), 66 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index ae16242718..77a24733aa 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -32,7 +32,6 @@ case class Aggregator[K, V, C] ( mergeCombiners: (C, C) => C) { def combineValuesByKey(iter: Iterator[_ <: Product2[K, V]]) : Iterator[(K, C)] = { - println("Combining values by key!!") //val combiners = new AppendOnlyMap[K, C] val combiners = new ExternalAppendOnlyMap[K, C](mergeCombiners) var kv: Product2[K, V] = null @@ -47,7 +46,6 @@ case class Aggregator[K, V, C] ( } def combineCombinersByKey(iter: Iterator[(K, C)]) : Iterator[(K, C)] = { - println("Combining combiners by key!!") //val combiners = new AppendOnlyMap[K, C] val combiners = new ExternalAppendOnlyMap[K, C](mergeCombiners) var kc: (K, C) = null diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 6283686322..4c45a94af9 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -101,14 +101,16 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: override val partitioner = Some(part) override def compute(s: Partition, context: TaskContext): Iterator[(K, Seq[Seq[_]])] = { - println("Computing in CoGroupedRDD!") // e.g. for `(k, a) cogroup (k, b)`, K -> Seq(ArrayBuffer as, ArrayBuffer bs) val split = s.asInstanceOf[CoGroupPartition] val numRdds = split.deps.size - val combineFunction: (Seq[ArrayBuffer[Any]], Seq[ArrayBuffer[Any]]) => Seq[ArrayBuffer[Any]] = - (x, y) => { x ++ y } + def combine(x: Seq[ArrayBuffer[Any]], y: Seq[ArrayBuffer[Any]]) = { + x.zipAll(y, ArrayBuffer[Any](), ArrayBuffer[Any]()).map { + case (a, b) => a ++ b + } + } //val map = new AppendOnlyMap[K, Seq[ArrayBuffer[Any]]] - val map = new ExternalAppendOnlyMap[K, Seq[ArrayBuffer[Any]]](combineFunction) + val map = new ExternalAppendOnlyMap[K, Seq[ArrayBuffer[Any]]](combine) val ser = SparkEnv.get.serializerManager.get(serializerClass) for ((dep, depNum) <- split.deps.zipWithIndex) dep match { @@ -128,22 +130,18 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: } def addToMap(key: K, value: Any, depNum: Int) { - val updateFunction: (Boolean, Seq[ArrayBuffer[Any]]) => Seq[ArrayBuffer[Any]] = - (hadVal, oldVal) => { - var newVal = oldVal - if (!hadVal){ - newVal = Array.fill(numRdds)(new ArrayBuffer[Any]) - } - newVal(depNum) += value - newVal + def update(hadVal: Boolean, oldVal: Seq[ArrayBuffer[Any]]): Seq[ArrayBuffer[Any]] = { + var newVal = oldVal + if (!hadVal){ + newVal = Array.fill(numRdds)(new ArrayBuffer[Any]) } - map.changeValue(key, updateFunction) + newVal(depNum) += value + newVal + } + map.changeValue(key, update) } - println("About to construct CoGroupedRDD iterator!") - val theIterator = map.iterator - println("Returning CoGroupedRDD iterator!") - new InterruptibleIterator(context, theIterator) + new InterruptibleIterator(context, map.iterator) } override def clearDependencies() { diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 6849703f05..9512b418d7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -85,12 +85,10 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) } val aggregator = new Aggregator[K, V, C](createCombiner, mergeValue, mergeCombiners) if (self.partitioner == Some(partitioner)) { - println("Partitioner is some partitioner! In fact, it is " + self.partitioner.toString()) self.mapPartitionsWithContext((context, iter) => { new InterruptibleIterator(context, aggregator.combineValuesByKey(iter)) }, preservesPartitioning = true) } else if (mapSideCombine) { - println("Otherwise, combining on map side.") val combined = self.mapPartitions(aggregator.combineValuesByKey, preservesPartitioning = true) val partitioned = new ShuffledRDD[K, C, (K, C)](combined, partitioner) .setSerializer(serializerClass) @@ -98,10 +96,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) new InterruptibleIterator(context, aggregator.combineCombinersByKey(iter)) }, preservesPartitioning = true) } else { - println("Else. No combining on map side!") // Don't apply map-side combiner. - // A sanity check to make sure mergeCombiners is not defined. - assert(mergeCombiners == null) val values = new ShuffledRDD[K, V, (K, V)](self, partitioner).setSerializer(serializerClass) values.mapPartitionsWithContext((context, iter) => { new InterruptibleIterator(context, aggregator.combineValuesByKey(iter)) @@ -229,8 +224,9 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) // into a hash table, leading to more objects in the old gen. def createCombiner(v: V) = ArrayBuffer(v) def mergeValue(buf: ArrayBuffer[V], v: V) = buf += v + def mergeCombiners(c1: ArrayBuffer[V], c2: ArrayBuffer[V]) = c1 ++ c2 val bufs = combineByKey[ArrayBuffer[V]]( - createCombiner _, mergeValue _, null, partitioner, mapSideCombine=false) + createCombiner _, mergeValue _, mergeCombiners _, partitioner, mapSideCombine=false) bufs.asInstanceOf[RDD[(K, Seq[V])]] } @@ -650,7 +646,6 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * MapReduce job. */ def saveAsHadoopDataset(conf: JobConf) { - println("SAVE AS HADOOP DATASET") val outputFormatClass = conf.getOutputFormat val keyClass = conf.getOutputKeyClass val valueClass = conf.getOutputValueClass @@ -670,7 +665,6 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) writer.preSetup() def writeToFile(context: TaskContext, iter: Iterator[(K, V)]) { - println("WRITE TO FILE") // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it // around by taking a mod. We expect that no task will be attempted 2 billion times. val attemptNumber = (context.attemptId % Int.MaxValue).toInt @@ -678,17 +672,13 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) writer.setup(context.stageId, context.partitionId, attemptNumber) writer.open() - println("START LOOP\n\n\n") var count = 0 while(iter.hasNext) { - println("Before next()") val record = iter.next() count += 1 - println("Before write. Record = "+record) writer.write(record._1.asInstanceOf[AnyRef], record._2.asInstanceOf[AnyRef]) - println("After write. Record = "+record) } - println("ALL DONE! Woohoo.") + writer.close() writer.commit() } diff --git a/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala index 28a3b7ed64..857f8e3439 100644 --- a/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala @@ -25,36 +25,32 @@ import scala.collection.mutable * A simple map that spills sorted content to disk when the memory threshold is exceeded. A combiner * function must be specified to merge values back into memory during read. */ -class ExternalAppendOnlyMap[K,V](combinerFunction: (V, V) => V, +class ExternalAppendOnlyMap[K, V](combineFunction: (V, V) => V, memoryThresholdMB: Int = 1024) - extends Iterable[(K,V)] with Serializable { + extends Iterable[(K, V)] with Serializable { - var currentMap = new AppendOnlyMap[K,V] + var currentMap = new AppendOnlyMap[K, V] var oldMaps = new ArrayBuffer[DiskKVIterator] def changeValue(key: K, updateFunc: (Boolean, V) => V): Unit = { currentMap.changeValue(key, updateFunc) val mapSize = SizeEstimator.estimate(currentMap) - //if (mapSize > memoryThresholdMB * math.pow(1024, 2)) { - if (mapSize > 1024 * 10) { + if (mapSize > memoryThresholdMB * math.pow(1024, 2)) { spill() } } def spill(): Unit = { - println("SPILL") val file = File.createTempFile("external_append_only_map", "") // Add spill location val out = new ObjectOutputStream(new FileOutputStream(file)) val sortedMap = currentMap.iterator.toList.sortBy(kv => kv._1.hashCode()) - sortedMap foreach { - out.writeObject( _ ) - } + sortedMap.foreach { out.writeObject( _ ) } out.close() - currentMap = new AppendOnlyMap[K,V] + currentMap = new AppendOnlyMap[K, V] oldMaps.append(new DiskKVIterator(file)) } - override def iterator: Iterator[(K,V)] = new ExternalIterator() + override def iterator: Iterator[(K, V)] = new ExternalIterator() /** * An iterator that merges KV pairs from memory and disk in sorted order @@ -67,49 +63,62 @@ class ExternalAppendOnlyMap[K,V](combinerFunction: (V, V) => V, } val pq = mutable.PriorityQueue[KVITuple]() val inputStreams = Seq(new MemoryKVIterator(currentMap)) ++ oldMaps - inputStreams foreach { readFromIterator } + inputStreams.foreach { readFromIterator( _ ) } override def hasNext: Boolean = !pq.isEmpty + // Combine all values from all input streams corresponding to the same key override def next(): (K,V) = { - println("ExternalIterator.next - How many left? "+pq.length) val minKVI = pq.dequeue() - var (minKey, minValue, minIter) = (minKVI.key, minKVI.value, minKVI.iter) -// println("Min key = "+minKey) - readFromIterator(minIter) - while (!pq.isEmpty && pq.head.key == minKey) { - val newKVI = pq.dequeue() - val (newValue, newIter) = (newKVI.value, newKVI.iter) -// println("\tfound new value to merge! "+newValue) -// println("\tcombinerFunction("+minValue+" <====> "+newValue+")") - minValue = combinerFunction(minValue, newValue) -// println("\tCombine complete! New value = "+minValue) - readFromIterator(newIter) + var (minKey, minValue) = (minKVI.key, minKVI.value) + val minHash = minKey.hashCode() + readFromIterator(minKVI.iter) + + var collidedKVI = ArrayBuffer[KVITuple]() + while (!pq.isEmpty && pq.head.key.hashCode() == minHash) { + val newKVI: KVITuple = pq.dequeue() + if (newKVI.key == minKey){ + minValue = combineFunction(minValue, newKVI.value) + readFromIterator(newKVI.iter) + } else { + // Collision + collidedKVI += newKVI + } } - println("Returning minKey = "+minKey+", minValue = "+minValue) + collidedKVI.foreach { pq.enqueue( _ ) } (minKey, minValue) } - def readFromIterator(iter: Iterator[(K,V)]): Unit = { - if (iter.hasNext) { + // Read from the given iterator until a key of different hash is retrieved, + // Add each KV pair read from this iterator to the heap + def readFromIterator(iter: Iterator[(K, V)]): Unit = { + var minHash : Option[Int] = None + while (iter.hasNext) { val (k, v) = iter.next() pq.enqueue(KVITuple(k, v, iter)) + minHash match { + case None => minHash = Some(k.hashCode()) + case Some(expectedHash) => + if (k.hashCode() != expectedHash){ + return + } + } } } - case class KVITuple(key:K, value:V, iter:Iterator[(K,V)]) + case class KVITuple(key:K, value:V, iter:Iterator[(K, V)]) } - class MemoryKVIterator(map: AppendOnlyMap[K,V]) extends Iterator[(K,V)] { + class MemoryKVIterator(map: AppendOnlyMap[K, V]) extends Iterator[(K, V)] { val sortedMap = currentMap.iterator.toList.sortBy(kv => kv._1.hashCode()) val it = sortedMap.iterator override def hasNext: Boolean = it.hasNext - override def next(): (K,V) = it.next() + override def next(): (K, V) = it.next() } - class DiskKVIterator(file: File) extends Iterator[(K,V)] { + class DiskKVIterator(file: File) extends Iterator[(K, V)] { val in = new ObjectInputStream(new FileInputStream(file)) - var nextItem:(K,V) = _ + var nextItem:(K, V) = _ var eof = false override def hasNext: Boolean = { @@ -117,7 +126,7 @@ class ExternalAppendOnlyMap[K,V](combinerFunction: (V, V) => V, return false } try { - nextItem = in.readObject().asInstanceOf[(K,V)] + nextItem = in.readObject().asInstanceOf[(K, V)] } catch { case e: EOFException => eof = true @@ -126,7 +135,7 @@ class ExternalAppendOnlyMap[K,V](combinerFunction: (V, V) => V, true } - override def next(): (K,V) = { + override def next(): (K, V) = { if (eof) { throw new NoSuchElementException } -- cgit v1.2.3 From 17def8cc1132a5c94d895dacba4217ef9a0e5bd0 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 24 Dec 2013 16:15:02 -0800 Subject: Refactor ExternalAppendOnlyMap to take in KVC instead of just KV --- .../main/scala/org/apache/spark/Aggregator.scala | 20 ++--- .../scala/org/apache/spark/rdd/CoGroupedRDD.scala | 46 +++++------ .../apache/spark/util/ExternalAppendOnlyMap.scala | 88 ++++++++++++---------- 3 files changed, 78 insertions(+), 76 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index 77a24733aa..c51fb1d630 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -33,28 +33,20 @@ case class Aggregator[K, V, C] ( def combineValuesByKey(iter: Iterator[_ <: Product2[K, V]]) : Iterator[(K, C)] = { //val combiners = new AppendOnlyMap[K, C] - val combiners = new ExternalAppendOnlyMap[K, C](mergeCombiners) - var kv: Product2[K, V] = null - val update = (hadValue: Boolean, oldValue: C) => { - if (hadValue) mergeValue(oldValue, kv._2) else createCombiner(kv._2) - } + val combiners = new ExternalAppendOnlyMap[K, V, C](createCombiner, mergeValue, mergeCombiners) while (iter.hasNext) { - kv = iter.next() - combiners.changeValue(kv._1, update) + val kv = iter.next() + combiners.insert(kv._1, kv._2) } combiners.iterator } def combineCombinersByKey(iter: Iterator[(K, C)]) : Iterator[(K, C)] = { //val combiners = new AppendOnlyMap[K, C] - val combiners = new ExternalAppendOnlyMap[K, C](mergeCombiners) - var kc: (K, C) = null - val update = (hadValue: Boolean, oldValue: C) => { - if (hadValue) mergeCombiners(oldValue, kc._2) else kc._2 - } + val combiners = new ExternalAppendOnlyMap[K, C, C]((c:C) => c, mergeCombiners, mergeCombiners) while (iter.hasNext) { - kc = iter.next() - combiners.changeValue(kc._1, update) + val kc = iter.next() + combiners.insert(kc._1, kc._2) } combiners.iterator } diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 4c45a94af9..b93c60cd67 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -100,52 +100,56 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: override val partitioner = Some(part) - override def compute(s: Partition, context: TaskContext): Iterator[(K, Seq[Seq[_]])] = { + override def compute(s: Partition, context: TaskContext): Iterator[(K, CoGroupCombiner)] = { // e.g. for `(k, a) cogroup (k, b)`, K -> Seq(ArrayBuffer as, ArrayBuffer bs) val split = s.asInstanceOf[CoGroupPartition] val numRdds = split.deps.size - def combine(x: Seq[ArrayBuffer[Any]], y: Seq[ArrayBuffer[Any]]) = { - x.zipAll(y, ArrayBuffer[Any](), ArrayBuffer[Any]()).map { - case (a, b) => a ++ b - } - } - //val map = new AppendOnlyMap[K, Seq[ArrayBuffer[Any]]] - val map = new ExternalAppendOnlyMap[K, Seq[ArrayBuffer[Any]]](combine) + //val combiners = new AppendOnlyMap[K, Seq[ArrayBuffer[Any]]] + val combiners = createExternalMap(numRdds) val ser = SparkEnv.get.serializerManager.get(serializerClass) for ((dep, depNum) <- split.deps.zipWithIndex) dep match { case NarrowCoGroupSplitDep(rdd, _, itsSplit) => { // Read them from the parent rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, Any]]].foreach { - kv => addToMap(kv._1, kv._2, depNum) + kv => combiners.insert(kv._1, new CoGroupValue(kv._2, depNum)) } } case ShuffleCoGroupSplitDep(shuffleId) => { // Read map outputs of shuffle val fetcher = SparkEnv.get.shuffleFetcher fetcher.fetch[Product2[K, Any]](shuffleId, split.index, context, ser).foreach { - kv => addToMap(kv._1, kv._2, depNum) + kv => combiners.insert(kv._1, new CoGroupValue(kv._2, depNum)) } } } + new InterruptibleIterator(context, combiners.iterator) + } - def addToMap(key: K, value: Any, depNum: Int) { - def update(hadVal: Boolean, oldVal: Seq[ArrayBuffer[Any]]): Seq[ArrayBuffer[Any]] = { - var newVal = oldVal - if (!hadVal){ - newVal = Array.fill(numRdds)(new ArrayBuffer[Any]) - } - newVal(depNum) += value - newVal + def createExternalMap(numRdds:Int): ExternalAppendOnlyMap[K, CoGroupValue, CoGroupCombiner] = { + def createCombiner(v: CoGroupValue): CoGroupCombiner = { + val newCombiner = Array.fill(numRdds)(new CoGroup) + mergeValue(newCombiner, v) + } + def mergeValue(c: CoGroupCombiner, v: CoGroupValue): CoGroupCombiner = { + v match { case (value, depNum) => c(depNum) += value } + c + } + def mergeCombiners(c1: CoGroupCombiner, c2: CoGroupCombiner): CoGroupCombiner = { + c1.zipAll(c2, new CoGroup, new CoGroup).map { + case (v1, v2) => v1 ++ v2 } - map.changeValue(key, update) } - - new InterruptibleIterator(context, map.iterator) + new ExternalAppendOnlyMap [K, CoGroupValue, CoGroupCombiner] ( + createCombiner,mergeValue, mergeCombiners) } override def clearDependencies() { super.clearDependencies() rdds = null } + + type CoGroup = ArrayBuffer[Any] + type CoGroupValue = (Any, Int) // Int is dependency number + type CoGroupCombiner = Seq[CoGroup] } diff --git a/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala index 857f8e3439..1a5753603e 100644 --- a/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala @@ -22,80 +22,86 @@ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable /** - * A simple map that spills sorted content to disk when the memory threshold is exceeded. A combiner - * function must be specified to merge values back into memory during read. + * An append-only map that spills sorted content to disk when the memory threshold is exceeded. */ -class ExternalAppendOnlyMap[K, V](combineFunction: (V, V) => V, - memoryThresholdMB: Int = 1024) - extends Iterable[(K, V)] with Serializable { +class ExternalAppendOnlyMap[K, V, C] (createCombiner: V => C, + mergeValue: (C, V) => C, + mergeCombiners: (C, C) => C, + memoryThresholdMB: Int = 1024) + extends Iterable[(K, C)] with Serializable { - var currentMap = new AppendOnlyMap[K, V] - var oldMaps = new ArrayBuffer[DiskKVIterator] + var currentMap = new AppendOnlyMap[K, C] + var oldMaps = new ArrayBuffer[DiskKCIterator] - def changeValue(key: K, updateFunc: (Boolean, V) => V): Unit = { - currentMap.changeValue(key, updateFunc) + def insert(key: K, value: V): Unit = { + currentMap.changeValue(key, updateFunction(value)) val mapSize = SizeEstimator.estimate(currentMap) if (mapSize > memoryThresholdMB * math.pow(1024, 2)) { spill() } } + def updateFunction(value: V) : (Boolean, C) => C = { + (hadVal: Boolean, oldVal: C) => + if (hadVal) mergeValue(oldVal, value) else createCombiner(value) + } + def spill(): Unit = { val file = File.createTempFile("external_append_only_map", "") // Add spill location val out = new ObjectOutputStream(new FileOutputStream(file)) val sortedMap = currentMap.iterator.toList.sortBy(kv => kv._1.hashCode()) sortedMap.foreach { out.writeObject( _ ) } out.close() - currentMap = new AppendOnlyMap[K, V] - oldMaps.append(new DiskKVIterator(file)) + currentMap = new AppendOnlyMap[K, C] + oldMaps.append(new DiskKCIterator(file)) } - override def iterator: Iterator[(K, V)] = new ExternalIterator() + override def iterator: Iterator[(K, C)] = new ExternalIterator() /** * An iterator that merges KV pairs from memory and disk in sorted order */ - class ExternalIterator extends Iterator[(K, V)] { + class ExternalIterator extends Iterator[(K, C)] { // Order by increasing key hash value - implicit object KVOrdering extends Ordering[KVITuple] { - def compare(a:KVITuple, b:KVITuple) = -a.key.hashCode().compareTo(b.key.hashCode()) + implicit object KVOrdering extends Ordering[KCITuple] { + def compare(a:KCITuple, b:KCITuple) = -a.key.hashCode().compareTo(b.key.hashCode()) } - val pq = mutable.PriorityQueue[KVITuple]() - val inputStreams = Seq(new MemoryKVIterator(currentMap)) ++ oldMaps + val pq = mutable.PriorityQueue[KCITuple]() + val inputStreams = Seq(new MemoryKCIterator(currentMap)) ++ oldMaps inputStreams.foreach { readFromIterator( _ ) } override def hasNext: Boolean = !pq.isEmpty // Combine all values from all input streams corresponding to the same key - override def next(): (K,V) = { - val minKVI = pq.dequeue() - var (minKey, minValue) = (minKVI.key, minKVI.value) + override def next(): (K, C) = { + val minKCI = pq.dequeue() + var (minKey, minCombiner) = (minKCI.key, minKCI.combiner) val minHash = minKey.hashCode() - readFromIterator(minKVI.iter) + readFromIterator(minKCI.iter) - var collidedKVI = ArrayBuffer[KVITuple]() + var collidedKCI = ArrayBuffer[KCITuple]() while (!pq.isEmpty && pq.head.key.hashCode() == minHash) { - val newKVI: KVITuple = pq.dequeue() - if (newKVI.key == minKey){ - minValue = combineFunction(minValue, newKVI.value) - readFromIterator(newKVI.iter) + val newKCI: KCITuple = pq.dequeue() + if (newKCI.key == minKey){ + minCombiner = mergeCombiners(minCombiner, newKCI.combiner) + readFromIterator(newKCI.iter) } else { // Collision - collidedKVI += newKVI + collidedKCI += newKCI } } - collidedKVI.foreach { pq.enqueue( _ ) } - (minKey, minValue) + collidedKCI.foreach { pq.enqueue( _ ) } + (minKey, minCombiner) } // Read from the given iterator until a key of different hash is retrieved, - // Add each KV pair read from this iterator to the heap - def readFromIterator(iter: Iterator[(K, V)]): Unit = { + // Add each KC pair read from this iterator to the heap + def readFromIterator(iter: Iterator[(K, C)]): Unit = { var minHash : Option[Int] = None while (iter.hasNext) { - val (k, v) = iter.next() - pq.enqueue(KVITuple(k, v, iter)) + val (k, c) = iter.next() + pq.enqueue(KCITuple(k, c, iter)) minHash match { case None => minHash = Some(k.hashCode()) case Some(expectedHash) => @@ -106,19 +112,19 @@ class ExternalAppendOnlyMap[K, V](combineFunction: (V, V) => V, } } - case class KVITuple(key:K, value:V, iter:Iterator[(K, V)]) + case class KCITuple(key:K, combiner:C, iter:Iterator[(K, C)]) } - class MemoryKVIterator(map: AppendOnlyMap[K, V]) extends Iterator[(K, V)] { - val sortedMap = currentMap.iterator.toList.sortBy(kv => kv._1.hashCode()) + class MemoryKCIterator(map: AppendOnlyMap[K, C]) extends Iterator[(K, C)] { + val sortedMap = currentMap.iterator.toList.sortBy(kc => kc._1.hashCode()) val it = sortedMap.iterator override def hasNext: Boolean = it.hasNext - override def next(): (K, V) = it.next() + override def next(): (K, C) = it.next() } - class DiskKVIterator(file: File) extends Iterator[(K, V)] { + class DiskKCIterator(file: File) extends Iterator[(K, C)] { val in = new ObjectInputStream(new FileInputStream(file)) - var nextItem:(K, V) = _ + var nextItem:(K, C) = _ var eof = false override def hasNext: Boolean = { @@ -126,7 +132,7 @@ class ExternalAppendOnlyMap[K, V](combineFunction: (V, V) => V, return false } try { - nextItem = in.readObject().asInstanceOf[(K, V)] + nextItem = in.readObject().asInstanceOf[(K, C)] } catch { case e: EOFException => eof = true @@ -135,7 +141,7 @@ class ExternalAppendOnlyMap[K, V](combineFunction: (V, V) => V, true } - override def next(): (K, V) = { + override def next(): (K, C) = { if (eof) { throw new NoSuchElementException } -- cgit v1.2.3 From 28685a482032f4b42cb46d1b24de1cc1dd1180c1 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 25 Dec 2013 15:16:57 -0800 Subject: Provide for cases when mergeCombiners is not specified in ExternalAppendOnlyMap --- .../scala/org/apache/spark/rdd/CoGroupedRDD.scala | 2 +- .../apache/spark/util/ExternalAppendOnlyMap.scala | 187 +++++++++++++-------- 2 files changed, 121 insertions(+), 68 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index b93c60cd67..a2a3de7d88 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -126,7 +126,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: new InterruptibleIterator(context, combiners.iterator) } - def createExternalMap(numRdds:Int): ExternalAppendOnlyMap[K, CoGroupValue, CoGroupCombiner] = { + private def createExternalMap(numRdds:Int): ExternalAppendOnlyMap [K, CoGroupValue, CoGroupCombiner] = { def createCombiner(v: CoGroupValue): CoGroupCombiner = { val newCombiner = Array.fill(numRdds)(new CoGroup) mergeValue(newCombiner, v) diff --git a/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala index 1a5753603e..e2205c6063 100644 --- a/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala @@ -18,11 +18,16 @@ package org.apache.spark.util import java.io._ -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable +import scala.collection.mutable.{ArrayBuffer, PriorityQueue} /** - * An append-only map that spills sorted content to disk when the memory threshold is exceeded. + * A wrapper for SpillableAppendOnlyMap that handles two cases: + * + * (1) If a mergeCombiners function is specified, merge values into combiners before + * disk spill, as it is possible to merge the resulting combiners later + * + * (2) Otherwise, group values of the same key together before disk spill, and merge + * them into combiners only after reading them back from disk */ class ExternalAppendOnlyMap[K, V, C] (createCombiner: V => C, mergeValue: (C, V) => C, @@ -30,78 +35,103 @@ class ExternalAppendOnlyMap[K, V, C] (createCombiner: V => C, memoryThresholdMB: Int = 1024) extends Iterable[(K, C)] with Serializable { - var currentMap = new AppendOnlyMap[K, C] - var oldMaps = new ArrayBuffer[DiskKCIterator] + private val mergeBeforeSpill: Boolean = mergeCombiners != null + + private val map: SpillableAppendOnlyMap[K, V, _, C] = { + if (mergeBeforeSpill) { + println("* Merge before spill *") + new SpillableAppendOnlyMap[K, V, C, C] (createCombiner, + mergeValue, mergeCombiners, combinerIdentity, memoryThresholdMB) + } else { + println("* Merge after spill *") + new SpillableAppendOnlyMap[K, V, ArrayBuffer[V], C] (createGroup, + mergeValueIntoGroup, mergeGroups, combineGroup, memoryThresholdMB) + } + } + + def insert(key: K, value: V): Unit = map.insert(key, value) + + override def iterator: Iterator[(K, C)] = map.iterator + + private def combinerIdentity(combiner: C): C = combiner + private def createGroup(value: V): ArrayBuffer[V] = ArrayBuffer[V](value) + private def mergeValueIntoGroup(group: ArrayBuffer[V], value: V): ArrayBuffer[V] = { + group += value + group + } + private def mergeGroups(group1: ArrayBuffer[V], group2: ArrayBuffer[V]): ArrayBuffer[V] = { + group1 ++= group2 + group1 + } + private def combineGroup(group: ArrayBuffer[V]): C = { + var combiner : Option[C] = None + group.foreach { v => + combiner match { + case None => combiner = Some(createCombiner(v)) + case Some(c) => combiner = Some(mergeValue(c, v)) + } + } + combiner.get + } +} + +/** + * An append-only map that spills sorted content to disk when the memory threshold + * is exceeded. A group with type M is an intermediate combiner, and shares the same + * type as either C or ArrayBuffer[V]. + */ +class SpillableAppendOnlyMap[K, V, M, C] (createGroup: V => M, + mergeValue: (M, V) => M, + mergeGroups: (M, M) => M, + createCombiner: M => C, + memoryThresholdMB: Int = 1024) + extends Iterable[(K, C)] with Serializable { + + var currentMap = new AppendOnlyMap[K, M] + var oldMaps = new ArrayBuffer[DiskIterator] def insert(key: K, value: V): Unit = { - currentMap.changeValue(key, updateFunction(value)) + def update(hadVal: Boolean, oldVal: M): M = { + if (hadVal) mergeValue(oldVal, value) else createGroup(value) + } + currentMap.changeValue(key, update) val mapSize = SizeEstimator.estimate(currentMap) - if (mapSize > memoryThresholdMB * math.pow(1024, 2)) { + //if (mapSize > memoryThresholdMB * math.pow(1024, 2)) { + if (mapSize > 1024 * 10) { spill() } } - def updateFunction(value: V) : (Boolean, C) => C = { - (hadVal: Boolean, oldVal: C) => - if (hadVal) mergeValue(oldVal, value) else createCombiner(value) - } - def spill(): Unit = { + println("> SPILL <") val file = File.createTempFile("external_append_only_map", "") // Add spill location val out = new ObjectOutputStream(new FileOutputStream(file)) val sortedMap = currentMap.iterator.toList.sortBy(kv => kv._1.hashCode()) sortedMap.foreach { out.writeObject( _ ) } out.close() - currentMap = new AppendOnlyMap[K, C] - oldMaps.append(new DiskKCIterator(file)) + currentMap = new AppendOnlyMap[K, M] + oldMaps.append(new DiskIterator(file)) } override def iterator: Iterator[(K, C)] = new ExternalIterator() - /** - * An iterator that merges KV pairs from memory and disk in sorted order - */ + // An iterator that sort-merges (K, M) pairs from memory and disk into (K, C) pairs class ExternalIterator extends Iterator[(K, C)] { // Order by increasing key hash value - implicit object KVOrdering extends Ordering[KCITuple] { - def compare(a:KCITuple, b:KCITuple) = -a.key.hashCode().compareTo(b.key.hashCode()) + implicit object KVOrdering extends Ordering[KMITuple] { + def compare(a:KMITuple, b:KMITuple) = -a.key.hashCode().compareTo(b.key.hashCode()) } - val pq = mutable.PriorityQueue[KCITuple]() - val inputStreams = Seq(new MemoryKCIterator(currentMap)) ++ oldMaps + val pq = PriorityQueue[KMITuple]() + val inputStreams = Seq(new MemoryIterator(currentMap)) ++ oldMaps inputStreams.foreach { readFromIterator( _ ) } - override def hasNext: Boolean = !pq.isEmpty - - // Combine all values from all input streams corresponding to the same key - override def next(): (K, C) = { - val minKCI = pq.dequeue() - var (minKey, minCombiner) = (minKCI.key, minKCI.combiner) - val minHash = minKey.hashCode() - readFromIterator(minKCI.iter) - - var collidedKCI = ArrayBuffer[KCITuple]() - while (!pq.isEmpty && pq.head.key.hashCode() == minHash) { - val newKCI: KCITuple = pq.dequeue() - if (newKCI.key == minKey){ - minCombiner = mergeCombiners(minCombiner, newKCI.combiner) - readFromIterator(newKCI.iter) - } else { - // Collision - collidedKCI += newKCI - } - } - collidedKCI.foreach { pq.enqueue( _ ) } - (minKey, minCombiner) - } - - // Read from the given iterator until a key of different hash is retrieved, - // Add each KC pair read from this iterator to the heap - def readFromIterator(iter: Iterator[(K, C)]): Unit = { + // Read from the given iterator until a key of different hash is retrieved + def readFromIterator(iter: Iterator[(K, M)]): Unit = { var minHash : Option[Int] = None while (iter.hasNext) { - val (k, c) = iter.next() - pq.enqueue(KCITuple(k, c, iter)) + val (k, m) = iter.next() + pq.enqueue(KMITuple(k, m, iter)) minHash match { case None => minHash = Some(k.hashCode()) case Some(expectedHash) => @@ -112,40 +142,63 @@ class ExternalAppendOnlyMap[K, V, C] (createCombiner: V => C, } } - case class KCITuple(key:K, combiner:C, iter:Iterator[(K, C)]) + override def hasNext: Boolean = !pq.isEmpty + + override def next(): (K, C) = { + val minKMI = pq.dequeue() + var (minKey, minGroup) = (minKMI.key, minKMI.group) + val minHash = minKey.hashCode() + readFromIterator(minKMI.iterator) + + // Merge groups with the same key into minGroup + var collidedKMI = ArrayBuffer[KMITuple]() + while (!pq.isEmpty && pq.head.key.hashCode() == minHash) { + val newKMI = pq.dequeue() + if (newKMI.key == minKey) { + minGroup = mergeGroups(minGroup, newKMI.group) + readFromIterator(newKMI.iterator) + } else { + // Collision + collidedKMI += newKMI + } + } + collidedKMI.foreach { pq.enqueue( _ ) } + (minKey, createCombiner(minGroup)) + } + + case class KMITuple(key:K, group:M, iterator:Iterator[(K, M)]) } - class MemoryKCIterator(map: AppendOnlyMap[K, C]) extends Iterator[(K, C)] { + // Iterate through (K, M) pairs in sorted order from the in-memory map + class MemoryIterator(map: AppendOnlyMap[K, M]) extends Iterator[(K, M)] { val sortedMap = currentMap.iterator.toList.sortBy(kc => kc._1.hashCode()) val it = sortedMap.iterator override def hasNext: Boolean = it.hasNext - override def next(): (K, C) = it.next() + override def next(): (K, M) = it.next() } - class DiskKCIterator(file: File) extends Iterator[(K, C)] { + // Iterate through (K, M) pairs in sorted order from an on-disk map + class DiskIterator(file: File) extends Iterator[(K, M)] { val in = new ObjectInputStream(new FileInputStream(file)) - var nextItem:(K, C) = _ - var eof = false + var nextItem: Option[(K, M)] = None override def hasNext: Boolean = { - if (eof) { - return false - } try { - nextItem = in.readObject().asInstanceOf[(K, C)] + nextItem = Some(in.readObject().asInstanceOf[(K, M)]) + true } catch { case e: EOFException => - eof = true - return false + nextItem = None + false } - true } - override def next(): (K, C) = { - if (eof) { - throw new NoSuchElementException + override def next(): (K, M) = { + nextItem match { + case Some(item) => item + case None => + throw new NoSuchElementException } - nextItem } } } -- cgit v1.2.3 From 2a2ca2a6610571bac45cdcedbf0c30927cea7c22 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 25 Dec 2013 17:32:01 -0800 Subject: Add toggle for ExternalAppendOnlyMap in Aggregator and CoGroupedRDD --- .../main/scala/org/apache/spark/Aggregator.scala | 47 ++++++++++++++++------ .../scala/org/apache/spark/rdd/CoGroupedRDD.scala | 39 +++++++++++++----- .../apache/spark/util/ExternalAppendOnlyMap.scala | 3 -- 3 files changed, 65 insertions(+), 24 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index c51fb1d630..ecaeb2d9b8 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -32,22 +32,45 @@ case class Aggregator[K, V, C] ( mergeCombiners: (C, C) => C) { def combineValuesByKey(iter: Iterator[_ <: Product2[K, V]]) : Iterator[(K, C)] = { - //val combiners = new AppendOnlyMap[K, C] - val combiners = new ExternalAppendOnlyMap[K, V, C](createCombiner, mergeValue, mergeCombiners) - while (iter.hasNext) { - val kv = iter.next() - combiners.insert(kv._1, kv._2) + val externalSorting = System.getProperty("spark.shuffle.externalSorting", "false").toBoolean + if (!externalSorting) { + val combiners = new AppendOnlyMap[K,C] + var kv: Product2[K, V] = null + val update = (hadValue: Boolean, oldValue: C) => { + if (hadValue) mergeValue(oldValue, kv._2) else createCombiner(kv._2) + } + while (iter.hasNext) { + kv = iter.next() + combiners.changeValue(kv._1, update) + } + combiners.iterator + } else { + // Spilling + val combiners = new ExternalAppendOnlyMap[K, V, C](createCombiner, mergeValue, mergeCombiners) + iter.foreach { case(k, v) => combiners.insert(k, v) } + combiners.iterator } - combiners.iterator } def combineCombinersByKey(iter: Iterator[(K, C)]) : Iterator[(K, C)] = { - //val combiners = new AppendOnlyMap[K, C] - val combiners = new ExternalAppendOnlyMap[K, C, C]((c:C) => c, mergeCombiners, mergeCombiners) - while (iter.hasNext) { - val kc = iter.next() - combiners.insert(kc._1, kc._2) + val externalSorting = System.getProperty("spark.shuffle.externalSorting", "false").toBoolean + if (!externalSorting) { + val combiners = new AppendOnlyMap[K,C] + var kc: Product2[K, C] = null + val update = (hadValue: Boolean, oldValue: C) => { + if (hadValue) mergeCombiners(oldValue, kc._2) else kc._2 + } + while (iter.hasNext) { + kc = iter.next() + combiners.changeValue(kc._1, update) + } + combiners.iterator + } else { + // Spilling + def combinerIdentity(combiner:C) = combiner + val combiners = new ExternalAppendOnlyMap[K, C, C](combinerIdentity, mergeCombiners, mergeCombiners) + iter.foreach { case(k, c) => combiners.insert(k, c) } + combiners.iterator } - combiners.iterator } } \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index a2a3de7d88..a7265f3cac 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -102,28 +102,49 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: override def compute(s: Partition, context: TaskContext): Iterator[(K, CoGroupCombiner)] = { // e.g. for `(k, a) cogroup (k, b)`, K -> Seq(ArrayBuffer as, ArrayBuffer bs) + val externalSorting = System.getProperty("spark.shuffle.externalSorting", "false").toBoolean val split = s.asInstanceOf[CoGroupPartition] val numRdds = split.deps.size - //val combiners = new AppendOnlyMap[K, Seq[ArrayBuffer[Any]]] - val combiners = createExternalMap(numRdds) - val ser = SparkEnv.get.serializerManager.get(serializerClass) + + // A list of (rdd iterator, dependency number) pairs + val rddIterators = new ArrayBuffer[(Iterator[Product2[K, Any]], Int)] for ((dep, depNum) <- split.deps.zipWithIndex) dep match { case NarrowCoGroupSplitDep(rdd, _, itsSplit) => { // Read them from the parent - rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, Any]]].foreach { - kv => combiners.insert(kv._1, new CoGroupValue(kv._2, depNum)) - } + val v = (rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, Any]]], depNum) + rddIterators += v } case ShuffleCoGroupSplitDep(shuffleId) => { // Read map outputs of shuffle val fetcher = SparkEnv.get.shuffleFetcher - fetcher.fetch[Product2[K, Any]](shuffleId, split.index, context, ser).foreach { - kv => combiners.insert(kv._1, new CoGroupValue(kv._2, depNum)) + val v = (fetcher.fetch[Product2[K, Any]](shuffleId, split.index, context, ser), depNum) + rddIterators += v + } + } + + if (!externalSorting) { + val map = new AppendOnlyMap[K, CoGroupCombiner] + val update: (Boolean, Seq[ArrayBuffer[Any]]) => Seq[ArrayBuffer[Any]] = (hadVal, oldVal) => { + if (hadVal) oldVal else Array.fill(numRdds)(new ArrayBuffer[Any]) + } + val getSeq = (k: K) => map.changeValue(k, update) + rddIterators.foreach { case(iter, depNum) => + iter.foreach { + case(k, v) => getSeq(k)(depNum) += v + } + } + new InterruptibleIterator(context, map.iterator) + } else { + // Spilling + val map = createExternalMap(numRdds) + rddIterators.foreach { case(iter, depNum) => + iter.foreach { + case(k, v) => map.insert(k, new CoGroupValue(v, depNum)) } } + new InterruptibleIterator(context, map.iterator) } - new InterruptibleIterator(context, combiners.iterator) } private def createExternalMap(numRdds:Int): ExternalAppendOnlyMap [K, CoGroupValue, CoGroupCombiner] = { diff --git a/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala index e2205c6063..790dcf06df 100644 --- a/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala @@ -39,11 +39,9 @@ class ExternalAppendOnlyMap[K, V, C] (createCombiner: V => C, private val map: SpillableAppendOnlyMap[K, V, _, C] = { if (mergeBeforeSpill) { - println("* Merge before spill *") new SpillableAppendOnlyMap[K, V, C, C] (createCombiner, mergeValue, mergeCombiners, combinerIdentity, memoryThresholdMB) } else { - println("* Merge after spill *") new SpillableAppendOnlyMap[K, V, ArrayBuffer[V], C] (createGroup, mergeValueIntoGroup, mergeGroups, combineGroup, memoryThresholdMB) } @@ -103,7 +101,6 @@ class SpillableAppendOnlyMap[K, V, M, C] (createGroup: V => M, } def spill(): Unit = { - println("> SPILL <") val file = File.createTempFile("external_append_only_map", "") // Add spill location val out = new ObjectOutputStream(new FileOutputStream(file)) val sortedMap = currentMap.iterator.toList.sortBy(kv => kv._1.hashCode()) -- cgit v1.2.3 From fcc443b3db3664987a6f863b59c06be7169175d5 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Wed, 25 Dec 2013 18:42:06 -0800 Subject: Minor cleanup for Scala style --- .../main/scala/org/apache/spark/Aggregator.scala | 6 +- .../scala/org/apache/spark/rdd/CoGroupedRDD.scala | 28 ++++---- .../apache/spark/util/ExternalAppendOnlyMap.scala | 76 +++++++++++----------- 3 files changed, 55 insertions(+), 55 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index ecaeb2d9b8..582625577f 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -67,10 +67,10 @@ case class Aggregator[K, V, C] ( combiners.iterator } else { // Spilling - def combinerIdentity(combiner:C) = combiner - val combiners = new ExternalAppendOnlyMap[K, C, C](combinerIdentity, mergeCombiners, mergeCombiners) + val combiners = + new ExternalAppendOnlyMap[K, C, C](Predef.identity, mergeCombiners, mergeCombiners) iter.foreach { case(k, c) => combiners.insert(k, c) } combiners.iterator } } -} \ No newline at end of file +} diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index a7265f3cac..3af0376a4d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -25,7 +25,6 @@ import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency} import org.apache.spark.util.{AppendOnlyMap, ExternalAppendOnlyMap} - private[spark] sealed trait CoGroupSplitDep extends Serializable private[spark] case class NarrowCoGroupSplitDep( @@ -62,6 +61,10 @@ class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep]) class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: Partitioner) extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) { + type CoGroup = ArrayBuffer[Any] + type CoGroupValue = (Any, Int) // Int is dependency number + type CoGroupCombiner = Seq[CoGroup] + private var serializerClass: String = null def setSerializer(cls: String): CoGroupedRDD[K] = { @@ -125,7 +128,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: if (!externalSorting) { val map = new AppendOnlyMap[K, CoGroupCombiner] - val update: (Boolean, Seq[ArrayBuffer[Any]]) => Seq[ArrayBuffer[Any]] = (hadVal, oldVal) => { + val update: (Boolean, CoGroupCombiner) => CoGroupCombiner = (hadVal, oldVal) => { if (hadVal) oldVal else Array.fill(numRdds)(new ArrayBuffer[Any]) } val getSeq = (k: K) => map.changeValue(k, update) @@ -147,30 +150,29 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: } } - private def createExternalMap(numRdds:Int): ExternalAppendOnlyMap [K, CoGroupValue, CoGroupCombiner] = { - def createCombiner(v: CoGroupValue): CoGroupCombiner = { + private def createExternalMap(numRdds: Int) + : ExternalAppendOnlyMap[K, CoGroupValue, CoGroupCombiner] = { + + val createCombiner: (CoGroupValue) => CoGroupCombiner = v => { val newCombiner = Array.fill(numRdds)(new CoGroup) - mergeValue(newCombiner, v) + v match { case (value, depNum) => newCombiner(depNum) += value } + newCombiner } - def mergeValue(c: CoGroupCombiner, v: CoGroupValue): CoGroupCombiner = { + val mergeValue: (CoGroupCombiner, CoGroupValue) => CoGroupCombiner = (c, v) => { v match { case (value, depNum) => c(depNum) += value } c } - def mergeCombiners(c1: CoGroupCombiner, c2: CoGroupCombiner): CoGroupCombiner = { + val mergeCombiners: (CoGroupCombiner, CoGroupCombiner) => CoGroupCombiner = (c1, c2) => { c1.zipAll(c2, new CoGroup, new CoGroup).map { case (v1, v2) => v1 ++ v2 } } - new ExternalAppendOnlyMap [K, CoGroupValue, CoGroupCombiner] ( - createCombiner,mergeValue, mergeCombiners) + new ExternalAppendOnlyMap[K, CoGroupValue, CoGroupCombiner]( + createCombiner, mergeValue, mergeCombiners) } override def clearDependencies() { super.clearDependencies() rdds = null } - - type CoGroup = ArrayBuffer[Any] - type CoGroupValue = (Any, Int) // Int is dependency number - type CoGroupCombiner = Seq[CoGroup] } diff --git a/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala index 790dcf06df..c8c053460c 100644 --- a/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala @@ -18,21 +18,27 @@ package org.apache.spark.util import java.io._ +import java.text.DecimalFormat + +import scala.Some +import scala.Predef._ import scala.collection.mutable.{ArrayBuffer, PriorityQueue} +import scala.util.Random /** * A wrapper for SpillableAppendOnlyMap that handles two cases: * * (1) If a mergeCombiners function is specified, merge values into combiners before - * disk spill, as it is possible to merge the resulting combiners later + * disk spill, as it is possible to merge the resulting combiners later. * * (2) Otherwise, group values of the same key together before disk spill, and merge - * them into combiners only after reading them back from disk + * them into combiners only after reading them back from disk. */ -class ExternalAppendOnlyMap[K, V, C] (createCombiner: V => C, - mergeValue: (C, V) => C, - mergeCombiners: (C, C) => C, - memoryThresholdMB: Int = 1024) +class ExternalAppendOnlyMap[K, V, C]( + createCombiner: V => C, + mergeValue: (C, V) => C, + mergeCombiners: (C, C) => C, + memoryThresholdMB: Long = 1024) extends Iterable[(K, C)] with Serializable { private val mergeBeforeSpill: Boolean = mergeCombiners != null @@ -40,8 +46,9 @@ class ExternalAppendOnlyMap[K, V, C] (createCombiner: V => C, private val map: SpillableAppendOnlyMap[K, V, _, C] = { if (mergeBeforeSpill) { new SpillableAppendOnlyMap[K, V, C, C] (createCombiner, - mergeValue, mergeCombiners, combinerIdentity, memoryThresholdMB) + mergeValue, mergeCombiners, Predef.identity, memoryThresholdMB) } else { + val createGroup: (V => ArrayBuffer[V]) = value => ArrayBuffer[V](value) new SpillableAppendOnlyMap[K, V, ArrayBuffer[V], C] (createGroup, mergeValueIntoGroup, mergeGroups, combineGroup, memoryThresholdMB) } @@ -51,8 +58,6 @@ class ExternalAppendOnlyMap[K, V, C] (createCombiner: V => C, override def iterator: Iterator[(K, C)] = map.iterator - private def combinerIdentity(combiner: C): C = combiner - private def createGroup(value: V): ArrayBuffer[V] = ArrayBuffer[V](value) private def mergeValueIntoGroup(group: ArrayBuffer[V], value: V): ArrayBuffer[V] = { group += value group @@ -78,14 +83,16 @@ class ExternalAppendOnlyMap[K, V, C] (createCombiner: V => C, * is exceeded. A group with type M is an intermediate combiner, and shares the same * type as either C or ArrayBuffer[V]. */ -class SpillableAppendOnlyMap[K, V, M, C] (createGroup: V => M, - mergeValue: (M, V) => M, - mergeGroups: (M, M) => M, - createCombiner: M => C, - memoryThresholdMB: Int = 1024) +class SpillableAppendOnlyMap[K, V, M, C]( + createGroup: V => M, + mergeValue: (M, V) => M, + mergeGroups: (M, M) => M, + createCombiner: M => C, + memoryThresholdMB: Long = 1024) extends Iterable[(K, C)] with Serializable { var currentMap = new AppendOnlyMap[K, M] + var sizeTracker = new SamplingSizeTracker(currentMap) var oldMaps = new ArrayBuffer[DiskIterator] def insert(key: K, value: V): Unit = { @@ -93,9 +100,8 @@ class SpillableAppendOnlyMap[K, V, M, C] (createGroup: V => M, if (hadVal) mergeValue(oldVal, value) else createGroup(value) } currentMap.changeValue(key, update) - val mapSize = SizeEstimator.estimate(currentMap) - //if (mapSize > memoryThresholdMB * math.pow(1024, 2)) { - if (mapSize > 1024 * 10) { + sizeTracker.updateMade() + if (sizeTracker.estimateSize() > memoryThresholdMB * 1024 * 1024) { spill() } } @@ -104,9 +110,10 @@ class SpillableAppendOnlyMap[K, V, M, C] (createGroup: V => M, val file = File.createTempFile("external_append_only_map", "") // Add spill location val out = new ObjectOutputStream(new FileOutputStream(file)) val sortedMap = currentMap.iterator.toList.sortBy(kv => kv._1.hashCode()) - sortedMap.foreach { out.writeObject( _ ) } + sortedMap.foreach(out.writeObject) out.close() currentMap = new AppendOnlyMap[K, M] + sizeTracker = new SamplingSizeTracker(currentMap) oldMaps.append(new DiskIterator(file)) } @@ -115,13 +122,10 @@ class SpillableAppendOnlyMap[K, V, M, C] (createGroup: V => M, // An iterator that sort-merges (K, M) pairs from memory and disk into (K, C) pairs class ExternalIterator extends Iterator[(K, C)] { - // Order by increasing key hash value - implicit object KVOrdering extends Ordering[KMITuple] { - def compare(a:KMITuple, b:KMITuple) = -a.key.hashCode().compareTo(b.key.hashCode()) - } - val pq = PriorityQueue[KMITuple]() + // Order by key hash value + val pq = PriorityQueue[KMITuple]()(Ordering.by(_.key.hashCode())) val inputStreams = Seq(new MemoryIterator(currentMap)) ++ oldMaps - inputStreams.foreach { readFromIterator( _ ) } + inputStreams.foreach(readFromIterator) // Read from the given iterator until a key of different hash is retrieved def readFromIterator(iter: Iterator[(K, M)]): Unit = { @@ -131,10 +135,7 @@ class SpillableAppendOnlyMap[K, V, M, C] (createGroup: V => M, pq.enqueue(KMITuple(k, m, iter)) minHash match { case None => minHash = Some(k.hashCode()) - case Some(expectedHash) => - if (k.hashCode() != expectedHash){ - return - } + case Some(expectedHash) if k.hashCode() != expectedHash => return } } } @@ -159,16 +160,16 @@ class SpillableAppendOnlyMap[K, V, M, C] (createGroup: V => M, collidedKMI += newKMI } } - collidedKMI.foreach { pq.enqueue( _ ) } + collidedKMI.foreach(pq.enqueue(_)) (minKey, createCombiner(minGroup)) } - case class KMITuple(key:K, group:M, iterator:Iterator[(K, M)]) + case class KMITuple(key: K, group: M, iterator: Iterator[(K, M)]) } // Iterate through (K, M) pairs in sorted order from the in-memory map class MemoryIterator(map: AppendOnlyMap[K, M]) extends Iterator[(K, M)] { - val sortedMap = currentMap.iterator.toList.sortBy(kc => kc._1.hashCode()) + val sortedMap = currentMap.iterator.toList.sortBy(km => km._1.hashCode()) val it = sortedMap.iterator override def hasNext: Boolean = it.hasNext override def next(): (K, M) = it.next() @@ -180,21 +181,18 @@ class SpillableAppendOnlyMap[K, V, M, C] (createGroup: V => M, var nextItem: Option[(K, M)] = None override def hasNext: Boolean = { - try { - nextItem = Some(in.readObject().asInstanceOf[(K, M)]) - true + nextItem = try { + Some(in.readObject().asInstanceOf[(K, M)]) } catch { - case e: EOFException => - nextItem = None - false + case e: EOFException => None } + nextItem.isDefined } override def next(): (K, M) = { nextItem match { case Some(item) => item - case None => - throw new NoSuchElementException + case None => throw new NoSuchElementException } } } -- cgit v1.2.3 From 7ad4408255e37f95e545d9c21a4460cbf98c05dd Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 25 Dec 2013 23:10:53 -0800 Subject: New minor edits --- .../main/scala/org/apache/spark/Aggregator.scala | 5 +- .../scala/org/apache/spark/rdd/CoGroupedRDD.scala | 37 +++++++------ .../apache/spark/util/ExternalAppendOnlyMap.scala | 61 ++++++++++------------ 3 files changed, 49 insertions(+), 54 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index 582625577f..8863c3175b 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -45,8 +45,8 @@ case class Aggregator[K, V, C] ( } combiners.iterator } else { - // Spilling - val combiners = new ExternalAppendOnlyMap[K, V, C](createCombiner, mergeValue, mergeCombiners) + val combiners = + new ExternalAppendOnlyMap[K, V, C](createCombiner, mergeValue, mergeCombiners) iter.foreach { case(k, v) => combiners.insert(k, v) } combiners.iterator } @@ -66,7 +66,6 @@ case class Aggregator[K, V, C] ( } combiners.iterator } else { - // Spilling val combiners = new ExternalAppendOnlyMap[K, C, C](Predef.identity, mergeCombiners, mergeCombiners) iter.foreach { case(k, c) => combiners.insert(k, c) } diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 3af0376a4d..113a912f16 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -50,7 +50,6 @@ class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep]) override def hashCode(): Int = idx } - /** * A RDD that cogroups its parents. For each key k in parent RDDs, the resulting RDD contains a * tuple with the list of values for that key. @@ -108,7 +107,6 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: val externalSorting = System.getProperty("spark.shuffle.externalSorting", "false").toBoolean val split = s.asInstanceOf[CoGroupPartition] val numRdds = split.deps.size - val ser = SparkEnv.get.serializerManager.get(serializerClass) // A list of (rdd iterator, dependency number) pairs val rddIterators = new ArrayBuffer[(Iterator[Product2[K, Any]], Int)] @@ -121,6 +119,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: case ShuffleCoGroupSplitDep(shuffleId) => { // Read map outputs of shuffle val fetcher = SparkEnv.get.shuffleFetcher + val ser = SparkEnv.get.serializerManager.get(serializerClass) val v = (fetcher.fetch[Product2[K, Any]](shuffleId, split.index, context, ser), depNum) rddIterators += v } @@ -131,39 +130,39 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: val update: (Boolean, CoGroupCombiner) => CoGroupCombiner = (hadVal, oldVal) => { if (hadVal) oldVal else Array.fill(numRdds)(new ArrayBuffer[Any]) } - val getSeq = (k: K) => map.changeValue(k, update) - rddIterators.foreach { case(iter, depNum) => - iter.foreach { - case(k, v) => getSeq(k)(depNum) += v + rddIterators.foreach { case(it, depNum) => + it.foreach { case(k, v) => + map.changeValue(k, update)(depNum) += v } } new InterruptibleIterator(context, map.iterator) } else { - // Spilling val map = createExternalMap(numRdds) - rddIterators.foreach { case(iter, depNum) => - iter.foreach { - case(k, v) => map.insert(k, new CoGroupValue(v, depNum)) + rddIterators.foreach { case(it, depNum) => + it.foreach { case(k, v) => + map.insert(k, new CoGroupValue(v, depNum)) } } new InterruptibleIterator(context, map.iterator) } } - private def createExternalMap(numRdds: Int) - : ExternalAppendOnlyMap[K, CoGroupValue, CoGroupCombiner] = { + private def createExternalMap(numRdds: Int): + ExternalAppendOnlyMap[K, CoGroupValue, CoGroupCombiner] = { - val createCombiner: (CoGroupValue) => CoGroupCombiner = v => { + val createCombiner: (CoGroupValue => CoGroupCombiner) = value => { val newCombiner = Array.fill(numRdds)(new CoGroup) - v match { case (value, depNum) => newCombiner(depNum) += value } + value match { case(v, depNum) => newCombiner(depNum) += v } newCombiner } - val mergeValue: (CoGroupCombiner, CoGroupValue) => CoGroupCombiner = (c, v) => { - v match { case (value, depNum) => c(depNum) += value } - c + val mergeValue: (CoGroupCombiner, CoGroupValue) => CoGroupCombiner = + (combiner, value) => { + value match { case(v, depNum) => combiner(depNum) += v } + combiner } - val mergeCombiners: (CoGroupCombiner, CoGroupCombiner) => CoGroupCombiner = (c1, c2) => { - c1.zipAll(c2, new CoGroup, new CoGroup).map { + val mergeCombiners: (CoGroupCombiner, CoGroupCombiner) => CoGroupCombiner = + (combiner1, combiner2) => { + combiner1.zipAll(combiner2, new CoGroup, new CoGroup).map { case (v1, v2) => v1 ++ v2 } } diff --git a/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala index c8c053460c..413f83862d 100644 --- a/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala @@ -28,11 +28,11 @@ import scala.util.Random /** * A wrapper for SpillableAppendOnlyMap that handles two cases: * - * (1) If a mergeCombiners function is specified, merge values into combiners before - * disk spill, as it is possible to merge the resulting combiners later. + * (1) If a mergeCombiners function is specified, merge values into combiners before disk + * spill, as it is possible to merge the resulting combiners later. * - * (2) Otherwise, group values of the same key together before disk spill, and merge - * them into combiners only after reading them back from disk. + * (2) Otherwise, group values of the same key together before disk spill, and merge them + * into combiners only after reading them back from disk. */ class ExternalAppendOnlyMap[K, V, C]( createCombiner: V => C, @@ -48,8 +48,25 @@ class ExternalAppendOnlyMap[K, V, C]( new SpillableAppendOnlyMap[K, V, C, C] (createCombiner, mergeValue, mergeCombiners, Predef.identity, memoryThresholdMB) } else { + // Use ArrayBuffer[V] as the intermediate combiner val createGroup: (V => ArrayBuffer[V]) = value => ArrayBuffer[V](value) - new SpillableAppendOnlyMap[K, V, ArrayBuffer[V], C] (createGroup, + val mergeValueIntoGroup: (ArrayBuffer[V], V) => ArrayBuffer[V] = (group, value) => { + group += value + } + val mergeGroups: (ArrayBuffer[V], ArrayBuffer[V]) => ArrayBuffer[V] = (group1, group2) => { + group1 ++= group2 + } + val combineGroup: (ArrayBuffer[V] => C) = group => { + var combiner : Option[C] = None + group.foreach { v => + combiner match { + case None => combiner = Some(createCombiner(v)) + case Some(c) => combiner = Some(mergeValue(c, v)) + } + } + combiner.getOrElse(null.asInstanceOf[C]) + } + new SpillableAppendOnlyMap[K, V, ArrayBuffer[V], C](createGroup, mergeValueIntoGroup, mergeGroups, combineGroup, memoryThresholdMB) } } @@ -57,31 +74,11 @@ class ExternalAppendOnlyMap[K, V, C]( def insert(key: K, value: V): Unit = map.insert(key, value) override def iterator: Iterator[(K, C)] = map.iterator - - private def mergeValueIntoGroup(group: ArrayBuffer[V], value: V): ArrayBuffer[V] = { - group += value - group - } - private def mergeGroups(group1: ArrayBuffer[V], group2: ArrayBuffer[V]): ArrayBuffer[V] = { - group1 ++= group2 - group1 - } - private def combineGroup(group: ArrayBuffer[V]): C = { - var combiner : Option[C] = None - group.foreach { v => - combiner match { - case None => combiner = Some(createCombiner(v)) - case Some(c) => combiner = Some(mergeValue(c, v)) - } - } - combiner.get - } } /** - * An append-only map that spills sorted content to disk when the memory threshold - * is exceeded. A group with type M is an intermediate combiner, and shares the same - * type as either C or ArrayBuffer[V]. + * An append-only map that spills sorted content to disk when the memory threshold is exceeded. + * A group is an intermediate combiner, with type M equal to either C or ArrayBuffer[V]. */ class SpillableAppendOnlyMap[K, V, M, C]( createGroup: V => M, @@ -96,7 +93,7 @@ class SpillableAppendOnlyMap[K, V, M, C]( var oldMaps = new ArrayBuffer[DiskIterator] def insert(key: K, value: V): Unit = { - def update(hadVal: Boolean, oldVal: M): M = { + val update: (Boolean, M) => M = (hadVal, oldVal) => { if (hadVal) mergeValue(oldVal, value) else createGroup(value) } currentMap.changeValue(key, update) @@ -128,11 +125,11 @@ class SpillableAppendOnlyMap[K, V, M, C]( inputStreams.foreach(readFromIterator) // Read from the given iterator until a key of different hash is retrieved - def readFromIterator(iter: Iterator[(K, M)]): Unit = { + def readFromIterator(it: Iterator[(K, M)]): Unit = { var minHash : Option[Int] = None - while (iter.hasNext) { - val (k, m) = iter.next() - pq.enqueue(KMITuple(k, m, iter)) + while (it.hasNext) { + val (k, m) = it.next() + pq.enqueue(KMITuple(k, m, it)) minHash match { case None => minHash = Some(k.hashCode()) case Some(expectedHash) if k.hashCode() != expectedHash => return -- cgit v1.2.3 From 804beb43bebe50e88814c0ca702a51571cd044e7 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 26 Dec 2013 01:28:18 -0800 Subject: SamplingSizeTracker + Map + test suite --- .../org/apache/spark/util/AppendOnlyMap.scala | 2 +- .../apache/spark/util/ExternalAppendOnlyMap.scala | 16 ++-- .../apache/spark/util/SamplingSizeTracker.scala | 67 ++++++++++++++ .../collection/SizeTrackingAppendOnlyMap.scala | 27 ++++++ .../spark/util/SamplingSizeTrackerSuite.scala | 103 +++++++++++++++++++++ 5 files changed, 204 insertions(+), 11 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/util/SamplingSizeTracker.scala create mode 100644 core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala create mode 100644 core/src/test/scala/org/apache/spark/util/SamplingSizeTrackerSuite.scala diff --git a/core/src/main/scala/org/apache/spark/util/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/AppendOnlyMap.scala index 8bb4ee3bfa..899cd6ac14 100644 --- a/core/src/main/scala/org/apache/spark/util/AppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/AppendOnlyMap.scala @@ -190,7 +190,7 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi } /** Double the table's size and re-hash everything */ - private def growTable() { + protected def growTable() { val newCapacity = capacity * 2 if (newCapacity >= (1 << 30)) { // We can't make the table this big because we want an array of 2x diff --git a/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala index 413f83862d..b97b28282a 100644 --- a/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala @@ -18,12 +18,10 @@ package org.apache.spark.util import java.io._ -import java.text.DecimalFormat -import scala.Some -import scala.Predef._ import scala.collection.mutable.{ArrayBuffer, PriorityQueue} -import scala.util.Random + +import org.apache.spark.util.collection.SizeTrackingAppendOnlyMap /** * A wrapper for SpillableAppendOnlyMap that handles two cases: @@ -88,8 +86,7 @@ class SpillableAppendOnlyMap[K, V, M, C]( memoryThresholdMB: Long = 1024) extends Iterable[(K, C)] with Serializable { - var currentMap = new AppendOnlyMap[K, M] - var sizeTracker = new SamplingSizeTracker(currentMap) + var currentMap = new SizeTrackingAppendOnlyMap[K, M] var oldMaps = new ArrayBuffer[DiskIterator] def insert(key: K, value: V): Unit = { @@ -97,8 +94,8 @@ class SpillableAppendOnlyMap[K, V, M, C]( if (hadVal) mergeValue(oldVal, value) else createGroup(value) } currentMap.changeValue(key, update) - sizeTracker.updateMade() - if (sizeTracker.estimateSize() > memoryThresholdMB * 1024 * 1024) { + // TODO: Make sure we're only using some % of the actual threshold due to error + if (currentMap.estimateSize() > memoryThresholdMB * 1024 * 1024) { spill() } } @@ -109,8 +106,7 @@ class SpillableAppendOnlyMap[K, V, M, C]( val sortedMap = currentMap.iterator.toList.sortBy(kv => kv._1.hashCode()) sortedMap.foreach(out.writeObject) out.close() - currentMap = new AppendOnlyMap[K, M] - sizeTracker = new SamplingSizeTracker(currentMap) + currentMap = new SizeTrackingAppendOnlyMap[K, M] oldMaps.append(new DiskIterator(file)) } diff --git a/core/src/main/scala/org/apache/spark/util/SamplingSizeTracker.scala b/core/src/main/scala/org/apache/spark/util/SamplingSizeTracker.scala new file mode 100644 index 0000000000..2262b7d1be --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/SamplingSizeTracker.scala @@ -0,0 +1,67 @@ +package org.apache.spark.util + +import org.apache.spark.util.SamplingSizeTracker.Sample + +/** + * Estimates the size of an object as it grows, in bytes. + * We sample with a slow exponential back-off using the SizeEstimator to amortize the time, + * as each call to SizeEstimator can take a sizable amount of time (order of a few milliseconds). + * + * Users should call updateMade() every time their object is updated with new data, or + * flushSamples() if there is a non-linear change in object size (otherwise linear is assumed). + * Not threadsafe. + */ +class SamplingSizeTracker(obj: AnyRef) { + /** + * Controls the base of the exponential which governs the rate of sampling. + * E.g., a value of 2 would mean we sample at 1, 2, 4, 8, ... elements. + */ + private val SAMPLE_GROWTH_RATE = 1.1 + + private var lastLastSample: Sample = _ + private var lastSample: Sample = _ + + private var numUpdates: Long = _ + private var nextSampleNum: Long = _ + + flushSamples() + + /** Called after a non-linear change in the tracked object. Takes a new sample. */ + def flushSamples() { + numUpdates = 0 + nextSampleNum = 1 + // Throw out both prior samples to avoid overestimating delta. + lastSample = Sample(SizeEstimator.estimate(obj), 0) + lastLastSample = lastSample + } + + /** To be called after an update to the tracked object. Amortized O(1) time. */ + def updateMade() { + numUpdates += 1 + if (nextSampleNum == numUpdates) { + lastLastSample = lastSample + lastSample = Sample(SizeEstimator.estimate(obj), numUpdates) + nextSampleNum = math.ceil(numUpdates * SAMPLE_GROWTH_RATE).toLong + } + } + + /** Estimates the current size of the tracked object. O(1) time. */ + def estimateSize(): Long = { + val interpolatedDelta = + if (lastLastSample != null && lastLastSample != lastSample) { + (lastSample.size - lastLastSample.size).toDouble / + (lastSample.numUpdates - lastLastSample.numUpdates) + } else if (lastSample.numUpdates > 0) { + lastSample.size.toDouble / lastSample.numUpdates + } else { + 0 + } + val extrapolatedDelta = interpolatedDelta * (numUpdates - lastSample.numUpdates) + val estimate = lastSample.size + extrapolatedDelta + math.max(0, estimate).toLong + } +} + +object SamplingSizeTracker { + case class Sample(size: Long, numUpdates: Long) +} diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala new file mode 100644 index 0000000000..2b2417efd9 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala @@ -0,0 +1,27 @@ +package org.apache.spark.util.collection + +import org.apache.spark.util.{AppendOnlyMap, SamplingSizeTracker} + +/** Append-only map that keeps track of its estimated size in bytes. */ +class SizeTrackingAppendOnlyMap[K, V] extends AppendOnlyMap[K, V] { + + val sizeTracker = new SamplingSizeTracker(this) + + def estimateSize() = sizeTracker.estimateSize() + + override def update(key: K, value: V): Unit = { + super.update(key, value) + sizeTracker.updateMade() + } + + override def changeValue(key: K, updateFunc: (Boolean, V) => V): V = { + val newValue = super.changeValue(key, updateFunc) + sizeTracker.updateMade() + newValue + } + + override protected def growTable() { + super.growTable() + sizeTracker.flushSamples() + } +} diff --git a/core/src/test/scala/org/apache/spark/util/SamplingSizeTrackerSuite.scala b/core/src/test/scala/org/apache/spark/util/SamplingSizeTrackerSuite.scala new file mode 100644 index 0000000000..bd3ff5ff41 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/SamplingSizeTrackerSuite.scala @@ -0,0 +1,103 @@ +package org.apache.spark.util + +import scala.util.Random + +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +import org.apache.spark.util.SamplingSizeTrackerSuite.LargeDummyClass +import org.apache.spark.util.collection.SizeTrackingAppendOnlyMap + +class SamplingSizeTrackerSuite extends FunSuite with BeforeAndAfterAll { + val NORMAL_ERROR = 0.20 + val HIGH_ERROR = 0.30 + + test("fixed size insertions") { + testWith[Int, Long](10000, i => (i, i.toLong)) + testWith[Int, (Long, Long)](10000, i => (i, (i.toLong, i.toLong))) + testWith[Int, LargeDummyClass](10000, i => (i, new LargeDummyClass())) + } + + test("variable size insertions") { + val rand = new Random(123456789) + def randString(minLen: Int, maxLen: Int): String = { + "a" * (rand.nextInt(maxLen - minLen) + minLen) + } + testWith[Int, String](10000, i => (i, randString(0, 10))) + testWith[Int, String](10000, i => (i, randString(0, 100))) + testWith[Int, String](10000, i => (i, randString(90, 100))) + } + + test("updates") { + val rand = new Random(123456789) + def randString(minLen: Int, maxLen: Int): String = { + "a" * (rand.nextInt(maxLen - minLen) + minLen) + } + testWith[String, Int](10000, i => (randString(0, 10000), i)) + } + + def testWith[K, V](numElements: Int, makeElement: (Int) => (K, V)) { + val map = new SizeTrackingAppendOnlyMap[K, V]() + for (i <- 0 until numElements) { + val (k, v) = makeElement(i) + map(k) = v + expectWithinError(map, map.estimateSize(), if (i < 32) HIGH_ERROR else NORMAL_ERROR) + } + } + + def expectWithinError(obj: AnyRef, estimatedSize: Long, error: Double) { + val betterEstimatedSize = SizeEstimator.estimate(obj) + assert(betterEstimatedSize * (1 - error) < estimatedSize, + s"Estimated size $estimatedSize was less than expected size $betterEstimatedSize") + assert(betterEstimatedSize * (1 + 2 * error) > estimatedSize, + s"Estimated size $estimatedSize was greater than expected size $betterEstimatedSize") + } +} + +object SamplingSizeTrackerSuite { + // Speed test, for reproducibility of results. + // These could be highly non-deterministic in general, however. + // Results: + // AppendOnlyMap: 30 ms + // SizeTracker: 45 ms + // SizeEstimator: 1500 ms + def main(args: Array[String]) { + val numElements = 100000 + + val baseTimes = for (i <- 0 until 3) yield time { + val map = new AppendOnlyMap[Int, LargeDummyClass]() + for (i <- 0 until numElements) { + map(i) = new LargeDummyClass() + } + } + + val sampledTimes = for (i <- 0 until 3) yield time { + val map = new SizeTrackingAppendOnlyMap[Int, LargeDummyClass]() + for (i <- 0 until numElements) { + map(i) = new LargeDummyClass() + map.estimateSize() + } + } + + val unsampledTimes = for (i <- 0 until 3) yield time { + val map = new AppendOnlyMap[Int, LargeDummyClass]() + for (i <- 0 until numElements) { + map(i) = new LargeDummyClass() + SizeEstimator.estimate(map) + } + } + + println("Base: " + baseTimes) + println("SizeTracker (sampled): " + sampledTimes) + println("SizeEstimator (unsampled): " + unsampledTimes) + } + + def time(f: => Unit): Long = { + val start = System.currentTimeMillis() + f + System.currentTimeMillis() - start + } + + private class LargeDummyClass { + val arr = new Array[Int](100) + } +} \ No newline at end of file -- cgit v1.2.3 From 64b2d54a02ba885c29bdfeaf36701721352de8e6 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 26 Dec 2013 14:26:22 -0800 Subject: Move maps to util, and refactor more --- .../main/scala/org/apache/spark/Aggregator.scala | 2 +- .../scala/org/apache/spark/rdd/CoGroupedRDD.scala | 2 +- .../org/apache/spark/util/AppendOnlyMap.scala | 237 --------------------- .../apache/spark/util/ExternalAppendOnlyMap.scala | 192 ----------------- .../spark/util/collection/AppendOnlyMap.scala | 237 +++++++++++++++++++++ .../util/collection/ExternalAppendOnlyMap.scala | 191 +++++++++++++++++ .../collection/SizeTrackingAppendOnlyMap.scala | 2 +- .../org/apache/spark/util/AppendOnlyMapSuite.scala | 154 ------------- .../spark/util/SamplingSizeTrackerSuite.scala | 4 +- .../spark/util/collection/AppendOnlyMapSuite.scala | 154 +++++++++++++ 10 files changed, 587 insertions(+), 588 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/util/AppendOnlyMap.scala delete mode 100644 core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala create mode 100644 core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala create mode 100644 core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala delete mode 100644 core/src/test/scala/org/apache/spark/util/AppendOnlyMapSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index 8863c3175b..59e5102d3e 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -17,7 +17,7 @@ package org.apache.spark -import org.apache.spark.util.{AppendOnlyMap, ExternalAppendOnlyMap} +import org.apache.spark.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap} /** * A set of functions used to aggregate data. diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 113a912f16..3a549b7b4a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -23,7 +23,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv, TaskContext} import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency} -import org.apache.spark.util.{AppendOnlyMap, ExternalAppendOnlyMap} +import org.apache.spark.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap} private[spark] sealed trait CoGroupSplitDep extends Serializable diff --git a/core/src/main/scala/org/apache/spark/util/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/AppendOnlyMap.scala deleted file mode 100644 index 899cd6ac14..0000000000 --- a/core/src/main/scala/org/apache/spark/util/AppendOnlyMap.scala +++ /dev/null @@ -1,237 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.util - -/** - * A simple open hash table optimized for the append-only use case, where keys - * are never removed, but the value for each key may be changed. - * - * This implementation uses quadratic probing with a power-of-2 hash table - * size, which is guaranteed to explore all spaces for each key (see - * http://en.wikipedia.org/wiki/Quadratic_probing). - * - * TODO: Cache the hash values of each key? java.util.HashMap does that. - */ -private[spark] -class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] with Serializable { - require(initialCapacity <= (1 << 29), "Can't make capacity bigger than 2^29 elements") - require(initialCapacity >= 1, "Invalid initial capacity") - - private var capacity = nextPowerOf2(initialCapacity) - private var mask = capacity - 1 - private var curSize = 0 - private var growThreshold = LOAD_FACTOR * capacity - - // Holds keys and values in the same array for memory locality; specifically, the order of - // elements is key0, value0, key1, value1, key2, value2, etc. - private var data = new Array[AnyRef](2 * capacity) - - // Treat the null key differently so we can use nulls in "data" to represent empty items. - private var haveNullValue = false - private var nullValue: V = null.asInstanceOf[V] - - private val LOAD_FACTOR = 0.7 - - /** Get the value for a given key */ - def apply(key: K): V = { - val k = key.asInstanceOf[AnyRef] - if (k.eq(null)) { - return nullValue - } - var pos = rehash(k.hashCode) & mask - var i = 1 - while (true) { - val curKey = data(2 * pos) - if (k.eq(curKey) || k.equals(curKey)) { - return data(2 * pos + 1).asInstanceOf[V] - } else if (curKey.eq(null)) { - return null.asInstanceOf[V] - } else { - val delta = i - pos = (pos + delta) & mask - i += 1 - } - } - return null.asInstanceOf[V] - } - - /** Set the value for a key */ - def update(key: K, value: V): Unit = { - val k = key.asInstanceOf[AnyRef] - if (k.eq(null)) { - if (!haveNullValue) { - incrementSize() - } - nullValue = value - haveNullValue = true - return - } - var pos = rehash(key.hashCode) & mask - var i = 1 - while (true) { - val curKey = data(2 * pos) - if (curKey.eq(null)) { - data(2 * pos) = k - data(2 * pos + 1) = value.asInstanceOf[AnyRef] - incrementSize() // Since we added a new key - return - } else if (k.eq(curKey) || k.equals(curKey)) { - data(2 * pos + 1) = value.asInstanceOf[AnyRef] - return - } else { - val delta = i - pos = (pos + delta) & mask - i += 1 - } - } - } - - /** - * Set the value for key to updateFunc(hadValue, oldValue), where oldValue will be the old value - * for key, if any, or null otherwise. Returns the newly updated value. - */ - def changeValue(key: K, updateFunc: (Boolean, V) => V): V = { - val k = key.asInstanceOf[AnyRef] - if (k.eq(null)) { - if (!haveNullValue) { - incrementSize() - } - nullValue = updateFunc(haveNullValue, nullValue) - haveNullValue = true - return nullValue - } - var pos = rehash(k.hashCode) & mask - var i = 1 - while (true) { - val curKey = data(2 * pos) - if (k.eq(curKey) || k.equals(curKey)) { - val newValue = updateFunc(true, data(2 * pos + 1).asInstanceOf[V]) - data(2 * pos + 1) = newValue.asInstanceOf[AnyRef] - return newValue - } else if (curKey.eq(null)) { - val newValue = updateFunc(false, null.asInstanceOf[V]) - data(2 * pos) = k - data(2 * pos + 1) = newValue.asInstanceOf[AnyRef] - incrementSize() - return newValue - } else { - val delta = i - pos = (pos + delta) & mask - i += 1 - } - } - null.asInstanceOf[V] // Never reached but needed to keep compiler happy - } - - /** Iterator method from Iterable */ - override def iterator: Iterator[(K, V)] = new Iterator[(K, V)] { - var pos = -1 - - /** Get the next value we should return from next(), or null if we're finished iterating */ - def nextValue(): (K, V) = { - if (pos == -1) { // Treat position -1 as looking at the null value - if (haveNullValue) { - return (null.asInstanceOf[K], nullValue) - } - pos += 1 - } - while (pos < capacity) { - if (!data(2 * pos).eq(null)) { - return (data(2 * pos).asInstanceOf[K], data(2 * pos + 1).asInstanceOf[V]) - } - pos += 1 - } - null - } - - override def hasNext: Boolean = nextValue() != null - - override def next(): (K, V) = { - val value = nextValue() - if (value == null) { - throw new NoSuchElementException("End of iterator") - } - pos += 1 - value - } - } - - override def size: Int = curSize - - /** Increase table size by 1, rehashing if necessary */ - private def incrementSize() { - curSize += 1 - if (curSize > growThreshold) { - growTable() - } - } - - /** - * Re-hash a value to deal better with hash functions that don't differ in the lower bits. - * We use the Murmur Hash 3 finalization step that's also used in fastutil. - */ - private def rehash(h: Int): Int = { - it.unimi.dsi.fastutil.HashCommon.murmurHash3(h) - } - - /** Double the table's size and re-hash everything */ - protected def growTable() { - val newCapacity = capacity * 2 - if (newCapacity >= (1 << 30)) { - // We can't make the table this big because we want an array of 2x - // that size for our data, but array sizes are at most Int.MaxValue - throw new Exception("Can't make capacity bigger than 2^29 elements") - } - val newData = new Array[AnyRef](2 * newCapacity) - val newMask = newCapacity - 1 - // Insert all our old values into the new array. Note that because our old keys are - // unique, there's no need to check for equality here when we insert. - var oldPos = 0 - while (oldPos < capacity) { - if (!data(2 * oldPos).eq(null)) { - val key = data(2 * oldPos) - val value = data(2 * oldPos + 1) - var newPos = rehash(key.hashCode) & newMask - var i = 1 - var keepGoing = true - while (keepGoing) { - val curKey = newData(2 * newPos) - if (curKey.eq(null)) { - newData(2 * newPos) = key - newData(2 * newPos + 1) = value - keepGoing = false - } else { - val delta = i - newPos = (newPos + delta) & newMask - i += 1 - } - } - } - oldPos += 1 - } - data = newData - capacity = newCapacity - mask = newMask - growThreshold = LOAD_FACTOR * newCapacity - } - - private def nextPowerOf2(n: Int): Int = { - val highBit = Integer.highestOneBit(n) - if (highBit == n) n else highBit << 1 - } -} diff --git a/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala deleted file mode 100644 index b97b28282a..0000000000 --- a/core/src/main/scala/org/apache/spark/util/ExternalAppendOnlyMap.scala +++ /dev/null @@ -1,192 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.util - -import java.io._ - -import scala.collection.mutable.{ArrayBuffer, PriorityQueue} - -import org.apache.spark.util.collection.SizeTrackingAppendOnlyMap - -/** - * A wrapper for SpillableAppendOnlyMap that handles two cases: - * - * (1) If a mergeCombiners function is specified, merge values into combiners before disk - * spill, as it is possible to merge the resulting combiners later. - * - * (2) Otherwise, group values of the same key together before disk spill, and merge them - * into combiners only after reading them back from disk. - */ -class ExternalAppendOnlyMap[K, V, C]( - createCombiner: V => C, - mergeValue: (C, V) => C, - mergeCombiners: (C, C) => C, - memoryThresholdMB: Long = 1024) - extends Iterable[(K, C)] with Serializable { - - private val mergeBeforeSpill: Boolean = mergeCombiners != null - - private val map: SpillableAppendOnlyMap[K, V, _, C] = { - if (mergeBeforeSpill) { - new SpillableAppendOnlyMap[K, V, C, C] (createCombiner, - mergeValue, mergeCombiners, Predef.identity, memoryThresholdMB) - } else { - // Use ArrayBuffer[V] as the intermediate combiner - val createGroup: (V => ArrayBuffer[V]) = value => ArrayBuffer[V](value) - val mergeValueIntoGroup: (ArrayBuffer[V], V) => ArrayBuffer[V] = (group, value) => { - group += value - } - val mergeGroups: (ArrayBuffer[V], ArrayBuffer[V]) => ArrayBuffer[V] = (group1, group2) => { - group1 ++= group2 - } - val combineGroup: (ArrayBuffer[V] => C) = group => { - var combiner : Option[C] = None - group.foreach { v => - combiner match { - case None => combiner = Some(createCombiner(v)) - case Some(c) => combiner = Some(mergeValue(c, v)) - } - } - combiner.getOrElse(null.asInstanceOf[C]) - } - new SpillableAppendOnlyMap[K, V, ArrayBuffer[V], C](createGroup, - mergeValueIntoGroup, mergeGroups, combineGroup, memoryThresholdMB) - } - } - - def insert(key: K, value: V): Unit = map.insert(key, value) - - override def iterator: Iterator[(K, C)] = map.iterator -} - -/** - * An append-only map that spills sorted content to disk when the memory threshold is exceeded. - * A group is an intermediate combiner, with type M equal to either C or ArrayBuffer[V]. - */ -class SpillableAppendOnlyMap[K, V, M, C]( - createGroup: V => M, - mergeValue: (M, V) => M, - mergeGroups: (M, M) => M, - createCombiner: M => C, - memoryThresholdMB: Long = 1024) - extends Iterable[(K, C)] with Serializable { - - var currentMap = new SizeTrackingAppendOnlyMap[K, M] - var oldMaps = new ArrayBuffer[DiskIterator] - - def insert(key: K, value: V): Unit = { - val update: (Boolean, M) => M = (hadVal, oldVal) => { - if (hadVal) mergeValue(oldVal, value) else createGroup(value) - } - currentMap.changeValue(key, update) - // TODO: Make sure we're only using some % of the actual threshold due to error - if (currentMap.estimateSize() > memoryThresholdMB * 1024 * 1024) { - spill() - } - } - - def spill(): Unit = { - val file = File.createTempFile("external_append_only_map", "") // Add spill location - val out = new ObjectOutputStream(new FileOutputStream(file)) - val sortedMap = currentMap.iterator.toList.sortBy(kv => kv._1.hashCode()) - sortedMap.foreach(out.writeObject) - out.close() - currentMap = new SizeTrackingAppendOnlyMap[K, M] - oldMaps.append(new DiskIterator(file)) - } - - override def iterator: Iterator[(K, C)] = new ExternalIterator() - - // An iterator that sort-merges (K, M) pairs from memory and disk into (K, C) pairs - class ExternalIterator extends Iterator[(K, C)] { - - // Order by key hash value - val pq = PriorityQueue[KMITuple]()(Ordering.by(_.key.hashCode())) - val inputStreams = Seq(new MemoryIterator(currentMap)) ++ oldMaps - inputStreams.foreach(readFromIterator) - - // Read from the given iterator until a key of different hash is retrieved - def readFromIterator(it: Iterator[(K, M)]): Unit = { - var minHash : Option[Int] = None - while (it.hasNext) { - val (k, m) = it.next() - pq.enqueue(KMITuple(k, m, it)) - minHash match { - case None => minHash = Some(k.hashCode()) - case Some(expectedHash) if k.hashCode() != expectedHash => return - } - } - } - - override def hasNext: Boolean = !pq.isEmpty - - override def next(): (K, C) = { - val minKMI = pq.dequeue() - var (minKey, minGroup) = (minKMI.key, minKMI.group) - val minHash = minKey.hashCode() - readFromIterator(minKMI.iterator) - - // Merge groups with the same key into minGroup - var collidedKMI = ArrayBuffer[KMITuple]() - while (!pq.isEmpty && pq.head.key.hashCode() == minHash) { - val newKMI = pq.dequeue() - if (newKMI.key == minKey) { - minGroup = mergeGroups(minGroup, newKMI.group) - readFromIterator(newKMI.iterator) - } else { - // Collision - collidedKMI += newKMI - } - } - collidedKMI.foreach(pq.enqueue(_)) - (minKey, createCombiner(minGroup)) - } - - case class KMITuple(key: K, group: M, iterator: Iterator[(K, M)]) - } - - // Iterate through (K, M) pairs in sorted order from the in-memory map - class MemoryIterator(map: AppendOnlyMap[K, M]) extends Iterator[(K, M)] { - val sortedMap = currentMap.iterator.toList.sortBy(km => km._1.hashCode()) - val it = sortedMap.iterator - override def hasNext: Boolean = it.hasNext - override def next(): (K, M) = it.next() - } - - // Iterate through (K, M) pairs in sorted order from an on-disk map - class DiskIterator(file: File) extends Iterator[(K, M)] { - val in = new ObjectInputStream(new FileInputStream(file)) - var nextItem: Option[(K, M)] = None - - override def hasNext: Boolean = { - nextItem = try { - Some(in.readObject().asInstanceOf[(K, M)]) - } catch { - case e: EOFException => None - } - nextItem.isDefined - } - - override def next(): (K, M) = { - nextItem match { - case Some(item) => item - case None => throw new NoSuchElementException - } - } - } -} diff --git a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala new file mode 100644 index 0000000000..cb0ca8f8c1 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala @@ -0,0 +1,237 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection + +/** + * A simple open hash table optimized for the append-only use case, where keys + * are never removed, but the value for each key may be changed. + * + * This implementation uses quadratic probing with a power-of-2 hash table + * size, which is guaranteed to explore all spaces for each key (see + * http://en.wikipedia.org/wiki/Quadratic_probing). + * + * TODO: Cache the hash values of each key? java.util.HashMap does that. + */ +private[spark] +class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] with Serializable { + require(initialCapacity <= (1 << 29), "Can't make capacity bigger than 2^29 elements") + require(initialCapacity >= 1, "Invalid initial capacity") + + private var capacity = nextPowerOf2(initialCapacity) + private var mask = capacity - 1 + private var curSize = 0 + private var growThreshold = LOAD_FACTOR * capacity + + // Holds keys and values in the same array for memory locality; specifically, the order of + // elements is key0, value0, key1, value1, key2, value2, etc. + private var data = new Array[AnyRef](2 * capacity) + + // Treat the null key differently so we can use nulls in "data" to represent empty items. + private var haveNullValue = false + private var nullValue: V = null.asInstanceOf[V] + + private val LOAD_FACTOR = 0.7 + + /** Get the value for a given key */ + def apply(key: K): V = { + val k = key.asInstanceOf[AnyRef] + if (k.eq(null)) { + return nullValue + } + var pos = rehash(k.hashCode) & mask + var i = 1 + while (true) { + val curKey = data(2 * pos) + if (k.eq(curKey) || k.equals(curKey)) { + return data(2 * pos + 1).asInstanceOf[V] + } else if (curKey.eq(null)) { + return null.asInstanceOf[V] + } else { + val delta = i + pos = (pos + delta) & mask + i += 1 + } + } + return null.asInstanceOf[V] + } + + /** Set the value for a key */ + def update(key: K, value: V): Unit = { + val k = key.asInstanceOf[AnyRef] + if (k.eq(null)) { + if (!haveNullValue) { + incrementSize() + } + nullValue = value + haveNullValue = true + return + } + var pos = rehash(key.hashCode) & mask + var i = 1 + while (true) { + val curKey = data(2 * pos) + if (curKey.eq(null)) { + data(2 * pos) = k + data(2 * pos + 1) = value.asInstanceOf[AnyRef] + incrementSize() // Since we added a new key + return + } else if (k.eq(curKey) || k.equals(curKey)) { + data(2 * pos + 1) = value.asInstanceOf[AnyRef] + return + } else { + val delta = i + pos = (pos + delta) & mask + i += 1 + } + } + } + + /** + * Set the value for key to updateFunc(hadValue, oldValue), where oldValue will be the old value + * for key, if any, or null otherwise. Returns the newly updated value. + */ + def changeValue(key: K, updateFunc: (Boolean, V) => V): V = { + val k = key.asInstanceOf[AnyRef] + if (k.eq(null)) { + if (!haveNullValue) { + incrementSize() + } + nullValue = updateFunc(haveNullValue, nullValue) + haveNullValue = true + return nullValue + } + var pos = rehash(k.hashCode) & mask + var i = 1 + while (true) { + val curKey = data(2 * pos) + if (k.eq(curKey) || k.equals(curKey)) { + val newValue = updateFunc(true, data(2 * pos + 1).asInstanceOf[V]) + data(2 * pos + 1) = newValue.asInstanceOf[AnyRef] + return newValue + } else if (curKey.eq(null)) { + val newValue = updateFunc(false, null.asInstanceOf[V]) + data(2 * pos) = k + data(2 * pos + 1) = newValue.asInstanceOf[AnyRef] + incrementSize() + return newValue + } else { + val delta = i + pos = (pos + delta) & mask + i += 1 + } + } + null.asInstanceOf[V] // Never reached but needed to keep compiler happy + } + + /** Iterator method from Iterable */ + override def iterator: Iterator[(K, V)] = new Iterator[(K, V)] { + var pos = -1 + + /** Get the next value we should return from next(), or null if we're finished iterating */ + def nextValue(): (K, V) = { + if (pos == -1) { // Treat position -1 as looking at the null value + if (haveNullValue) { + return (null.asInstanceOf[K], nullValue) + } + pos += 1 + } + while (pos < capacity) { + if (!data(2 * pos).eq(null)) { + return (data(2 * pos).asInstanceOf[K], data(2 * pos + 1).asInstanceOf[V]) + } + pos += 1 + } + null + } + + override def hasNext: Boolean = nextValue() != null + + override def next(): (K, V) = { + val value = nextValue() + if (value == null) { + throw new NoSuchElementException("End of iterator") + } + pos += 1 + value + } + } + + override def size: Int = curSize + + /** Increase table size by 1, rehashing if necessary */ + private def incrementSize() { + curSize += 1 + if (curSize > growThreshold) { + growTable() + } + } + + /** + * Re-hash a value to deal better with hash functions that don't differ in the lower bits. + * We use the Murmur Hash 3 finalization step that's also used in fastutil. + */ + private def rehash(h: Int): Int = { + it.unimi.dsi.fastutil.HashCommon.murmurHash3(h) + } + + /** Double the table's size and re-hash everything */ + protected def growTable() { + val newCapacity = capacity * 2 + if (newCapacity >= (1 << 30)) { + // We can't make the table this big because we want an array of 2x + // that size for our data, but array sizes are at most Int.MaxValue + throw new Exception("Can't make capacity bigger than 2^29 elements") + } + val newData = new Array[AnyRef](2 * newCapacity) + val newMask = newCapacity - 1 + // Insert all our old values into the new array. Note that because our old keys are + // unique, there's no need to check for equality here when we insert. + var oldPos = 0 + while (oldPos < capacity) { + if (!data(2 * oldPos).eq(null)) { + val key = data(2 * oldPos) + val value = data(2 * oldPos + 1) + var newPos = rehash(key.hashCode) & newMask + var i = 1 + var keepGoing = true + while (keepGoing) { + val curKey = newData(2 * newPos) + if (curKey.eq(null)) { + newData(2 * newPos) = key + newData(2 * newPos + 1) = value + keepGoing = false + } else { + val delta = i + newPos = (newPos + delta) & newMask + i += 1 + } + } + } + oldPos += 1 + } + data = newData + capacity = newCapacity + mask = newMask + growThreshold = LOAD_FACTOR * newCapacity + } + + private def nextPowerOf2(n: Int): Int = { + val highBit = Integer.highestOneBit(n) + if (highBit == n) n else highBit << 1 + } +} diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala new file mode 100644 index 0000000000..4bda763ffe --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -0,0 +1,191 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection + +import java.io._ +import scala.collection.mutable.{ArrayBuffer, PriorityQueue} + +/** + * A wrapper for SpillableAppendOnlyMap that handles two cases: + * + * (1) If a mergeCombiners function is specified, merge values into combiners before disk + * spill, as it is possible to merge the resulting combiners later. + * + * (2) Otherwise, group values of the same key together before disk spill, and merge them + * into combiners only after reading them back from disk. + */ +class ExternalAppendOnlyMap[K, V, C]( + createCombiner: V => C, + mergeValue: (C, V) => C, + mergeCombiners: (C, C) => C) + extends Iterable[(K, C)] with Serializable { + + private val mergeBeforeSpill: Boolean = mergeCombiners != null + + private val map: SpillableAppendOnlyMap[K, V, _, C] = { + if (mergeBeforeSpill) { + new SpillableAppendOnlyMap[K, V, C, C] (createCombiner, mergeValue, + mergeCombiners, Predef.identity) + } else { + // Use ArrayBuffer[V] as the intermediate combiner + val createGroup: (V => ArrayBuffer[V]) = value => ArrayBuffer[V](value) + val mergeValueIntoGroup: (ArrayBuffer[V], V) => ArrayBuffer[V] = (group, value) => { + group += value + } + val mergeGroups: (ArrayBuffer[V], ArrayBuffer[V]) => ArrayBuffer[V] = (group1, group2) => { + group1 ++= group2 + } + val combineGroup: (ArrayBuffer[V] => C) = group => { + var combiner : Option[C] = None + group.foreach { v => + combiner match { + case None => combiner = Some(createCombiner(v)) + case Some(c) => combiner = Some(mergeValue(c, v)) + } + } + combiner.getOrElse(null.asInstanceOf[C]) + } + new SpillableAppendOnlyMap[K, V, ArrayBuffer[V], C](createGroup, mergeValueIntoGroup, + mergeGroups, combineGroup) + } + } + + def insert(key: K, value: V): Unit = map.insert(key, value) + + override def iterator: Iterator[(K, C)] = map.iterator +} + +/** + * An append-only map that spills sorted content to disk when the memory threshold is exceeded. + * A group is an intermediate combiner, with type M equal to either C or ArrayBuffer[V]. + */ +class SpillableAppendOnlyMap[K, V, M, C]( + createGroup: V => M, + mergeValue: (M, V) => M, + mergeGroups: (M, M) => M, + createCombiner: M => C) + extends Iterable[(K, C)] with Serializable { + + var currentMap = new SizeTrackingAppendOnlyMap[K, M] + val oldMaps = new ArrayBuffer[DiskIterator] + val memoryThreshold = { + val bufferSize = System.getProperty("spark.shuffle.buffer", "1024").toLong * 1024 * 1024 + val bufferPercent = System.getProperty("spark.shuffle.buffer.percent", "0.8").toFloat + bufferSize * bufferPercent + } + + def insert(key: K, value: V): Unit = { + val update: (Boolean, M) => M = (hadVal, oldVal) => { + if (hadVal) mergeValue(oldVal, value) else createGroup(value) + } + currentMap.changeValue(key, update) + if (currentMap.estimateSize() > memoryThreshold) { + spill() + } + } + + def spill(): Unit = { + val file = File.createTempFile("external_append_only_map", "") + val out = new ObjectOutputStream(new FileOutputStream(file)) + val sortedMap = currentMap.iterator.toList.sortBy(kv => kv._1.hashCode()) + sortedMap.foreach(out.writeObject) + out.close() + currentMap = new SizeTrackingAppendOnlyMap[K, M] + oldMaps.append(new DiskIterator(file)) + } + + override def iterator: Iterator[(K, C)] = new ExternalIterator() + + // An iterator that sort-merges (K, M) pairs from memory and disk into (K, C) pairs + class ExternalIterator extends Iterator[(K, C)] { + + // Order by key hash value + val pq = PriorityQueue[KMITuple]()(Ordering.by(_.key.hashCode())) + val inputStreams = Seq(new MemoryIterator(currentMap)) ++ oldMaps + inputStreams.foreach(readFromIterator) + + // Read from the given iterator until a key of different hash is retrieved + def readFromIterator(it: Iterator[(K, M)]): Unit = { + var minHash : Option[Int] = None + while (it.hasNext) { + val (k, m) = it.next() + pq.enqueue(KMITuple(k, m, it)) + minHash match { + case None => minHash = Some(k.hashCode()) + case Some(expectedHash) if k.hashCode() != expectedHash => return + } + } + } + + override def hasNext: Boolean = !pq.isEmpty + + override def next(): (K, C) = { + val minKMI = pq.dequeue() + var (minKey, minGroup) = (minKMI.key, minKMI.group) + val minHash = minKey.hashCode() + readFromIterator(minKMI.iterator) + + // Merge groups with the same key into minGroup + var collidedKMI = ArrayBuffer[KMITuple]() + while (!pq.isEmpty && pq.head.key.hashCode() == minHash) { + val newKMI = pq.dequeue() + if (newKMI.key == minKey) { + minGroup = mergeGroups(minGroup, newKMI.group) + readFromIterator(newKMI.iterator) + } else { + // Collision + collidedKMI += newKMI + } + } + collidedKMI.foreach(pq.enqueue(_)) + (minKey, createCombiner(minGroup)) + } + + case class KMITuple(key: K, group: M, iterator: Iterator[(K, M)]) + } + + // Iterate through (K, M) pairs in sorted order from the in-memory map + class MemoryIterator(map: AppendOnlyMap[K, M]) extends Iterator[(K, M)] { + val sortedMap = currentMap.iterator.toList.sortBy(km => km._1.hashCode()) + val it = sortedMap.iterator + override def hasNext: Boolean = it.hasNext + override def next(): (K, M) = it.next() + } + + // Iterate through (K, M) pairs in sorted order from an on-disk map + class DiskIterator(file: File) extends Iterator[(K, M)] { + val in = new ObjectInputStream(new FileInputStream(file)) + var nextItem: Option[(K, M)] = None + + override def hasNext: Boolean = { + nextItem = try { + Some(in.readObject().asInstanceOf[(K, M)]) + } catch { + case e: EOFException => None + } + nextItem.isDefined + } + + override def next(): (K, M) = { + nextItem match { + case Some(item) => item + case None => throw new NoSuchElementException + } + } + } +} diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala index 2b2417efd9..738908a660 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala @@ -1,6 +1,6 @@ package org.apache.spark.util.collection -import org.apache.spark.util.{AppendOnlyMap, SamplingSizeTracker} +import org.apache.spark.util.SamplingSizeTracker /** Append-only map that keeps track of its estimated size in bytes. */ class SizeTrackingAppendOnlyMap[K, V] extends AppendOnlyMap[K, V] { diff --git a/core/src/test/scala/org/apache/spark/util/AppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/AppendOnlyMapSuite.scala deleted file mode 100644 index 7177919a58..0000000000 --- a/core/src/test/scala/org/apache/spark/util/AppendOnlyMapSuite.scala +++ /dev/null @@ -1,154 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.util - -import scala.collection.mutable.HashSet - -import org.scalatest.FunSuite - -class AppendOnlyMapSuite extends FunSuite { - test("initialization") { - val goodMap1 = new AppendOnlyMap[Int, Int](1) - assert(goodMap1.size === 0) - val goodMap2 = new AppendOnlyMap[Int, Int](255) - assert(goodMap2.size === 0) - val goodMap3 = new AppendOnlyMap[Int, Int](256) - assert(goodMap3.size === 0) - intercept[IllegalArgumentException] { - new AppendOnlyMap[Int, Int](1 << 30) // Invalid map size: bigger than 2^29 - } - intercept[IllegalArgumentException] { - new AppendOnlyMap[Int, Int](-1) - } - intercept[IllegalArgumentException] { - new AppendOnlyMap[Int, Int](0) - } - } - - test("object keys and values") { - val map = new AppendOnlyMap[String, String]() - for (i <- 1 to 100) { - map("" + i) = "" + i - } - assert(map.size === 100) - for (i <- 1 to 100) { - assert(map("" + i) === "" + i) - } - assert(map("0") === null) - assert(map("101") === null) - assert(map(null) === null) - val set = new HashSet[(String, String)] - for ((k, v) <- map) { // Test the foreach method - set += ((k, v)) - } - assert(set === (1 to 100).map(_.toString).map(x => (x, x)).toSet) - } - - test("primitive keys and values") { - val map = new AppendOnlyMap[Int, Int]() - for (i <- 1 to 100) { - map(i) = i - } - assert(map.size === 100) - for (i <- 1 to 100) { - assert(map(i) === i) - } - assert(map(0) === null) - assert(map(101) === null) - val set = new HashSet[(Int, Int)] - for ((k, v) <- map) { // Test the foreach method - set += ((k, v)) - } - assert(set === (1 to 100).map(x => (x, x)).toSet) - } - - test("null keys") { - val map = new AppendOnlyMap[String, String]() - for (i <- 1 to 100) { - map("" + i) = "" + i - } - assert(map.size === 100) - assert(map(null) === null) - map(null) = "hello" - assert(map.size === 101) - assert(map(null) === "hello") - } - - test("null values") { - val map = new AppendOnlyMap[String, String]() - for (i <- 1 to 100) { - map("" + i) = null - } - assert(map.size === 100) - assert(map("1") === null) - assert(map(null) === null) - assert(map.size === 100) - map(null) = null - assert(map.size === 101) - assert(map(null) === null) - } - - test("changeValue") { - val map = new AppendOnlyMap[String, String]() - for (i <- 1 to 100) { - map("" + i) = "" + i - } - assert(map.size === 100) - for (i <- 1 to 100) { - val res = map.changeValue("" + i, (hadValue, oldValue) => { - assert(hadValue === true) - assert(oldValue === "" + i) - oldValue + "!" - }) - assert(res === i + "!") - } - // Iterate from 101 to 400 to make sure the map grows a couple of times, because we had a - // bug where changeValue would return the wrong result when the map grew on that insert - for (i <- 101 to 400) { - val res = map.changeValue("" + i, (hadValue, oldValue) => { - assert(hadValue === false) - i + "!" - }) - assert(res === i + "!") - } - assert(map.size === 400) - assert(map(null) === null) - map.changeValue(null, (hadValue, oldValue) => { - assert(hadValue === false) - "null!" - }) - assert(map.size === 401) - map.changeValue(null, (hadValue, oldValue) => { - assert(hadValue === true) - assert(oldValue === "null!") - "null!!" - }) - assert(map.size === 401) - } - - test("inserting in capacity-1 map") { - val map = new AppendOnlyMap[String, String](1) - for (i <- 1 to 100) { - map("" + i) = "" + i - } - assert(map.size === 100) - for (i <- 1 to 100) { - assert(map("" + i) === "" + i) - } - } -} diff --git a/core/src/test/scala/org/apache/spark/util/SamplingSizeTrackerSuite.scala b/core/src/test/scala/org/apache/spark/util/SamplingSizeTrackerSuite.scala index bd3ff5ff41..6b772131a7 100644 --- a/core/src/test/scala/org/apache/spark/util/SamplingSizeTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/SamplingSizeTrackerSuite.scala @@ -5,7 +5,7 @@ import scala.util.Random import org.scalatest.{BeforeAndAfterAll, FunSuite} import org.apache.spark.util.SamplingSizeTrackerSuite.LargeDummyClass -import org.apache.spark.util.collection.SizeTrackingAppendOnlyMap +import org.apache.spark.util.collection.{AppendOnlyMap, SizeTrackingAppendOnlyMap} class SamplingSizeTrackerSuite extends FunSuite with BeforeAndAfterAll { val NORMAL_ERROR = 0.20 @@ -100,4 +100,4 @@ object SamplingSizeTrackerSuite { private class LargeDummyClass { val arr = new Array[Int](100) } -} \ No newline at end of file +} diff --git a/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala new file mode 100644 index 0000000000..7e7aa7800d --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection + +import scala.collection.mutable.HashSet + +import org.scalatest.FunSuite + +class AppendOnlyMapSuite extends FunSuite { + test("initialization") { + val goodMap1 = new AppendOnlyMap[Int, Int](1) + assert(goodMap1.size === 0) + val goodMap2 = new AppendOnlyMap[Int, Int](255) + assert(goodMap2.size === 0) + val goodMap3 = new AppendOnlyMap[Int, Int](256) + assert(goodMap3.size === 0) + intercept[IllegalArgumentException] { + new AppendOnlyMap[Int, Int](1 << 30) // Invalid map size: bigger than 2^29 + } + intercept[IllegalArgumentException] { + new AppendOnlyMap[Int, Int](-1) + } + intercept[IllegalArgumentException] { + new AppendOnlyMap[Int, Int](0) + } + } + + test("object keys and values") { + val map = new AppendOnlyMap[String, String]() + for (i <- 1 to 100) { + map("" + i) = "" + i + } + assert(map.size === 100) + for (i <- 1 to 100) { + assert(map("" + i) === "" + i) + } + assert(map("0") === null) + assert(map("101") === null) + assert(map(null) === null) + val set = new HashSet[(String, String)] + for ((k, v) <- map) { // Test the foreach method + set += ((k, v)) + } + assert(set === (1 to 100).map(_.toString).map(x => (x, x)).toSet) + } + + test("primitive keys and values") { + val map = new AppendOnlyMap[Int, Int]() + for (i <- 1 to 100) { + map(i) = i + } + assert(map.size === 100) + for (i <- 1 to 100) { + assert(map(i) === i) + } + assert(map(0) === null) + assert(map(101) === null) + val set = new HashSet[(Int, Int)] + for ((k, v) <- map) { // Test the foreach method + set += ((k, v)) + } + assert(set === (1 to 100).map(x => (x, x)).toSet) + } + + test("null keys") { + val map = new AppendOnlyMap[String, String]() + for (i <- 1 to 100) { + map("" + i) = "" + i + } + assert(map.size === 100) + assert(map(null) === null) + map(null) = "hello" + assert(map.size === 101) + assert(map(null) === "hello") + } + + test("null values") { + val map = new AppendOnlyMap[String, String]() + for (i <- 1 to 100) { + map("" + i) = null + } + assert(map.size === 100) + assert(map("1") === null) + assert(map(null) === null) + assert(map.size === 100) + map(null) = null + assert(map.size === 101) + assert(map(null) === null) + } + + test("changeValue") { + val map = new AppendOnlyMap[String, String]() + for (i <- 1 to 100) { + map("" + i) = "" + i + } + assert(map.size === 100) + for (i <- 1 to 100) { + val res = map.changeValue("" + i, (hadValue, oldValue) => { + assert(hadValue === true) + assert(oldValue === "" + i) + oldValue + "!" + }) + assert(res === i + "!") + } + // Iterate from 101 to 400 to make sure the map grows a couple of times, because we had a + // bug where changeValue would return the wrong result when the map grew on that insert + for (i <- 101 to 400) { + val res = map.changeValue("" + i, (hadValue, oldValue) => { + assert(hadValue === false) + i + "!" + }) + assert(res === i + "!") + } + assert(map.size === 400) + assert(map(null) === null) + map.changeValue(null, (hadValue, oldValue) => { + assert(hadValue === false) + "null!" + }) + assert(map.size === 401) + map.changeValue(null, (hadValue, oldValue) => { + assert(hadValue === true) + assert(oldValue === "null!") + "null!!" + }) + assert(map.size === 401) + } + + test("inserting in capacity-1 map") { + val map = new AppendOnlyMap[String, String](1) + for (i <- 1 to 100) { + map("" + i) = "" + i + } + assert(map.size === 100) + for (i <- 1 to 100) { + assert(map("" + i) === "" + i) + } + } +} -- cgit v1.2.3 From 0289eb752abfc71ac0cc6796b57f1d21603cfd90 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 26 Dec 2013 14:39:53 -0800 Subject: Allow Product2 rather than just tuple kv pairs --- core/src/main/scala/org/apache/spark/Aggregator.scala | 4 ++-- core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala | 12 ++++++------ 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index 59e5102d3e..f977c03d3a 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -47,7 +47,7 @@ case class Aggregator[K, V, C] ( } else { val combiners = new ExternalAppendOnlyMap[K, V, C](createCombiner, mergeValue, mergeCombiners) - iter.foreach { case(k, v) => combiners.insert(k, v) } + iter.foreach { kv => combiners.insert(kv._1, kv._2) } combiners.iterator } } @@ -68,7 +68,7 @@ case class Aggregator[K, V, C] ( } else { val combiners = new ExternalAppendOnlyMap[K, C, C](Predef.identity, mergeCombiners, mergeCombiners) - iter.foreach { case(k, c) => combiners.insert(k, c) } + iter.foreach { kc => combiners.insert(kc._1, kc._2) } combiners.iterator } } diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 3a549b7b4a..367dc3ea43 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -130,17 +130,17 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: val update: (Boolean, CoGroupCombiner) => CoGroupCombiner = (hadVal, oldVal) => { if (hadVal) oldVal else Array.fill(numRdds)(new ArrayBuffer[Any]) } - rddIterators.foreach { case(it, depNum) => - it.foreach { case(k, v) => - map.changeValue(k, update)(depNum) += v + rddIterators.foreach { case (it, depNum) => + it.foreach { kv => + map.changeValue(kv._1, update)(depNum) += kv._2 } } new InterruptibleIterator(context, map.iterator) } else { val map = createExternalMap(numRdds) - rddIterators.foreach { case(it, depNum) => - it.foreach { case(k, v) => - map.insert(k, new CoGroupValue(v, depNum)) + rddIterators.foreach { case (it, depNum) => + it.foreach { kv => + map.insert(kv._1, new CoGroupValue(kv._2, depNum)) } } new InterruptibleIterator(context, map.iterator) -- cgit v1.2.3 From 1ffe26c7c03b5d128952e7d3ea7f130cd242a468 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 26 Dec 2013 16:19:25 -0800 Subject: Fix streaming JavaAPISuite that depended on order --- .../org/apache/spark/streaming/JavaAPISuite.java | 27 +++++++++++++--------- 1 file changed, 16 insertions(+), 11 deletions(-) diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index daeb99f5b7..bf23469936 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -20,6 +20,7 @@ package org.apache.spark.streaming; import com.google.common.base.Optional; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import com.google.common.io.Files; import kafka.serializer.StringDecoder; @@ -473,13 +474,13 @@ public class JavaAPISuite implements Serializable { new Tuple2("new york", "islanders"))); - List>>> expected = Arrays.asList( - Arrays.asList( - new Tuple2>("california", - new Tuple2("dodgers", "giants")), - new Tuple2>("new york", - new Tuple2("yankees", "mets"))), - Arrays.asList( + List>>> expected = Arrays.asList( + Sets.newHashSet( + new Tuple2>("california", + new Tuple2("dodgers", "giants")), + new Tuple2>("new york", + new Tuple2("yankees", "mets"))), + Sets.newHashSet( new Tuple2>("california", new Tuple2("sharks", "ducks")), new Tuple2>("new york", @@ -514,8 +515,12 @@ public class JavaAPISuite implements Serializable { JavaTestUtils.attachTestOutputStream(joined); List>>> result = JavaTestUtils.runStreams(ssc, 2, 2); + List>>> unorderedResult = Lists.newArrayList(); + for (List>> res: result) { + unorderedResult.add(Sets.newHashSet(res)); + } - Assert.assertEquals(expected, result); + Assert.assertEquals(expected, unorderedResult); } @@ -1230,11 +1235,11 @@ public class JavaAPISuite implements Serializable { List>> expected = Arrays.asList( Arrays.asList( - new Tuple2("hello", 1L), - new Tuple2("world", 1L)), + new Tuple2("world", 1L), + new Tuple2("hello", 1L)), Arrays.asList( - new Tuple2("hello", 2L), new Tuple2("world", 1L), + new Tuple2("hello", 2L), new Tuple2("moon", 1L)), Arrays.asList( new Tuple2("hello", 2L), -- cgit v1.2.3 From a515706d9cb2c94ed981d9015026331aaf582f36 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 26 Dec 2013 17:42:13 -0800 Subject: Fix streaming JavaAPISuite again --- .../org/apache/spark/streaming/JavaAPISuite.java | 20 ++++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index bf23469936..947668369f 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -1233,15 +1233,15 @@ public class JavaAPISuite implements Serializable { Arrays.asList("hello", "moon"), Arrays.asList("hello")); - List>> expected = Arrays.asList( - Arrays.asList( - new Tuple2("world", 1L), - new Tuple2("hello", 1L)), - Arrays.asList( - new Tuple2("world", 1L), + List>> expected = Arrays.asList( + Sets.newHashSet( + new Tuple2("hello", 1L), + new Tuple2("world", 1L)), + Sets.newHashSet( new Tuple2("hello", 2L), + new Tuple2("world", 1L), new Tuple2("moon", 1L)), - Arrays.asList( + Sets.newHashSet( new Tuple2("hello", 2L), new Tuple2("moon", 1L))); @@ -1251,8 +1251,12 @@ public class JavaAPISuite implements Serializable { stream.countByValueAndWindow(new Duration(2000), new Duration(1000)); JavaTestUtils.attachTestOutputStream(counted); List>> result = JavaTestUtils.runStreams(ssc, 3, 3); + List>> unorderedResult = Lists.newArrayList(); + for (List> res: result) { + unorderedResult.add(Sets.newHashSet(res)); + } - Assert.assertEquals(expected, result); + Assert.assertEquals(expected, unorderedResult); } @Test -- cgit v1.2.3 From ec8c5dc644ce97c8cf6e13ba2b216ddbe16e9e0a Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 26 Dec 2013 21:22:38 -0800 Subject: Sort AppendOnlyMap in-place --- .../spark/util/collection/AppendOnlyMap.scala | 33 ++++++++++++++++++++++ .../util/collection/ExternalAppendOnlyMap.scala | 32 ++++++++++++--------- 2 files changed, 51 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala index cb0ca8f8c1..38f3c556ae 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala @@ -17,6 +17,8 @@ package org.apache.spark.util.collection +import java.util + /** * A simple open hash table optimized for the append-only use case, where keys * are never removed, but the value for each key may be changed. @@ -234,4 +236,35 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi val highBit = Integer.highestOneBit(n) if (highBit == n) n else highBit << 1 } + + // Return an iterator of the map in sorted order. + // Note that the validity of the map is no longer preserved. + def destructiveSortedIterator(ord: Ordering[(K, V)]): Iterator[(K, V)] = { + var keyIndex, newIndex = 0 + while (keyIndex < capacity) { + if (data(2 * keyIndex) != null) { + data(newIndex) = (data(2 * keyIndex), data(2 * keyIndex + 1)) + newIndex += 1 + } + keyIndex += 1 + } + // sort + assert(newIndex == curSize) + val rawOrdering = new Ordering[AnyRef] { + def compare(x: AnyRef, y: AnyRef): Int ={ + ord.compare(x.asInstanceOf[(K, V)], y.asInstanceOf[(K, V)]) + } + } + util.Arrays.sort(data, 0, curSize, rawOrdering) + + new Iterator[(K, V)] { + var i = 0 + def hasNext = i < curSize + def next(): (K, V) = { + val item = data(i).asInstanceOf[(K, V)] + i += 1 + item + } + } + } } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 4bda763ffe..ed8b1d36a9 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -88,6 +88,7 @@ class SpillableAppendOnlyMap[K, V, M, C]( val bufferPercent = System.getProperty("spark.shuffle.buffer.percent", "0.8").toFloat bufferSize * bufferPercent } + val KMOrdering: Ordering[(K, M)] = Ordering.by(km => km._1.hashCode()) def insert(key: K, value: V): Unit = { val update: (Boolean, M) => M = (hadVal, oldVal) => { @@ -100,10 +101,14 @@ class SpillableAppendOnlyMap[K, V, M, C]( } def spill(): Unit = { + println("******************* SPILL *********************") val file = File.createTempFile("external_append_only_map", "") val out = new ObjectOutputStream(new FileOutputStream(file)) - val sortedMap = currentMap.iterator.toList.sortBy(kv => kv._1.hashCode()) - sortedMap.foreach(out.writeObject) + val it = currentMap.destructiveSortedIterator(KMOrdering) + while (it.hasNext) { + val kv = it.next() + out.writeObject(kv) + } out.close() currentMap = new SizeTrackingAppendOnlyMap[K, M] oldMaps.append(new DiskIterator(file)) @@ -115,8 +120,8 @@ class SpillableAppendOnlyMap[K, V, M, C]( class ExternalIterator extends Iterator[(K, C)] { // Order by key hash value - val pq = PriorityQueue[KMITuple]()(Ordering.by(_.key.hashCode())) - val inputStreams = Seq(new MemoryIterator(currentMap)) ++ oldMaps + val pq = new PriorityQueue[KMITuple] + val inputStreams = Seq(currentMap.destructiveSortedIterator(KMOrdering)) ++ oldMaps inputStreams.foreach(readFromIterator) // Read from the given iterator until a key of different hash is retrieved @@ -127,7 +132,10 @@ class SpillableAppendOnlyMap[K, V, M, C]( pq.enqueue(KMITuple(k, m, it)) minHash match { case None => minHash = Some(k.hashCode()) - case Some(expectedHash) if k.hashCode() != expectedHash => return + case Some(expectedHash) => + if (k.hashCode() != expectedHash) { + return + } } } } @@ -156,15 +164,11 @@ class SpillableAppendOnlyMap[K, V, M, C]( (minKey, createCombiner(minGroup)) } - case class KMITuple(key: K, group: M, iterator: Iterator[(K, M)]) - } - - // Iterate through (K, M) pairs in sorted order from the in-memory map - class MemoryIterator(map: AppendOnlyMap[K, M]) extends Iterator[(K, M)] { - val sortedMap = currentMap.iterator.toList.sortBy(km => km._1.hashCode()) - val it = sortedMap.iterator - override def hasNext: Boolean = it.hasNext - override def next(): (K, M) = it.next() + case class KMITuple(key: K, group: M, iterator: Iterator[(K, M)]) extends Ordered[KMITuple] { + def compare(other: KMITuple): Int = { + -key.hashCode().compareTo(other.key.hashCode()) + } + } } // Iterate through (K, M) pairs in sorted order from an on-disk map -- cgit v1.2.3 From 0f66b7f2fc25704fc299917a138a530c1c5f13c2 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 26 Dec 2013 21:23:18 -0800 Subject: Return efficient iterator if no spillage happened --- core/src/main/scala/org/apache/spark/Aggregator.scala | 6 ++++-- .../main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 9 +++++---- .../spark/util/collection/ExternalAppendOnlyMap.scala | 14 +++++++++++--- 3 files changed, 20 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index f977c03d3a..aedb832eb5 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -17,7 +17,9 @@ package org.apache.spark -import org.apache.spark.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap} +import scala.reflect.ClassTag + +import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap} /** * A set of functions used to aggregate data. @@ -26,7 +28,7 @@ import org.apache.spark.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap} * @param mergeValue function to merge a new value into the aggregation result. * @param mergeCombiners function to merge outputs from multiple mergeValue function. */ -case class Aggregator[K, V, C] ( +case class Aggregator[K, V, C: ClassTag] ( createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) { diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 9512b418d7..0316d89398 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -69,7 +69,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * 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: ClassTag](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, partitioner: Partitioner, @@ -107,7 +107,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) /** * Simplified version of combineByKey that hash-partitions the output RDD. */ - def combineByKey[C](createCombiner: V => C, + def combineByKey[C: ClassTag](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, numPartitions: Int): RDD[(K, C)] = { @@ -296,8 +296,9 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * 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) - : RDD[(K, C)] = { + def combineByKey[C: ClassTag]( + createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) + : RDD[(K, C)] = { combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(self)) } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index ed8b1d36a9..991dd1845d 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -18,7 +18,9 @@ package org.apache.spark.util.collection import java.io._ + import scala.collection.mutable.{ArrayBuffer, PriorityQueue} +import scala.reflect.ClassTag /** * A wrapper for SpillableAppendOnlyMap that handles two cases: @@ -29,7 +31,7 @@ import scala.collection.mutable.{ArrayBuffer, PriorityQueue} * (2) Otherwise, group values of the same key together before disk spill, and merge them * into combiners only after reading them back from disk. */ -class ExternalAppendOnlyMap[K, V, C]( +class ExternalAppendOnlyMap[K, V, C: ClassTag]( createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) @@ -74,7 +76,7 @@ class ExternalAppendOnlyMap[K, V, C]( * An append-only map that spills sorted content to disk when the memory threshold is exceeded. * A group is an intermediate combiner, with type M equal to either C or ArrayBuffer[V]. */ -class SpillableAppendOnlyMap[K, V, M, C]( +class SpillableAppendOnlyMap[K, V, M: ClassTag, C: ClassTag]( createGroup: V => M, mergeValue: (M, V) => M, mergeGroups: (M, M) => M, @@ -114,7 +116,13 @@ class SpillableAppendOnlyMap[K, V, M, C]( oldMaps.append(new DiskIterator(file)) } - override def iterator: Iterator[(K, C)] = new ExternalIterator() + override def iterator: Iterator[(K, C)] = { + if (oldMaps.isEmpty && implicitly[ClassTag[M]] == implicitly[ClassTag[C]]) { + currentMap.iterator.asInstanceOf[Iterator[(K, C)]] + } else { + new ExternalIterator() + } + } // An iterator that sort-merges (K, M) pairs from memory and disk into (K, C) pairs class ExternalIterator extends Iterator[(K, C)] { -- cgit v1.2.3 From 1dc0440c1ac8882c13de99169e5535b005a801e4 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 26 Dec 2013 23:20:14 -0800 Subject: Use real serializer & manual ordering --- .../util/collection/ExternalAppendOnlyMap.scala | 38 +++++++++++++++------- 1 file changed, 27 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 991dd1845d..b8045297d1 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -22,6 +22,10 @@ import java.io._ import scala.collection.mutable.{ArrayBuffer, PriorityQueue} import scala.reflect.ClassTag +import org.apache.spark.SparkEnv +import org.apache.spark.serializer.Serializer +import org.apache.spark.util.collection.SpillableAppendOnlyMap.KeyHashOrdering + /** * A wrapper for SpillableAppendOnlyMap that handles two cases: * @@ -31,18 +35,20 @@ import scala.reflect.ClassTag * (2) Otherwise, group values of the same key together before disk spill, and merge them * into combiners only after reading them back from disk. */ -class ExternalAppendOnlyMap[K, V, C: ClassTag]( +private[spark] class ExternalAppendOnlyMap[K, V, C: ClassTag]( createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) extends Iterable[(K, C)] with Serializable { + private val serializer = SparkEnv.get.serializerManager.default + private val mergeBeforeSpill: Boolean = mergeCombiners != null private val map: SpillableAppendOnlyMap[K, V, _, C] = { if (mergeBeforeSpill) { new SpillableAppendOnlyMap[K, V, C, C] (createCombiner, mergeValue, - mergeCombiners, Predef.identity) + mergeCombiners, Predef.identity, serializer) } else { // Use ArrayBuffer[V] as the intermediate combiner val createGroup: (V => ArrayBuffer[V]) = value => ArrayBuffer[V](value) @@ -63,7 +69,7 @@ class ExternalAppendOnlyMap[K, V, C: ClassTag]( combiner.getOrElse(null.asInstanceOf[C]) } new SpillableAppendOnlyMap[K, V, ArrayBuffer[V], C](createGroup, mergeValueIntoGroup, - mergeGroups, combineGroup) + mergeGroups, combineGroup, serializer) } } @@ -76,13 +82,16 @@ class ExternalAppendOnlyMap[K, V, C: ClassTag]( * An append-only map that spills sorted content to disk when the memory threshold is exceeded. * A group is an intermediate combiner, with type M equal to either C or ArrayBuffer[V]. */ -class SpillableAppendOnlyMap[K, V, M: ClassTag, C: ClassTag]( +private[spark] class SpillableAppendOnlyMap[K, V, M: ClassTag, C: ClassTag]( createGroup: V => M, mergeValue: (M, V) => M, mergeGroups: (M, M) => M, - createCombiner: M => C) + createCombiner: M => C, + serializer: Serializer) extends Iterable[(K, C)] with Serializable { + val ser = serializer.newInstance() + var currentMap = new SizeTrackingAppendOnlyMap[K, M] val oldMaps = new ArrayBuffer[DiskIterator] val memoryThreshold = { @@ -90,7 +99,7 @@ class SpillableAppendOnlyMap[K, V, M: ClassTag, C: ClassTag]( val bufferPercent = System.getProperty("spark.shuffle.buffer.percent", "0.8").toFloat bufferSize * bufferPercent } - val KMOrdering: Ordering[(K, M)] = Ordering.by(km => km._1.hashCode()) + val ordering = new KeyHashOrdering[K, M]() def insert(key: K, value: V): Unit = { val update: (Boolean, M) => M = (hadVal, oldVal) => { @@ -103,10 +112,9 @@ class SpillableAppendOnlyMap[K, V, M: ClassTag, C: ClassTag]( } def spill(): Unit = { - println("******************* SPILL *********************") val file = File.createTempFile("external_append_only_map", "") - val out = new ObjectOutputStream(new FileOutputStream(file)) - val it = currentMap.destructiveSortedIterator(KMOrdering) + val out = ser.serializeStream(new FileOutputStream(file)) + val it = currentMap.destructiveSortedIterator(ordering) while (it.hasNext) { val kv = it.next() out.writeObject(kv) @@ -129,7 +137,7 @@ class SpillableAppendOnlyMap[K, V, M: ClassTag, C: ClassTag]( // Order by key hash value val pq = new PriorityQueue[KMITuple] - val inputStreams = Seq(currentMap.destructiveSortedIterator(KMOrdering)) ++ oldMaps + val inputStreams = Seq(currentMap.destructiveSortedIterator(ordering)) ++ oldMaps inputStreams.foreach(readFromIterator) // Read from the given iterator until a key of different hash is retrieved @@ -181,7 +189,7 @@ class SpillableAppendOnlyMap[K, V, M: ClassTag, C: ClassTag]( // Iterate through (K, M) pairs in sorted order from an on-disk map class DiskIterator(file: File) extends Iterator[(K, M)] { - val in = new ObjectInputStream(new FileInputStream(file)) + val in = ser.deserializeStream(new FileInputStream(file)) var nextItem: Option[(K, M)] = None override def hasNext: Boolean = { @@ -201,3 +209,11 @@ class SpillableAppendOnlyMap[K, V, M: ClassTag, C: ClassTag]( } } } + +private[spark] object SpillableAppendOnlyMap { + private class KeyHashOrdering[K, M] extends Ordering[(K, M)] { + def compare(x: (K, M), y: (K, M)): Int = { + x._1.hashCode().compareTo(y._1.hashCode()) + } + } +} -- cgit v1.2.3 From 8f3175773c86c2e789cf5717486c295fccaa9106 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 26 Dec 2013 23:38:10 -0800 Subject: Final cleanup --- .../main/scala/org/apache/spark/Aggregator.scala | 10 ++++++-- .../scala/org/apache/spark/rdd/CoGroupedRDD.scala | 12 ++++++---- .../util/collection/ExternalAppendOnlyMap.scala | 27 +++++++++------------- .../collection/SizeTrackingAppendOnlyMap.scala | 4 ++-- 4 files changed, 28 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index aedb832eb5..470694ed35 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -49,7 +49,10 @@ case class Aggregator[K, V, C: ClassTag] ( } else { val combiners = new ExternalAppendOnlyMap[K, V, C](createCombiner, mergeValue, mergeCombiners) - iter.foreach { kv => combiners.insert(kv._1, kv._2) } + while (iter.hasNext) { + val kv = iter.next() + combiners.insert(kv._1, kv._2) + } combiners.iterator } } @@ -70,7 +73,10 @@ case class Aggregator[K, V, C: ClassTag] ( } else { val combiners = new ExternalAppendOnlyMap[K, C, C](Predef.identity, mergeCombiners, mergeCombiners) - iter.foreach { kc => combiners.insert(kc._1, kc._2) } + while (iter.hasNext) { + var kc = iter.next() + combiners.insert(kc._1, kc._2) + } combiners.iterator } } diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 367dc3ea43..dd02c8a3fe 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -60,9 +60,9 @@ class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep]) class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: Partitioner) extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) { - type CoGroup = ArrayBuffer[Any] - type CoGroupValue = (Any, Int) // Int is dependency number - type CoGroupCombiner = Seq[CoGroup] + private type CoGroup = ArrayBuffer[Any] + private type CoGroupValue = (Any, Int) // Int is dependency number + private type CoGroupCombiner = Seq[CoGroup] private var serializerClass: String = null @@ -131,7 +131,8 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: if (hadVal) oldVal else Array.fill(numRdds)(new ArrayBuffer[Any]) } rddIterators.foreach { case (it, depNum) => - it.foreach { kv => + while (it.hasNext) { + val kv = it.next() map.changeValue(kv._1, update)(depNum) += kv._2 } } @@ -139,7 +140,8 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: } else { val map = createExternalMap(numRdds) rddIterators.foreach { case (it, depNum) => - it.foreach { kv => + while (it.hasNext) { + val kv = it.next() map.insert(kv._1, new CoGroupValue(kv._2, depNum)) } } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index b8045297d1..4b0a87c116 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -24,7 +24,6 @@ import scala.reflect.ClassTag import org.apache.spark.SparkEnv import org.apache.spark.serializer.Serializer -import org.apache.spark.util.collection.SpillableAppendOnlyMap.KeyHashOrdering /** * A wrapper for SpillableAppendOnlyMap that handles two cases: @@ -42,13 +41,12 @@ private[spark] class ExternalAppendOnlyMap[K, V, C: ClassTag]( extends Iterable[(K, C)] with Serializable { private val serializer = SparkEnv.get.serializerManager.default - private val mergeBeforeSpill: Boolean = mergeCombiners != null private val map: SpillableAppendOnlyMap[K, V, _, C] = { if (mergeBeforeSpill) { - new SpillableAppendOnlyMap[K, V, C, C] (createCombiner, mergeValue, - mergeCombiners, Predef.identity, serializer) + new SpillableAppendOnlyMap[K, V, C, C] (createCombiner, mergeValue, mergeCombiners, + Predef.identity, serializer) } else { // Use ArrayBuffer[V] as the intermediate combiner val createGroup: (V => ArrayBuffer[V]) = value => ArrayBuffer[V](value) @@ -90,16 +88,15 @@ private[spark] class SpillableAppendOnlyMap[K, V, M: ClassTag, C: ClassTag]( serializer: Serializer) extends Iterable[(K, C)] with Serializable { - val ser = serializer.newInstance() - - var currentMap = new SizeTrackingAppendOnlyMap[K, M] - val oldMaps = new ArrayBuffer[DiskIterator] - val memoryThreshold = { + private var currentMap = new SizeTrackingAppendOnlyMap[K, M] + private val oldMaps = new ArrayBuffer[DiskIterator] + private val memoryThreshold = { val bufferSize = System.getProperty("spark.shuffle.buffer", "1024").toLong * 1024 * 1024 val bufferPercent = System.getProperty("spark.shuffle.buffer.percent", "0.8").toFloat bufferSize * bufferPercent } - val ordering = new KeyHashOrdering[K, M]() + private val ordering = new SpillableAppendOnlyMap.KeyHashOrdering[K, M]() + private val ser = serializer.newInstance() def insert(key: K, value: V): Unit = { val update: (Boolean, M) => M = (hadVal, oldVal) => { @@ -111,7 +108,7 @@ private[spark] class SpillableAppendOnlyMap[K, V, M: ClassTag, C: ClassTag]( } } - def spill(): Unit = { + private def spill(): Unit = { val file = File.createTempFile("external_append_only_map", "") val out = ser.serializeStream(new FileOutputStream(file)) val it = currentMap.destructiveSortedIterator(ordering) @@ -133,9 +130,7 @@ private[spark] class SpillableAppendOnlyMap[K, V, M: ClassTag, C: ClassTag]( } // An iterator that sort-merges (K, M) pairs from memory and disk into (K, C) pairs - class ExternalIterator extends Iterator[(K, C)] { - - // Order by key hash value + private class ExternalIterator extends Iterator[(K, C)] { val pq = new PriorityQueue[KMITuple] val inputStreams = Seq(currentMap.destructiveSortedIterator(ordering)) ++ oldMaps inputStreams.foreach(readFromIterator) @@ -182,13 +177,13 @@ private[spark] class SpillableAppendOnlyMap[K, V, M: ClassTag, C: ClassTag]( case class KMITuple(key: K, group: M, iterator: Iterator[(K, M)]) extends Ordered[KMITuple] { def compare(other: KMITuple): Int = { - -key.hashCode().compareTo(other.key.hashCode()) + other.key.hashCode().compareTo(key.hashCode()) } } } // Iterate through (K, M) pairs in sorted order from an on-disk map - class DiskIterator(file: File) extends Iterator[(K, M)] { + private class DiskIterator(file: File) extends Iterator[(K, M)] { val in = ser.deserializeStream(new FileInputStream(file)) var nextItem: Option[(K, M)] = None diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala index 738908a660..4b11749b14 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala @@ -3,9 +3,9 @@ package org.apache.spark.util.collection import org.apache.spark.util.SamplingSizeTracker /** Append-only map that keeps track of its estimated size in bytes. */ -class SizeTrackingAppendOnlyMap[K, V] extends AppendOnlyMap[K, V] { +private[spark] class SizeTrackingAppendOnlyMap[K, V] extends AppendOnlyMap[K, V] { - val sizeTracker = new SamplingSizeTracker(this) + private val sizeTracker = new SamplingSizeTracker(this) def estimateSize() = sizeTracker.estimateSize() -- cgit v1.2.3 From d0cfbc41e2dc48fa04a79cdff107991cd096d0cc Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 27 Dec 2013 00:05:27 -0800 Subject: Rename spark.shuffle.buffer variables --- .../org/apache/spark/util/collection/ExternalAppendOnlyMap.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 4b0a87c116..f5a2e8db59 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -91,8 +91,8 @@ private[spark] class SpillableAppendOnlyMap[K, V, M: ClassTag, C: ClassTag]( private var currentMap = new SizeTrackingAppendOnlyMap[K, M] private val oldMaps = new ArrayBuffer[DiskIterator] private val memoryThreshold = { - val bufferSize = System.getProperty("spark.shuffle.buffer", "1024").toLong * 1024 * 1024 - val bufferPercent = System.getProperty("spark.shuffle.buffer.percent", "0.8").toFloat + val bufferSize = System.getProperty("spark.shuffle.buffer.mb", "1024").toLong * 1024 * 1024 + val bufferPercent = System.getProperty("spark.shuffle.buffer.fraction", "0.8").toFloat bufferSize * bufferPercent } private val ordering = new SpillableAppendOnlyMap.KeyHashOrdering[K, M]() -- cgit v1.2.3 From 2a7b3511f45dcaceaa099e99b7a561b4a266d647 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Fri, 27 Dec 2013 10:55:16 -0800 Subject: Add Apache headers --- .../apache/spark/util/SamplingSizeTracker.scala | 24 ++++++++++++++++++---- .../collection/SizeTrackingAppendOnlyMap.scala | 17 +++++++++++++++ .../spark/util/SamplingSizeTrackerSuite.scala | 17 +++++++++++++++ 3 files changed, 54 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/SamplingSizeTracker.scala b/core/src/main/scala/org/apache/spark/util/SamplingSizeTracker.scala index 2262b7d1be..3eb80661cb 100644 --- a/core/src/main/scala/org/apache/spark/util/SamplingSizeTracker.scala +++ b/core/src/main/scala/org/apache/spark/util/SamplingSizeTracker.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.spark.util import org.apache.spark.util.SamplingSizeTracker.Sample @@ -11,7 +28,7 @@ import org.apache.spark.util.SamplingSizeTracker.Sample * flushSamples() if there is a non-linear change in object size (otherwise linear is assumed). * Not threadsafe. */ -class SamplingSizeTracker(obj: AnyRef) { +private[spark] class SamplingSizeTracker(obj: AnyRef) { /** * Controls the base of the exponential which governs the rate of sampling. * E.g., a value of 2 would mean we sample at 1, 2, 4, 8, ... elements. @@ -56,9 +73,8 @@ class SamplingSizeTracker(obj: AnyRef) { } else { 0 } - val extrapolatedDelta = interpolatedDelta * (numUpdates - lastSample.numUpdates) - val estimate = lastSample.size + extrapolatedDelta - math.max(0, estimate).toLong + val extrapolatedDelta = math.max(0, interpolatedDelta * (numUpdates - lastSample.numUpdates)) + (lastSample.size + extrapolatedDelta).toLong } } diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala index 4b11749b14..ea0f2fd68f 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.spark.util.collection import org.apache.spark.util.SamplingSizeTracker diff --git a/core/src/test/scala/org/apache/spark/util/SamplingSizeTrackerSuite.scala b/core/src/test/scala/org/apache/spark/util/SamplingSizeTrackerSuite.scala index 6b772131a7..47e4723cf3 100644 --- a/core/src/test/scala/org/apache/spark/util/SamplingSizeTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/SamplingSizeTrackerSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.spark.util import scala.util.Random -- cgit v1.2.3 From 8fbff9f5d04064b870e372db0e3885e3fbf28222 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sun, 29 Dec 2013 16:22:44 -0800 Subject: Address Aaron's comments --- .../main/scala/org/apache/spark/Aggregator.scala | 2 +- .../scala/org/apache/spark/rdd/CoGroupedRDD.scala | 31 +-- .../spark/util/collection/AppendOnlyMap.scala | 14 +- .../util/collection/ExternalAppendOnlyMap.scala | 221 +++++++++++++++------ .../org/apache/spark/streaming/JavaAPISuite.java | 8 +- 5 files changed, 188 insertions(+), 88 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index 470694ed35..784c09ec51 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -74,7 +74,7 @@ case class Aggregator[K, V, C: ClassTag] ( val combiners = new ExternalAppendOnlyMap[K, C, C](Predef.identity, mergeCombiners, mergeCombiners) while (iter.hasNext) { - var kc = iter.next() + val kc = iter.next() combiners.insert(kc._1, kc._2) } combiners.iterator diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index dd02c8a3fe..77a594a3e4 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -43,8 +43,7 @@ private[spark] case class NarrowCoGroupSplitDep( private[spark] case class ShuffleCoGroupSplitDep(shuffleId: Int) extends CoGroupSplitDep -private[spark] -class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep]) +private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep]) extends Partition with Serializable { override val index: Int = idx override def hashCode(): Int = idx @@ -60,6 +59,9 @@ class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep]) class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: Partitioner) extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) { + // For example, `(k, a) cogroup (k, b)` produces k -> Seq(ArrayBuffer as, ArrayBuffer bs). + // Each ArrayBuffer is represented as a CoGroup, and the resulting Seq as a CoGroupCombiner. + // CoGroupValue is the intermediate state of each value before being merged in compute. private type CoGroup = ArrayBuffer[Any] private type CoGroupValue = (Any, Int) // Int is dependency number private type CoGroupCombiner = Seq[CoGroup] @@ -103,7 +105,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: override val partitioner = Some(part) override def compute(s: Partition, context: TaskContext): Iterator[(K, CoGroupCombiner)] = { - // e.g. for `(k, a) cogroup (k, b)`, K -> Seq(ArrayBuffer as, ArrayBuffer bs) + val externalSorting = System.getProperty("spark.shuffle.externalSorting", "false").toBoolean val split = s.asInstanceOf[CoGroupPartition] val numRdds = split.deps.size @@ -113,27 +115,30 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: for ((dep, depNum) <- split.deps.zipWithIndex) dep match { case NarrowCoGroupSplitDep(rdd, _, itsSplit) => { // Read them from the parent - val v = (rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, Any]]], depNum) - rddIterators += v + val it = rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, Any]]] + rddIterators += ((it, depNum)) } case ShuffleCoGroupSplitDep(shuffleId) => { // Read map outputs of shuffle val fetcher = SparkEnv.get.shuffleFetcher val ser = SparkEnv.get.serializerManager.get(serializerClass) - val v = (fetcher.fetch[Product2[K, Any]](shuffleId, split.index, context, ser), depNum) - rddIterators += v + val it = fetcher.fetch[Product2[K, Any]](shuffleId, split.index, context, ser) + rddIterators += ((it, depNum)) } } if (!externalSorting) { val map = new AppendOnlyMap[K, CoGroupCombiner] val update: (Boolean, CoGroupCombiner) => CoGroupCombiner = (hadVal, oldVal) => { - if (hadVal) oldVal else Array.fill(numRdds)(new ArrayBuffer[Any]) + if (hadVal) oldVal else Array.fill(numRdds)(new CoGroup) + } + val getCombiner: K => CoGroupCombiner = key => { + map.changeValue(key, update) } rddIterators.foreach { case (it, depNum) => while (it.hasNext) { val kv = it.next() - map.changeValue(kv._1, update)(depNum) += kv._2 + getCombiner(kv._1)(depNum) += kv._2 } } new InterruptibleIterator(context, map.iterator) @@ -149,17 +154,17 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: } } - private def createExternalMap(numRdds: Int): - ExternalAppendOnlyMap[K, CoGroupValue, CoGroupCombiner] = { + private def createExternalMap(numRdds: Int) + : ExternalAppendOnlyMap[K, CoGroupValue, CoGroupCombiner] = { val createCombiner: (CoGroupValue => CoGroupCombiner) = value => { val newCombiner = Array.fill(numRdds)(new CoGroup) - value match { case(v, depNum) => newCombiner(depNum) += v } + value match { case (v, depNum) => newCombiner(depNum) += v } newCombiner } val mergeValue: (CoGroupCombiner, CoGroupValue) => CoGroupCombiner = (combiner, value) => { - value match { case(v, depNum) => combiner(depNum) += v } + value match { case (v, depNum) => combiner(depNum) += v } combiner } val mergeCombiners: (CoGroupCombiner, CoGroupCombiner) => CoGroupCombiner = diff --git a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala index 38f3c556ae..7810119847 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala @@ -237,10 +237,12 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi if (highBit == n) n else highBit << 1 } - // Return an iterator of the map in sorted order. - // Note that the validity of the map is no longer preserved. - def destructiveSortedIterator(ord: Ordering[(K, V)]): Iterator[(K, V)] = { + /** Return an iterator of the map in sorted order. This provides a way to sort the map without + * using additional memory, at the expense of destroying the validity of the map. + */ + def destructiveSortedIterator(ordering: Ordering[(K, V)]): Iterator[(K, V)] = { var keyIndex, newIndex = 0 + // Pack KV pairs into the front of the underlying array while (keyIndex < capacity) { if (data(2 * keyIndex) != null) { data(newIndex) = (data(2 * keyIndex), data(2 * keyIndex + 1)) @@ -248,11 +250,11 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi } keyIndex += 1 } - // sort assert(newIndex == curSize) + // Sort by the given ordering val rawOrdering = new Ordering[AnyRef] { - def compare(x: AnyRef, y: AnyRef): Int ={ - ord.compare(x.asInstanceOf[(K, V)], y.asInstanceOf[(K, V)]) + def compare(x: AnyRef, y: AnyRef): Int = { + ordering.compare(x.asInstanceOf[(K, V)], y.asInstanceOf[(K, V)]) } } util.Arrays.sort(data, 0, curSize, rawOrdering) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index f5a2e8db59..1de545c05b 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -33,6 +33,10 @@ import org.apache.spark.serializer.Serializer * * (2) Otherwise, group values of the same key together before disk spill, and merge them * into combiners only after reading them back from disk. + * + * In the latter case, values occupy much more space because they are not collapsed as soon + * as they are inserted. This in turn leads to more disk spills, degrading performance. + * For this reason, a mergeCombiners function should be specified if possible. */ private[spark] class ExternalAppendOnlyMap[K, V, C: ClassTag]( createCombiner: V => C, @@ -78,28 +82,42 @@ private[spark] class ExternalAppendOnlyMap[K, V, C: ClassTag]( /** * An append-only map that spills sorted content to disk when the memory threshold is exceeded. - * A group is an intermediate combiner, with type M equal to either C or ArrayBuffer[V]. + * A group is an intermediate combiner, with type G equal to either C or ArrayBuffer[V]. + * + * This map takes two passes over the data: + * (1) Values are merged into groups, which are spilled to disk as necessary. + * (2) Groups are read from disk and merged into combiners, which are returned. + * + * If we never spill to disk, we avoid the second pass provided that groups G are already + * combiners C. + * + * Note that OOM is still possible with the SpillableAppendOnlyMap. This may occur if the + * collective G values do not fit into memory, or if the size estimation is not sufficiently + * accurate. To account for the latter, `spark.shuffle.buffer.fraction` specifies an additional + * margin of safety, while `spark.shuffle.buffer.mb` specifies the raw memory threshold. */ -private[spark] class SpillableAppendOnlyMap[K, V, M: ClassTag, C: ClassTag]( - createGroup: V => M, - mergeValue: (M, V) => M, - mergeGroups: (M, M) => M, - createCombiner: M => C, +private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( + createGroup: V => G, + mergeValue: (G, V) => G, + mergeGroups: (G, G) => G, + createCombiner: G => C, serializer: Serializer) extends Iterable[(K, C)] with Serializable { - private var currentMap = new SizeTrackingAppendOnlyMap[K, M] - private val oldMaps = new ArrayBuffer[DiskIterator] + import SpillableAppendOnlyMap._ + + private var currentMap = new SizeTrackingAppendOnlyMap[K, G] + private val oldMaps = new ArrayBuffer[DiskKGIterator] private val memoryThreshold = { val bufferSize = System.getProperty("spark.shuffle.buffer.mb", "1024").toLong * 1024 * 1024 val bufferPercent = System.getProperty("spark.shuffle.buffer.fraction", "0.8").toFloat bufferSize * bufferPercent } - private val ordering = new SpillableAppendOnlyMap.KeyHashOrdering[K, M]() + private val ordering = new KeyGroupOrdering[K, G] private val ser = serializer.newInstance() def insert(key: K, value: V): Unit = { - val update: (Boolean, M) => M = (hadVal, oldVal) => { + val update: (Boolean, G) => G = (hadVal, oldVal) => { if (hadVal) mergeValue(oldVal, value) else createGroup(value) } currentMap.changeValue(key, update) @@ -117,98 +135,173 @@ private[spark] class SpillableAppendOnlyMap[K, V, M: ClassTag, C: ClassTag]( out.writeObject(kv) } out.close() - currentMap = new SizeTrackingAppendOnlyMap[K, M] - oldMaps.append(new DiskIterator(file)) + currentMap = new SizeTrackingAppendOnlyMap[K, G] + oldMaps.append(new DiskKGIterator(file)) } override def iterator: Iterator[(K, C)] = { - if (oldMaps.isEmpty && implicitly[ClassTag[M]] == implicitly[ClassTag[C]]) { + if (oldMaps.isEmpty && implicitly[ClassTag[G]] == implicitly[ClassTag[C]]) { currentMap.iterator.asInstanceOf[Iterator[(K, C)]] } else { new ExternalIterator() } } - // An iterator that sort-merges (K, M) pairs from memory and disk into (K, C) pairs + // An iterator that sort-merges (K, G) pairs from memory and disk into (K, C) pairs private class ExternalIterator extends Iterator[(K, C)] { - val pq = new PriorityQueue[KMITuple] - val inputStreams = Seq(currentMap.destructiveSortedIterator(ordering)) ++ oldMaps - inputStreams.foreach(readFromIterator) - - // Read from the given iterator until a key of different hash is retrieved - def readFromIterator(it: Iterator[(K, M)]): Unit = { - var minHash : Option[Int] = None - while (it.hasNext) { - val (k, m) = it.next() - pq.enqueue(KMITuple(k, m, it)) - minHash match { - case None => minHash = Some(k.hashCode()) - case Some(expectedHash) => - if (k.hashCode() != expectedHash) { - return - } + val mergeHeap = new PriorityQueue[KGITuple] + val inputStreams = oldMaps ++ Seq(currentMap.destructiveSortedIterator(ordering)) + + // Invariant: size of mergeHeap == number of input streams + inputStreams.foreach{ it => + val kgPairs = readFromIterator(it) + mergeHeap.enqueue(KGITuple(it, kgPairs)) + } + + // Read from the given iterator until a key of different hash is retrieved. + // The resulting ArrayBuffer includes this key, and is ordered by key hash. + def readFromIterator(it: Iterator[(K, G)]): ArrayBuffer[(K, G)] = { + val kgPairs = new ArrayBuffer[(K, G)] + if (it.hasNext) { + var kg = it.next() + kgPairs += kg + val minHash = kg._1.hashCode() + while (it.hasNext && kg._1.hashCode() == minHash) { + kg = it.next() + kgPairs += kg + } + } + kgPairs + } + + // Drop and return all (K, G) pairs with K = the given key from the given KGITuple + def dropKey(kgi: KGITuple, key: K): ArrayBuffer[(K, G)] = { + val dropped = new ArrayBuffer[(K, G)] + var i = 0 + while (i < kgi.pairs.length) { + if (kgi.pairs(i)._1 == key) { + dropped += kgi.pairs.remove(i) + } else { + i += 1 } } + dropped } - override def hasNext: Boolean = !pq.isEmpty + // Merge all (K, G) pairs with K = the given key into baseGroup + def mergeIntoGroup(key: K, baseGroup: G, kgPairs: ArrayBuffer[(K, G)]): G = { + var mergedGroup = baseGroup + kgPairs.foreach { case (k, g) => + if (k == key){ + mergedGroup = mergeGroups(mergedGroup, g) + } + } + mergedGroup + } + + override def hasNext: Boolean = { + mergeHeap.foreach{ kgi => + if (!kgi.pairs.isEmpty) { + return true + } + } + false + } override def next(): (K, C) = { - val minKMI = pq.dequeue() - var (minKey, minGroup) = (minKMI.key, minKMI.group) - val minHash = minKey.hashCode() - readFromIterator(minKMI.iterator) - - // Merge groups with the same key into minGroup - var collidedKMI = ArrayBuffer[KMITuple]() - while (!pq.isEmpty && pq.head.key.hashCode() == minHash) { - val newKMI = pq.dequeue() - if (newKMI.key == minKey) { - minGroup = mergeGroups(minGroup, newKMI.group) - readFromIterator(newKMI.iterator) - } else { - // Collision - collidedKMI += newKMI + var minKGI = mergeHeap.dequeue() + val (minPairs, minHash) = (minKGI.pairs, minKGI.minHash) + if (minPairs.length == 0) { + // Should only happen when hasNext is false + throw new NoSuchElementException + } + var (minKey, minGroup) = minPairs(0) + assert(minKey.hashCode() == minHash) + + // Merge the rest of minPairs into minGroup + val minPairsWithKey = dropKey(minKGI, minKey).tail + minGroup = mergeIntoGroup(minKey, minGroup, minPairsWithKey) + if (minPairs.length == 0) { + minPairs ++= readFromIterator(minKGI.iterator) + } + + // Do the same for all other KGITuples with the same minHash + val tuplesToAddBack = ArrayBuffer[KGITuple](minKGI) + while (!mergeHeap.isEmpty && mergeHeap.head.minHash == minHash) { + val newKGI = mergeHeap.dequeue() + val pairsWithKey = dropKey(newKGI, minKey) + minGroup = mergeIntoGroup(minKey, minGroup, pairsWithKey) + if (newKGI.pairs.length == 0) { + newKGI.pairs ++= readFromIterator(newKGI.iterator) } + tuplesToAddBack += newKGI } - collidedKMI.foreach(pq.enqueue(_)) + tuplesToAddBack.foreach(mergeHeap.enqueue(_)) (minKey, createCombiner(minGroup)) } - case class KMITuple(key: K, group: M, iterator: Iterator[(K, M)]) extends Ordered[KMITuple] { - def compare(other: KMITuple): Int = { - other.key.hashCode().compareTo(key.hashCode()) + case class KGITuple(iterator: Iterator[(K, G)], pairs: ArrayBuffer[(K, G)]) + extends Ordered[KGITuple] { + + // Invariant: pairs are ordered by key hash + def minHash: Int = { + if (pairs.length > 0){ + pairs(0)._1.hashCode() + } else { + Int.MaxValue + } + } + + def compare(other: KGITuple): Int = { + // mutable.PriorityQueue dequeues the max, not the min + -minHash.compareTo(other.minHash) } } } - // Iterate through (K, M) pairs in sorted order from an on-disk map - private class DiskIterator(file: File) extends Iterator[(K, M)] { + // Iterate through (K, G) pairs in sorted order from an on-disk map + private class DiskKGIterator(file: File) extends Iterator[(K, G)] { val in = ser.deserializeStream(new FileInputStream(file)) - var nextItem: Option[(K, M)] = None + var nextItem: Option[(K, G)] = None + var eof = false + + def readNextItem(): Option[(K, G)] = { + if (!eof) { + try { + return Some(in.readObject().asInstanceOf[(K, G)]) + } catch { + case e: EOFException => eof = true + } + } + None + } override def hasNext: Boolean = { - nextItem = try { - Some(in.readObject().asInstanceOf[(K, M)]) - } catch { - case e: EOFException => None + nextItem match { + case Some(item) => true + case None => + nextItem = readNextItem() + nextItem.isDefined } - nextItem.isDefined } - override def next(): (K, M) = { + override def next(): (K, G) = { nextItem match { - case Some(item) => item - case None => throw new NoSuchElementException + case Some(item) => + nextItem = None + item + case None => + val item = readNextItem() + item.getOrElse(throw new NoSuchElementException) } } } } private[spark] object SpillableAppendOnlyMap { - private class KeyHashOrdering[K, M] extends Ordering[(K, M)] { - def compare(x: (K, M), y: (K, M)): Int = { - x._1.hashCode().compareTo(y._1.hashCode()) + private class KeyGroupOrdering[K, G] extends Ordering[(K, G)] { + def compare(kg1: (K, G), kg2: (K, G)): Int = { + kg1._1.hashCode().compareTo(kg2._1.hashCode()) } } } diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index 947668369f..a0a8129948 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -476,10 +476,10 @@ public class JavaAPISuite implements Serializable { List>>> expected = Arrays.asList( Sets.newHashSet( - new Tuple2>("california", - new Tuple2("dodgers", "giants")), - new Tuple2>("new york", - new Tuple2("yankees", "mets"))), + new Tuple2>("california", + new Tuple2("dodgers", "giants")), + new Tuple2>("new york", + new Tuple2("yankees", "mets"))), Sets.newHashSet( new Tuple2>("california", new Tuple2("sharks", "ducks")), -- cgit v1.2.3 From e3cac47e65bd57fb8fec298eaf9412203d525e68 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Sun, 29 Dec 2013 19:54:53 -0800 Subject: Use Comparator instead of Ordering lower object creation costs --- .../org/apache/spark/util/collection/AppendOnlyMap.scala | 7 ++++--- .../spark/util/collection/ExternalAppendOnlyMap.scala | 13 +++++++------ 2 files changed, 11 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala index 7810119847..a32416afae 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala @@ -18,6 +18,7 @@ package org.apache.spark.util.collection import java.util +import java.util.Comparator /** * A simple open hash table optimized for the append-only use case, where keys @@ -240,7 +241,7 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi /** Return an iterator of the map in sorted order. This provides a way to sort the map without * using additional memory, at the expense of destroying the validity of the map. */ - def destructiveSortedIterator(ordering: Ordering[(K, V)]): Iterator[(K, V)] = { + def destructiveSortedIterator(cmp: Comparator[(K, V)]): Iterator[(K, V)] = { var keyIndex, newIndex = 0 // Pack KV pairs into the front of the underlying array while (keyIndex < capacity) { @@ -252,9 +253,9 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi } assert(newIndex == curSize) // Sort by the given ordering - val rawOrdering = new Ordering[AnyRef] { + val rawOrdering = new Comparator[AnyRef] { def compare(x: AnyRef, y: AnyRef): Int = { - ordering.compare(x.asInstanceOf[(K, V)], y.asInstanceOf[(K, V)]) + cmp.compare(x.asInstanceOf[(K, V)], y.asInstanceOf[(K, V)]) } } util.Arrays.sort(data, 0, curSize, rawOrdering) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 1de545c05b..bbf96f71ce 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -18,6 +18,7 @@ package org.apache.spark.util.collection import java.io._ +import java.util.Comparator import scala.collection.mutable.{ArrayBuffer, PriorityQueue} import scala.reflect.ClassTag @@ -113,7 +114,7 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( val bufferPercent = System.getProperty("spark.shuffle.buffer.fraction", "0.8").toFloat bufferSize * bufferPercent } - private val ordering = new KeyGroupOrdering[K, G] + private val comparator = new KeyGroupComparator[K, G] private val ser = serializer.newInstance() def insert(key: K, value: V): Unit = { @@ -129,7 +130,7 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( private def spill(): Unit = { val file = File.createTempFile("external_append_only_map", "") val out = ser.serializeStream(new FileOutputStream(file)) - val it = currentMap.destructiveSortedIterator(ordering) + val it = currentMap.destructiveSortedIterator(comparator) while (it.hasNext) { val kv = it.next() out.writeObject(kv) @@ -150,7 +151,7 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( // An iterator that sort-merges (K, G) pairs from memory and disk into (K, C) pairs private class ExternalIterator extends Iterator[(K, C)] { val mergeHeap = new PriorityQueue[KGITuple] - val inputStreams = oldMaps ++ Seq(currentMap.destructiveSortedIterator(ordering)) + val inputStreams = oldMaps ++ Seq(currentMap.destructiveSortedIterator(comparator)) // Invariant: size of mergeHeap == number of input streams inputStreams.foreach{ it => @@ -241,7 +242,7 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( } case class KGITuple(iterator: Iterator[(K, G)], pairs: ArrayBuffer[(K, G)]) - extends Ordered[KGITuple] { + extends Comparable[KGITuple] { // Invariant: pairs are ordered by key hash def minHash: Int = { @@ -252,7 +253,7 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( } } - def compare(other: KGITuple): Int = { + override def compareTo(other: KGITuple): Int = { // mutable.PriorityQueue dequeues the max, not the min -minHash.compareTo(other.minHash) } @@ -299,7 +300,7 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( } private[spark] object SpillableAppendOnlyMap { - private class KeyGroupOrdering[K, G] extends Ordering[(K, G)] { + private class KeyGroupComparator[K, G] extends Comparator[(K, G)] { def compare(kg1: (K, G), kg2: (K, G)): Int = { kg1._1.hashCode().compareTo(kg2._1.hashCode()) } -- cgit v1.2.3 From 4a014dc59c15bd35e025a754cf436629117e581a Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sun, 29 Dec 2013 21:55:53 -0800 Subject: Make serializer a parameter to ExternalAppendOnlyMap --- core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala | 4 ++-- .../org/apache/spark/util/collection/ExternalAppendOnlyMap.scala | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 77a594a3e4..9066215866 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -173,8 +173,8 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: case (v1, v2) => v1 ++ v2 } } - new ExternalAppendOnlyMap[K, CoGroupValue, CoGroupCombiner]( - createCombiner, mergeValue, mergeCombiners) + new ExternalAppendOnlyMap[K, CoGroupValue, CoGroupCombiner](createCombiner, mergeValue, + mergeCombiners, SparkEnv.get.serializerManager.get(serializerClass)) } override def clearDependencies() { diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 1de545c05b..b15cae1259 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -41,10 +41,10 @@ import org.apache.spark.serializer.Serializer private[spark] class ExternalAppendOnlyMap[K, V, C: ClassTag]( createCombiner: V => C, mergeValue: (C, V) => C, - mergeCombiners: (C, C) => C) + mergeCombiners: (C, C) => C, + serializer: Serializer = SparkEnv.get.serializerManager.default) extends Iterable[(K, C)] with Serializable { - private val serializer = SparkEnv.get.serializerManager.default private val mergeBeforeSpill: Boolean = mergeCombiners != null private val map: SpillableAppendOnlyMap[K, V, _, C] = { -- cgit v1.2.3 From 2a48d71528f8be31068b9d60f161bf8e8b32c659 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sun, 29 Dec 2013 21:56:13 -0800 Subject: Add test suite for ExternalAppendOnlyMap --- .../collection/ExternalAppendOnlyMapSuite.scala | 217 +++++++++++++++++++++ 1 file changed, 217 insertions(+) create mode 100644 core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala new file mode 100644 index 0000000000..3bc88caaf3 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -0,0 +1,217 @@ +package org.apache.spark.util.collection + +import scala.collection.mutable.ArrayBuffer + +import org.scalatest.{BeforeAndAfter, FunSuite} + +import org.apache.spark.{HashPartitioner, SparkContext, SparkEnv, LocalSparkContext} +import org.apache.spark.SparkContext.rddToPairRDDFunctions + +class ExternalAppendOnlyMapSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { + + override def beforeEach() { + sc = new SparkContext("local", "test") + System.setProperty("spark.shuffle.externalSorting", "true") + } + + after { + System.setProperty("spark.shuffle.externalSorting", "false") + System.setProperty("spark.shuffle.buffer.mb", "1024") + System.setProperty("spark.shuffle.buffer.fraction", "0.8") + } + + val createCombiner: (Int => ArrayBuffer[Int]) = i => ArrayBuffer[Int](i) + val mergeValue: (ArrayBuffer[Int], Int) => ArrayBuffer[Int] = (buffer, i) => { + buffer += i + } + val mergeCombiners: (ArrayBuffer[Int], ArrayBuffer[Int]) => ArrayBuffer[Int] = + (buf1, buf2) => { + buf1 ++= buf2 + } + + test("simple insert") { + var map = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]](createCombiner, + mergeValue, mergeCombiners) + + // Single insert + map.insert(1, 10) + var it = map.iterator + assert(it.hasNext) + var kv = it.next() + assert(kv._1 == 1 && kv._2 == ArrayBuffer[Int](10)) + assert(!it.hasNext) + + // Multiple insert + map.insert(2, 20) + map.insert(3, 30) + it = map.iterator + assert(it.hasNext) + assert(it.toSet == Set[(Int, ArrayBuffer[Int])]( + (1, ArrayBuffer[Int](10)), + (2, ArrayBuffer[Int](20)), + (3, ArrayBuffer[Int](30)))) + } + + test("insert with collision") { + val map = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]](createCombiner, + mergeValue, mergeCombiners) + + map.insert(1, 10) + map.insert(2, 20) + map.insert(3, 30) + map.insert(1, 100) + map.insert(2, 200) + map.insert(1, 1000) + var it = map.iterator + assert(it.hasNext) + val result = it.toSet[(Int, ArrayBuffer[Int])].map(kv => (kv._1, kv._2.toSet)) + assert(result == Set[(Int, Set[Int])]( + (1, Set[Int](10, 100, 1000)), + (2, Set[Int](20, 200)), + (3, Set[Int](30)))) + } + + test("ordering") { + val map1 = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]](createCombiner, + mergeValue, mergeCombiners) + map1.insert(1, 10) + map1.insert(2, 20) + map1.insert(3, 30) + + val map2 = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]](createCombiner, + mergeValue, mergeCombiners) + map2.insert(2, 20) + map2.insert(3, 30) + map2.insert(1, 10) + + val map3 = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]](createCombiner, + mergeValue, mergeCombiners) + map3.insert(3, 30) + map3.insert(1, 10) + map3.insert(2, 20) + + val it1 = map1.iterator + val it2 = map2.iterator + val it3 = map3.iterator + + var kv1 = it1.next() + var kv2 = it2.next() + var kv3 = it3.next() + assert(kv1._1 == kv2._1 && kv2._1 == kv3._1) + assert(kv1._2 == kv2._2 && kv2._2 == kv3._2) + + kv1 = it1.next() + kv2 = it2.next() + kv3 = it3.next() + assert(kv1._1 == kv2._1 && kv2._1 == kv3._1) + assert(kv1._2 == kv2._2 && kv2._2 == kv3._2) + + kv1 = it1.next() + kv2 = it2.next() + kv3 = it3.next() + assert(kv1._1 == kv2._1 && kv2._1 == kv3._1) + assert(kv1._2 == kv2._2 && kv2._2 == kv3._2) + } + + test("simple aggregator") { + // reduceByKey + val rdd = sc.parallelize(1 to 10).map(i => (i%2, 1)) + val result1 = rdd.reduceByKey(_+_).collect() + assert(result1.toSet == Set[(Int, Int)]((0, 5), (1, 5))) + + // groupByKey + val result2 = rdd.groupByKey().collect() + assert(result2.toSet == Set[(Int, Seq[Int])] + ((0, ArrayBuffer[Int](1, 1, 1, 1, 1)), (1, ArrayBuffer[Int](1, 1, 1, 1, 1)))) + } + + test("simple cogroup") { + val rdd1 = sc.parallelize(1 to 4).map(i => (i, i)) + val rdd2 = sc.parallelize(1 to 4).map(i => (i%2, i)) + val result = rdd1.cogroup(rdd2).collect() + + result.foreach { case (i, (seq1, seq2)) => + i match { + case 0 => assert(seq1.toSet == Set[Int]() && seq2.toSet == Set[Int](2, 4)) + case 1 => assert(seq1.toSet == Set[Int](1) && seq2.toSet == Set[Int](1, 3)) + case 2 => assert(seq1.toSet == Set[Int](2) && seq2.toSet == Set[Int]()) + case 3 => assert(seq1.toSet == Set[Int](3) && seq2.toSet == Set[Int]()) + case 4 => assert(seq1.toSet == Set[Int](4) && seq2.toSet == Set[Int]()) + } + } + } + + test("spilling") { + System.setProperty("spark.shuffle.buffer.mb", "1") + System.setProperty("spark.shuffle.buffer.fraction", "0.05") + + // reduceByKey - should spill exactly 6 times + val rddA = sc.parallelize(0 until 10000).map(i => (i/2, i)) + val resultA = rddA.reduceByKey(math.max(_, _)).collect() + assert(resultA.length == 5000) + resultA.foreach { case(k, v) => + k match { + case 0 => assert(v == 1) + case 2500 => assert(v == 5001) + case 4999 => assert(v == 9999) + case _ => + } + } + + // groupByKey - should spill exactly 11 times + val rddB = sc.parallelize(0 until 10000).map(i => (i/4, i)) + val resultB = rddB.groupByKey().collect() + assert(resultB.length == 2500) + resultB.foreach { case(i, seq) => + i match { + case 0 => assert(seq.toSet == Set[Int](0, 1, 2, 3)) + case 1250 => assert(seq.toSet == Set[Int](5000, 5001, 5002, 5003)) + case 2499 => assert(seq.toSet == Set[Int](9996, 9997, 9998, 9999)) + case _ => + } + } + + // cogroup - should spill exactly 7 times + val rddC1 = sc.parallelize(0 until 1000).map(i => (i, i)) + val rddC2 = sc.parallelize(0 until 1000).map(i => (i%100, i)) + val resultC = rddC1.cogroup(rddC2).collect() + assert(resultC.length == 1000) + resultC.foreach { case(i, (seq1, seq2)) => + i match { + case 0 => + assert(seq1.toSet == Set[Int](0)) + assert(seq2.toSet == Set[Int](0, 100, 200, 300, 400, 500, 600, 700, 800, 900)) + case 500 => + assert(seq1.toSet == Set[Int](500)) + assert(seq2.toSet == Set[Int]()) + case 999 => + assert(seq1.toSet == Set[Int](999)) + assert(seq2.toSet == Set[Int]()) + case _ => + } + } + } + + test("spilling with no mergeCombiners function") { + System.setProperty("spark.shuffle.buffer.mb", "1") + System.setProperty("spark.shuffle.buffer.fraction", "0.05") + + // combineByKey - should spill exactly 11 times + val _createCombiner: Int => ArrayBuffer[Int] = i => ArrayBuffer[Int](i) + val _mergeValue: (ArrayBuffer[Int], Int) => ArrayBuffer[Int] = (buf, i) => buf += i + val rdd = sc.parallelize(0 until 10000).map(i => (i/4, i)) + val result = rdd.combineByKey[ArrayBuffer[Int]](_createCombiner, _mergeValue, null, + new HashPartitioner(1), mapSideCombine=false).collect() + + // result should be the same as groupByKey + assert(result.length == 2500) + result.foreach { case(i, seq) => + i match { + case 0 => assert(seq.toSet == Set[Int](0, 1, 2, 3)) + case 1250 => assert(seq.toSet == Set[Int](5000, 5001, 5002, 5003)) + case 2499 => assert(seq.toSet == Set[Int](9996, 9997, 9998, 9999)) + case _ => + } + } + } +} -- cgit v1.2.3 From 2b71ab97c4787e7f82d026e37b41f3a5767b4e89 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 30 Dec 2013 11:01:30 -0800 Subject: Merge pull request from aarondav: Utilize DiskBlockManager pathway for temp file writing This gives us a couple advantages: - Uses spark.local.dir and randomly selects a directory/disk. - Ensure files are deleted on normal DiskBlockManager cleanup. - Availability of same stats as usual DiskBlockObjectWriter (currenty unused). Also enable basic cleanup when iterator is fully drained. Still requires cleanup for operations that fail or don't go through all elements. --- .../scala/org/apache/spark/storage/BlockId.scala | 11 +++++- .../apache/spark/storage/DiskBlockManager.scala | 11 +++++- .../util/collection/ExternalAppendOnlyMap.scala | 43 +++++++++++++++------- 3 files changed, 49 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index 7156d855d8..c5dacf3fd2 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -22,7 +22,7 @@ package org.apache.spark.storage * A Block can be uniquely identified by its filename, but each type of Block has a different * set of keys which produce its unique name. * - * If your BlockId should be serializable, be sure to add it to the BlockId.fromString() method. + * If your BlockId should be serializable, be sure to add it to the BlockId.apply() method. */ private[spark] sealed abstract class BlockId { /** A globally unique identifier for this Block. Can be used for ser/de. */ @@ -55,7 +55,8 @@ private[spark] case class BroadcastBlockId(broadcastId: Long) extends BlockId { def name = "broadcast_" + broadcastId } -private[spark] case class BroadcastHelperBlockId(broadcastId: BroadcastBlockId, hType: String) extends BlockId { +private[spark] +case class BroadcastHelperBlockId(broadcastId: BroadcastBlockId, hType: String) extends BlockId { def name = broadcastId.name + "_" + hType } @@ -67,6 +68,11 @@ private[spark] case class StreamBlockId(streamId: Int, uniqueId: Long) extends B def name = "input-" + streamId + "-" + uniqueId } +/** Block associated with intermediate (temporary) data managed as blocks. */ +private[spark] case class IntermediateBlockId(id: String) extends BlockId { + def name = "intermediate_" + id +} + // Intended only for testing purposes private[spark] case class TestBlockId(id: String) extends BlockId { def name = "test_" + id @@ -79,6 +85,7 @@ private[spark] object BlockId { val BROADCAST_HELPER = "broadcast_([0-9]+)_([A-Za-z0-9]+)".r val TASKRESULT = "taskresult_([0-9]+)".r val STREAM = "input-([0-9]+)-([0-9]+)".r + val INTERMEDIATE = "intermediate_(.*)".r val TEST = "test_(.*)".r /** Converts a BlockId "name" String back into a BlockId. */ diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index fcd2e97982..58320f254a 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -19,7 +19,7 @@ package org.apache.spark.storage import java.io.File import java.text.SimpleDateFormat -import java.util.{Date, Random} +import java.util.{Date, Random, UUID} import org.apache.spark.Logging import org.apache.spark.executor.ExecutorExitCode @@ -90,6 +90,15 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD def getFile(blockId: BlockId): File = getFile(blockId.name) + /** Produces a unique block id and File suitable for intermediate results. */ + def createIntermediateBlock: (IntermediateBlockId, File) = { + var blockId = new IntermediateBlockId(UUID.randomUUID().toString) + while (getFile(blockId).exists()) { + blockId = new IntermediateBlockId(UUID.randomUUID().toString) + } + (blockId, getFile(blockId)) + } + private def createLocalDirs(): Array[File] = { logDebug("Creating local directories at root dirs '" + rootDirs + "'") val dateFormat = new SimpleDateFormat("yyyyMMddHHmmss") diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index ac9431cb0d..4349e8d638 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -25,6 +25,7 @@ import scala.reflect.ClassTag import org.apache.spark.SparkEnv import org.apache.spark.serializer.Serializer +import org.apache.spark.storage.{DiskBlockManager, DiskBlockObjectWriter} /** * A wrapper for SpillableAppendOnlyMap that handles two cases: @@ -43,7 +44,8 @@ private[spark] class ExternalAppendOnlyMap[K, V, C: ClassTag]( createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, - serializer: Serializer = SparkEnv.get.serializerManager.default) + serializer: Serializer = SparkEnv.get.serializerManager.default, + diskBlockManager: DiskBlockManager = SparkEnv.get.blockManager.diskBlockManager) extends Iterable[(K, C)] with Serializable { private val mergeBeforeSpill: Boolean = mergeCombiners != null @@ -51,7 +53,7 @@ private[spark] class ExternalAppendOnlyMap[K, V, C: ClassTag]( private val map: SpillableAppendOnlyMap[K, V, _, C] = { if (mergeBeforeSpill) { new SpillableAppendOnlyMap[K, V, C, C] (createCombiner, mergeValue, mergeCombiners, - Predef.identity, serializer) + Predef.identity, serializer, diskBlockManager) } else { // Use ArrayBuffer[V] as the intermediate combiner val createGroup: (V => ArrayBuffer[V]) = value => ArrayBuffer[V](value) @@ -72,7 +74,7 @@ private[spark] class ExternalAppendOnlyMap[K, V, C: ClassTag]( combiner.getOrElse(null.asInstanceOf[C]) } new SpillableAppendOnlyMap[K, V, ArrayBuffer[V], C](createGroup, mergeValueIntoGroup, - mergeGroups, combineGroup, serializer) + mergeGroups, combineGroup, serializer, diskBlockManager) } } @@ -102,7 +104,8 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( mergeValue: (G, V) => G, mergeGroups: (G, G) => G, createCombiner: G => C, - serializer: Serializer) + serializer: Serializer, + diskBlockManager: DiskBlockManager) extends Iterable[(K, C)] with Serializable { import SpillableAppendOnlyMap._ @@ -114,6 +117,8 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( val bufferPercent = System.getProperty("spark.shuffle.buffer.fraction", "0.8").toFloat bufferSize * bufferPercent } + private val fileBufferSize = + System.getProperty("spark.shuffle.file.buffer.kb", "100").toInt * 1024 private val comparator = new KeyGroupComparator[K, G] private val ser = serializer.newInstance() @@ -128,14 +133,19 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( } private def spill(): Unit = { - val file = File.createTempFile("external_append_only_map", "") - val out = ser.serializeStream(new FileOutputStream(file)) - val it = currentMap.destructiveSortedIterator(comparator) - while (it.hasNext) { - val kv = it.next() - out.writeObject(kv) + val (blockId, file) = diskBlockManager.createIntermediateBlock + val writer = new DiskBlockObjectWriter(blockId, file, serializer, fileBufferSize, identity) + try { + val it = currentMap.destructiveSortedIterator(comparator) + while (it.hasNext) { + val kv = it.next() + writer.write(kv) + } + writer.commit() + } finally { + // Partial failures cannot be tolerated; do not revert partial writes + writer.close() } - out.close() currentMap = new SizeTrackingAppendOnlyMap[K, G] oldMaps.append(new DiskKGIterator(file)) } @@ -210,7 +220,7 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( } override def next(): (K, C) = { - var minKGI = mergeHeap.dequeue() + val minKGI = mergeHeap.dequeue() val (minPairs, minHash) = (minKGI.pairs, minKGI.minHash) if (minPairs.length == 0) { // Should only happen when hasNext is false @@ -271,7 +281,9 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( try { return Some(in.readObject().asInstanceOf[(K, G)]) } catch { - case e: EOFException => eof = true + case e: EOFException => + eof = true + cleanup() } } None @@ -296,6 +308,11 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( item.getOrElse(throw new NoSuchElementException) } } + + // TODO: Ensure this gets called even if the iterator isn't drained. + def cleanup() { + file.delete() + } } } -- cgit v1.2.3 From d6e7910d925039d9b57d82e7ca17e775c52fbee5 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 30 Dec 2013 13:01:00 -0800 Subject: Simplify merge logic based on the invariant that all spills contain unique keys --- .../util/collection/ExternalAppendOnlyMap.scala | 59 ++++++++-------------- 1 file changed, 22 insertions(+), 37 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 4349e8d638..0e8f46cfc7 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -185,29 +185,15 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( kgPairs } - // Drop and return all (K, G) pairs with K = the given key from the given KGITuple - def dropKey(kgi: KGITuple, key: K): ArrayBuffer[(K, G)] = { - val dropped = new ArrayBuffer[(K, G)] - var i = 0 - while (i < kgi.pairs.length) { - if (kgi.pairs(i)._1 == key) { - dropped += kgi.pairs.remove(i) - } else { - i += 1 - } - } - dropped - } - - // Merge all (K, G) pairs with K = the given key into baseGroup - def mergeIntoGroup(key: K, baseGroup: G, kgPairs: ArrayBuffer[(K, G)]): G = { - var mergedGroup = baseGroup - kgPairs.foreach { case (k, g) => - if (k == key){ - mergedGroup = mergeGroups(mergedGroup, g) + // From the given KGITuple, remove the (K, G) pair with K = key and merge it into baseGroup + def mergeIntoGroup(key: K, baseGroup: G, kgi: KGITuple): G = { + kgi.pairs.zipWithIndex.foreach { case ((k, g), i) => + if (k == key) { + kgi.pairs.remove(i) + return mergeGroups(baseGroup, g) } } - mergedGroup + baseGroup } override def hasNext: Boolean = { @@ -226,28 +212,27 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( // Should only happen when hasNext is false throw new NoSuchElementException } - var (minKey, minGroup) = minPairs(0) - assert(minKey.hashCode() == minHash) - // Merge the rest of minPairs into minGroup - val minPairsWithKey = dropKey(minKGI, minKey).tail - minGroup = mergeIntoGroup(minKey, minGroup, minPairsWithKey) - if (minPairs.length == 0) { - minPairs ++= readFromIterator(minKGI.iterator) - } + // Select a return key with the minimum hash + var (minKey, minGroup) = minPairs.remove(0) + assert(minKey.hashCode() == minHash) - // Do the same for all other KGITuples with the same minHash - val tuplesToAddBack = ArrayBuffer[KGITuple](minKGI) + // Merge all other KGITuple's with the same minHash + val dequeuedKGI = ArrayBuffer[KGITuple](minKGI) while (!mergeHeap.isEmpty && mergeHeap.head.minHash == minHash) { val newKGI = mergeHeap.dequeue() - val pairsWithKey = dropKey(newKGI, minKey) - minGroup = mergeIntoGroup(minKey, minGroup, pairsWithKey) - if (newKGI.pairs.length == 0) { - newKGI.pairs ++= readFromIterator(newKGI.iterator) + minGroup = mergeIntoGroup(minKey, minGroup, newKGI) + dequeuedKGI += newKGI + } + + // Repopulate and add back all dequeued KGI to mergeHeap + dequeuedKGI.foreach { kgi => + if (kgi.pairs.length == 0) { + kgi.pairs ++= readFromIterator(kgi.iterator) } - tuplesToAddBack += newKGI + mergeHeap.enqueue(kgi) } - tuplesToAddBack.foreach(mergeHeap.enqueue(_)) + (minKey, createCombiner(minGroup)) } -- cgit v1.2.3 From 347fafe4fccc9345ed0ffa6c7863bc233c079b43 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 30 Dec 2013 13:10:33 -0800 Subject: Fix CheckpointSuite test fail --- core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 9066215866..77a594a3e4 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -173,8 +173,8 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: case (v1, v2) => v1 ++ v2 } } - new ExternalAppendOnlyMap[K, CoGroupValue, CoGroupCombiner](createCombiner, mergeValue, - mergeCombiners, SparkEnv.get.serializerManager.get(serializerClass)) + new ExternalAppendOnlyMap[K, CoGroupValue, CoGroupCombiner]( + createCombiner, mergeValue, mergeCombiners) } override def clearDependencies() { -- cgit v1.2.3 From daa7792ad654e24012439db79c5a7f4abf149dc1 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Mon, 30 Dec 2013 23:07:29 -0800 Subject: Refactor SamplingSizeTracker into SizeTrackingAppendOnlyMap --- .../apache/spark/util/SamplingSizeTracker.scala | 83 -------------- .../collection/SizeTrackingAppendOnlyMap.scala | 71 ++++++++++-- .../spark/util/SamplingSizeTrackerSuite.scala | 120 --------------------- .../util/SizeTrackingAppendOnlyMapSuite.scala | 120 +++++++++++++++++++++ 4 files changed, 184 insertions(+), 210 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/util/SamplingSizeTracker.scala delete mode 100644 core/src/test/scala/org/apache/spark/util/SamplingSizeTrackerSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/util/SizeTrackingAppendOnlyMapSuite.scala diff --git a/core/src/main/scala/org/apache/spark/util/SamplingSizeTracker.scala b/core/src/main/scala/org/apache/spark/util/SamplingSizeTracker.scala deleted file mode 100644 index 3eb80661cb..0000000000 --- a/core/src/main/scala/org/apache/spark/util/SamplingSizeTracker.scala +++ /dev/null @@ -1,83 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.util - -import org.apache.spark.util.SamplingSizeTracker.Sample - -/** - * Estimates the size of an object as it grows, in bytes. - * We sample with a slow exponential back-off using the SizeEstimator to amortize the time, - * as each call to SizeEstimator can take a sizable amount of time (order of a few milliseconds). - * - * Users should call updateMade() every time their object is updated with new data, or - * flushSamples() if there is a non-linear change in object size (otherwise linear is assumed). - * Not threadsafe. - */ -private[spark] class SamplingSizeTracker(obj: AnyRef) { - /** - * Controls the base of the exponential which governs the rate of sampling. - * E.g., a value of 2 would mean we sample at 1, 2, 4, 8, ... elements. - */ - private val SAMPLE_GROWTH_RATE = 1.1 - - private var lastLastSample: Sample = _ - private var lastSample: Sample = _ - - private var numUpdates: Long = _ - private var nextSampleNum: Long = _ - - flushSamples() - - /** Called after a non-linear change in the tracked object. Takes a new sample. */ - def flushSamples() { - numUpdates = 0 - nextSampleNum = 1 - // Throw out both prior samples to avoid overestimating delta. - lastSample = Sample(SizeEstimator.estimate(obj), 0) - lastLastSample = lastSample - } - - /** To be called after an update to the tracked object. Amortized O(1) time. */ - def updateMade() { - numUpdates += 1 - if (nextSampleNum == numUpdates) { - lastLastSample = lastSample - lastSample = Sample(SizeEstimator.estimate(obj), numUpdates) - nextSampleNum = math.ceil(numUpdates * SAMPLE_GROWTH_RATE).toLong - } - } - - /** Estimates the current size of the tracked object. O(1) time. */ - def estimateSize(): Long = { - val interpolatedDelta = - if (lastLastSample != null && lastLastSample != lastSample) { - (lastSample.size - lastLastSample.size).toDouble / - (lastSample.numUpdates - lastLastSample.numUpdates) - } else if (lastSample.numUpdates > 0) { - lastSample.size.toDouble / lastSample.numUpdates - } else { - 0 - } - val extrapolatedDelta = math.max(0, interpolatedDelta * (numUpdates - lastSample.numUpdates)) - (lastSample.size + extrapolatedDelta).toLong - } -} - -object SamplingSizeTracker { - case class Sample(size: Long, numUpdates: Long) -} diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala index ea0f2fd68f..e8401ab9d7 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala @@ -17,28 +17,85 @@ package org.apache.spark.util.collection -import org.apache.spark.util.SamplingSizeTracker +import scala.collection.mutable.ArrayBuffer -/** Append-only map that keeps track of its estimated size in bytes. */ +import org.apache.spark.util.SizeEstimator +import org.apache.spark.util.collection.SizeTrackingAppendOnlyMap.Sample + +/** + * Append-only map that keeps track of its estimated size in bytes. + * We sample with a slow exponential back-off using the SizeEstimator to amortize the time, + * as each call to SizeEstimator can take a sizable amount of time (order of a few milliseconds). + */ private[spark] class SizeTrackingAppendOnlyMap[K, V] extends AppendOnlyMap[K, V] { - private val sizeTracker = new SamplingSizeTracker(this) + /** + * Controls the base of the exponential which governs the rate of sampling. + * E.g., a value of 2 would mean we sample at 1, 2, 4, 8, ... elements. + */ + private val SAMPLE_GROWTH_RATE = 1.1 + + /** All samples taken since last resetSamples(). Only the last two are used for extrapolation. */ + private val samples = new ArrayBuffer[Sample]() + + /** Total number of insertions and updates into the map since the last resetSamples(). */ + private var numUpdates: Long = _ - def estimateSize() = sizeTracker.estimateSize() + /** The value of 'numUpdates' at which we will take our next sample. */ + private var nextSampleNum: Long = _ + + /** The average number of bytes per update between our last two samples. */ + private var bytesPerUpdate: Double = _ + + resetSamples() + + /** Called after the map grows in size, as this can be a dramatic change for small objects. */ + def resetSamples() { + numUpdates = 1 + nextSampleNum = 1 + samples.clear() + takeSample() + } override def update(key: K, value: V): Unit = { super.update(key, value) - sizeTracker.updateMade() + numUpdates += 1 + if (nextSampleNum == numUpdates) { takeSample() } } override def changeValue(key: K, updateFunc: (Boolean, V) => V): V = { val newValue = super.changeValue(key, updateFunc) - sizeTracker.updateMade() + numUpdates += 1 + if (nextSampleNum == numUpdates) { takeSample() } newValue } + /** Takes a new sample of the current map's size. */ + def takeSample() { + samples += Sample(SizeEstimator.estimate(this), numUpdates) + // Only use the last two samples to extrapolate. If fewer than 2 samples, assume no change. + bytesPerUpdate = math.max(0, samples.toSeq.reverse match { + case latest :: previous :: tail => + (latest.size - previous.size).toDouble / (latest.numUpdates - previous.numUpdates) + case _ => + 0 + }) + nextSampleNum = math.ceil(numUpdates * SAMPLE_GROWTH_RATE).toLong + } + override protected def growTable() { super.growTable() - sizeTracker.flushSamples() + resetSamples() + } + + /** Estimates the current size of the map in bytes. O(1) time. */ + def estimateSize(): Long = { + assert(samples.nonEmpty) + val extrapolatedDelta = bytesPerUpdate * (numUpdates - samples.last.numUpdates) + (samples.last.size + extrapolatedDelta).toLong } } + +object SizeTrackingAppendOnlyMap { + case class Sample(size: Long, numUpdates: Long) +} \ No newline at end of file diff --git a/core/src/test/scala/org/apache/spark/util/SamplingSizeTrackerSuite.scala b/core/src/test/scala/org/apache/spark/util/SamplingSizeTrackerSuite.scala deleted file mode 100644 index 47e4723cf3..0000000000 --- a/core/src/test/scala/org/apache/spark/util/SamplingSizeTrackerSuite.scala +++ /dev/null @@ -1,120 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.util - -import scala.util.Random - -import org.scalatest.{BeforeAndAfterAll, FunSuite} - -import org.apache.spark.util.SamplingSizeTrackerSuite.LargeDummyClass -import org.apache.spark.util.collection.{AppendOnlyMap, SizeTrackingAppendOnlyMap} - -class SamplingSizeTrackerSuite extends FunSuite with BeforeAndAfterAll { - val NORMAL_ERROR = 0.20 - val HIGH_ERROR = 0.30 - - test("fixed size insertions") { - testWith[Int, Long](10000, i => (i, i.toLong)) - testWith[Int, (Long, Long)](10000, i => (i, (i.toLong, i.toLong))) - testWith[Int, LargeDummyClass](10000, i => (i, new LargeDummyClass())) - } - - test("variable size insertions") { - val rand = new Random(123456789) - def randString(minLen: Int, maxLen: Int): String = { - "a" * (rand.nextInt(maxLen - minLen) + minLen) - } - testWith[Int, String](10000, i => (i, randString(0, 10))) - testWith[Int, String](10000, i => (i, randString(0, 100))) - testWith[Int, String](10000, i => (i, randString(90, 100))) - } - - test("updates") { - val rand = new Random(123456789) - def randString(minLen: Int, maxLen: Int): String = { - "a" * (rand.nextInt(maxLen - minLen) + minLen) - } - testWith[String, Int](10000, i => (randString(0, 10000), i)) - } - - def testWith[K, V](numElements: Int, makeElement: (Int) => (K, V)) { - val map = new SizeTrackingAppendOnlyMap[K, V]() - for (i <- 0 until numElements) { - val (k, v) = makeElement(i) - map(k) = v - expectWithinError(map, map.estimateSize(), if (i < 32) HIGH_ERROR else NORMAL_ERROR) - } - } - - def expectWithinError(obj: AnyRef, estimatedSize: Long, error: Double) { - val betterEstimatedSize = SizeEstimator.estimate(obj) - assert(betterEstimatedSize * (1 - error) < estimatedSize, - s"Estimated size $estimatedSize was less than expected size $betterEstimatedSize") - assert(betterEstimatedSize * (1 + 2 * error) > estimatedSize, - s"Estimated size $estimatedSize was greater than expected size $betterEstimatedSize") - } -} - -object SamplingSizeTrackerSuite { - // Speed test, for reproducibility of results. - // These could be highly non-deterministic in general, however. - // Results: - // AppendOnlyMap: 30 ms - // SizeTracker: 45 ms - // SizeEstimator: 1500 ms - def main(args: Array[String]) { - val numElements = 100000 - - val baseTimes = for (i <- 0 until 3) yield time { - val map = new AppendOnlyMap[Int, LargeDummyClass]() - for (i <- 0 until numElements) { - map(i) = new LargeDummyClass() - } - } - - val sampledTimes = for (i <- 0 until 3) yield time { - val map = new SizeTrackingAppendOnlyMap[Int, LargeDummyClass]() - for (i <- 0 until numElements) { - map(i) = new LargeDummyClass() - map.estimateSize() - } - } - - val unsampledTimes = for (i <- 0 until 3) yield time { - val map = new AppendOnlyMap[Int, LargeDummyClass]() - for (i <- 0 until numElements) { - map(i) = new LargeDummyClass() - SizeEstimator.estimate(map) - } - } - - println("Base: " + baseTimes) - println("SizeTracker (sampled): " + sampledTimes) - println("SizeEstimator (unsampled): " + unsampledTimes) - } - - def time(f: => Unit): Long = { - val start = System.currentTimeMillis() - f - System.currentTimeMillis() - start - } - - private class LargeDummyClass { - val arr = new Array[Int](100) - } -} diff --git a/core/src/test/scala/org/apache/spark/util/SizeTrackingAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/SizeTrackingAppendOnlyMapSuite.scala new file mode 100644 index 0000000000..93f0c6a8e6 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/SizeTrackingAppendOnlyMapSuite.scala @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import scala.util.Random + +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +import org.apache.spark.util.SizeTrackingAppendOnlyMapSuite.LargeDummyClass +import org.apache.spark.util.collection.{AppendOnlyMap, SizeTrackingAppendOnlyMap} + +class SizeTrackingAppendOnlyMapSuite extends FunSuite with BeforeAndAfterAll { + val NORMAL_ERROR = 0.20 + val HIGH_ERROR = 0.30 + + test("fixed size insertions") { + testWith[Int, Long](10000, i => (i, i.toLong)) + testWith[Int, (Long, Long)](10000, i => (i, (i.toLong, i.toLong))) + testWith[Int, LargeDummyClass](10000, i => (i, new LargeDummyClass())) + } + + test("variable size insertions") { + val rand = new Random(123456789) + def randString(minLen: Int, maxLen: Int): String = { + "a" * (rand.nextInt(maxLen - minLen) + minLen) + } + testWith[Int, String](10000, i => (i, randString(0, 10))) + testWith[Int, String](10000, i => (i, randString(0, 100))) + testWith[Int, String](10000, i => (i, randString(90, 100))) + } + + test("updates") { + val rand = new Random(123456789) + def randString(minLen: Int, maxLen: Int): String = { + "a" * (rand.nextInt(maxLen - minLen) + minLen) + } + testWith[String, Int](10000, i => (randString(0, 10000), i)) + } + + def testWith[K, V](numElements: Int, makeElement: (Int) => (K, V)) { + val map = new SizeTrackingAppendOnlyMap[K, V]() + for (i <- 0 until numElements) { + val (k, v) = makeElement(i) + map(k) = v + expectWithinError(map, map.estimateSize(), if (i < 32) HIGH_ERROR else NORMAL_ERROR) + } + } + + def expectWithinError(obj: AnyRef, estimatedSize: Long, error: Double) { + val betterEstimatedSize = SizeEstimator.estimate(obj) + assert(betterEstimatedSize * (1 - error) < estimatedSize, + s"Estimated size $estimatedSize was less than expected size $betterEstimatedSize") + assert(betterEstimatedSize * (1 + 2 * error) > estimatedSize, + s"Estimated size $estimatedSize was greater than expected size $betterEstimatedSize") + } +} + +object SizeTrackingAppendOnlyMapSuite { + // Speed test, for reproducibility of results. + // These could be highly non-deterministic in general, however. + // Results: + // AppendOnlyMap: 31 ms + // SizeTracker: 54 ms + // SizeEstimator: 1500 ms + def main(args: Array[String]) { + val numElements = 100000 + + val baseTimes = for (i <- 0 until 10) yield time { + val map = new AppendOnlyMap[Int, LargeDummyClass]() + for (i <- 0 until numElements) { + map(i) = new LargeDummyClass() + } + } + + val sampledTimes = for (i <- 0 until 10) yield time { + val map = new SizeTrackingAppendOnlyMap[Int, LargeDummyClass]() + for (i <- 0 until numElements) { + map(i) = new LargeDummyClass() + map.estimateSize() + } + } + + val unsampledTimes = for (i <- 0 until 3) yield time { + val map = new AppendOnlyMap[Int, LargeDummyClass]() + for (i <- 0 until numElements) { + map(i) = new LargeDummyClass() + SizeEstimator.estimate(map) + } + } + + println("Base: " + baseTimes) + println("SizeTracker (sampled): " + sampledTimes) + println("SizeEstimator (unsampled): " + unsampledTimes) + } + + def time(f: => Unit): Long = { + val start = System.currentTimeMillis() + f + System.currentTimeMillis() - start + } + + private class LargeDummyClass { + val arr = new Array[Int](100) + } +} -- cgit v1.2.3 From 375d11743cda0781e9fb929920851ebef424dcf6 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Mon, 30 Dec 2013 23:42:37 -0800 Subject: Add new line at end of file --- .../org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala index e8401ab9d7..e6b6103d96 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala @@ -98,4 +98,4 @@ private[spark] class SizeTrackingAppendOnlyMap[K, V] extends AppendOnlyMap[K, V] object SizeTrackingAppendOnlyMap { case class Sample(size: Long, numUpdates: Long) -} \ No newline at end of file +} -- cgit v1.2.3 From 94ddc91d063f290a0e230a153f9e63b2f7357d4a Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 31 Dec 2013 10:50:08 -0800 Subject: Address Aaron's and Jerry's comments --- core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala | 4 +--- .../org/apache/spark/util/collection/ExternalAppendOnlyMap.scala | 7 +++++-- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 77a594a3e4..1b2e5417e7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -169,10 +169,8 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: } val mergeCombiners: (CoGroupCombiner, CoGroupCombiner) => CoGroupCombiner = (combiner1, combiner2) => { - combiner1.zipAll(combiner2, new CoGroup, new CoGroup).map { - case (v1, v2) => v1 ++ v2 + combiner1.zip(combiner2).map { case (v1, v2) => v1 ++ v2 } } - } new ExternalAppendOnlyMap[K, CoGroupValue, CoGroupCombiner]( createCombiner, mergeValue, mergeCombiners) } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 0e8f46cfc7..680ebf9b80 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -257,14 +257,15 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( // Iterate through (K, G) pairs in sorted order from an on-disk map private class DiskKGIterator(file: File) extends Iterator[(K, G)] { - val in = ser.deserializeStream(new FileInputStream(file)) + val fstream = new FileInputStream(file) + val dstream = ser.deserializeStream(fstream) var nextItem: Option[(K, G)] = None var eof = false def readNextItem(): Option[(K, G)] = { if (!eof) { try { - return Some(in.readObject().asInstanceOf[(K, G)]) + return Some(dstream.readObject().asInstanceOf[(K, G)]) } catch { case e: EOFException => eof = true @@ -296,6 +297,8 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( // TODO: Ensure this gets called even if the iterator isn't drained. def cleanup() { + fstream.close() + dstream.close() file.delete() } } -- cgit v1.2.3 From 3ce22df954a17a582b5000b87db8fa887ad8392b Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 31 Dec 2013 11:33:10 -0800 Subject: Add warning message for spilling --- .../spark/util/collection/ExternalAppendOnlyMap.scala | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 680ebf9b80..317a6c168c 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -23,7 +23,7 @@ import java.util.Comparator import scala.collection.mutable.{ArrayBuffer, PriorityQueue} import scala.reflect.ClassTag -import org.apache.spark.SparkEnv +import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{DiskBlockManager, DiskBlockObjectWriter} @@ -106,14 +106,14 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( createCombiner: G => C, serializer: Serializer, diskBlockManager: DiskBlockManager) - extends Iterable[(K, C)] with Serializable { + extends Iterable[(K, C)] with Serializable with Logging { import SpillableAppendOnlyMap._ private var currentMap = new SizeTrackingAppendOnlyMap[K, G] private val oldMaps = new ArrayBuffer[DiskKGIterator] - private val memoryThreshold = { - val bufferSize = System.getProperty("spark.shuffle.buffer.mb", "1024").toLong * 1024 * 1024 + private val memoryThresholdMB = { + val bufferSize = System.getProperty("spark.shuffle.buffer.mb", "1024").toLong val bufferPercent = System.getProperty("spark.shuffle.buffer.fraction", "0.8").toFloat bufferSize * bufferPercent } @@ -121,18 +121,22 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( System.getProperty("spark.shuffle.file.buffer.kb", "100").toInt * 1024 private val comparator = new KeyGroupComparator[K, G] private val ser = serializer.newInstance() + private var spillCount = 0 def insert(key: K, value: V): Unit = { val update: (Boolean, G) => G = (hadVal, oldVal) => { if (hadVal) mergeValue(oldVal, value) else createGroup(value) } currentMap.changeValue(key, update) - if (currentMap.estimateSize() > memoryThreshold) { + if (currentMap.estimateSize() > memoryThresholdMB * 1024 * 1024) { spill() } } private def spill(): Unit = { + spillCount += 1 + logWarning(s"In-memory KV map exceeded threshold of $memoryThresholdMB MB!") + logWarning(s"Spilling to disk ($spillCount time"+(if (spillCount > 1) "s" else "")+" so far)") val (blockId, file) = diskBlockManager.createIntermediateBlock val writer = new DiskBlockObjectWriter(blockId, file, serializer, fileBufferSize, identity) try { -- cgit v1.2.3 From 53d8d36684b16ae536a5e065e690bb21b9aadc49 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 31 Dec 2013 17:19:02 -0800 Subject: Add support and test for null keys in ExternalAppendOnlyMap Also add safeguard against use of destructively sorted AppendOnlyMap --- .../spark/util/collection/AppendOnlyMap.scala | 88 ++++++++++++++-------- .../util/collection/ExternalAppendOnlyMap.scala | 1 + .../spark/util/collection/AppendOnlyMapSuite.scala | 44 +++++++++++ .../collection/ExternalAppendOnlyMapSuite.scala | 38 ++++++++++ 4 files changed, 139 insertions(+), 32 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala index a32416afae..d2a9574a71 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala @@ -48,10 +48,14 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi private var haveNullValue = false private var nullValue: V = null.asInstanceOf[V] + // Triggered by destructiveSortedIterator; the underlying data array may no longer be used + private var destroyed = false + private val LOAD_FACTOR = 0.7 /** Get the value for a given key */ def apply(key: K): V = { + checkValidityOrThrowException() val k = key.asInstanceOf[AnyRef] if (k.eq(null)) { return nullValue @@ -75,6 +79,7 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi /** Set the value for a key */ def update(key: K, value: V): Unit = { + checkValidityOrThrowException() val k = key.asInstanceOf[AnyRef] if (k.eq(null)) { if (!haveNullValue) { @@ -109,6 +114,7 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi * for key, if any, or null otherwise. Returns the newly updated value. */ def changeValue(key: K, updateFunc: (Boolean, V) => V): V = { + checkValidityOrThrowException() val k = key.asInstanceOf[AnyRef] if (k.eq(null)) { if (!haveNullValue) { @@ -142,35 +148,38 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi } /** Iterator method from Iterable */ - override def iterator: Iterator[(K, V)] = new Iterator[(K, V)] { - var pos = -1 + override def iterator: Iterator[(K, V)] = { + checkValidityOrThrowException() + new Iterator[(K, V)] { + var pos = -1 - /** Get the next value we should return from next(), or null if we're finished iterating */ - def nextValue(): (K, V) = { - if (pos == -1) { // Treat position -1 as looking at the null value - if (haveNullValue) { - return (null.asInstanceOf[K], nullValue) + /** Get the next value we should return from next(), or null if we're finished iterating */ + def nextValue(): (K, V) = { + if (pos == -1) { // Treat position -1 as looking at the null value + if (haveNullValue) { + return (null.asInstanceOf[K], nullValue) + } + pos += 1 } - pos += 1 - } - while (pos < capacity) { - if (!data(2 * pos).eq(null)) { - return (data(2 * pos).asInstanceOf[K], data(2 * pos + 1).asInstanceOf[V]) + while (pos < capacity) { + if (!data(2 * pos).eq(null)) { + return (data(2 * pos).asInstanceOf[K], data(2 * pos + 1).asInstanceOf[V]) + } + pos += 1 } - pos += 1 + null } - null - } - override def hasNext: Boolean = nextValue() != null + override def hasNext: Boolean = nextValue() != null - override def next(): (K, V) = { - val value = nextValue() - if (value == null) { - throw new NoSuchElementException("End of iterator") + override def next(): (K, V) = { + val value = nextValue() + if (value == null) { + throw new NoSuchElementException("End of iterator") + } + pos += 1 + value } - pos += 1 - value } } @@ -238,12 +247,14 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi if (highBit == n) n else highBit << 1 } - /** Return an iterator of the map in sorted order. This provides a way to sort the map without - * using additional memory, at the expense of destroying the validity of the map. - */ + /** + * Return an iterator of the map in sorted order. This provides a way to sort the map without + * using additional memory, at the expense of destroying the validity of the map. + */ def destructiveSortedIterator(cmp: Comparator[(K, V)]): Iterator[(K, V)] = { - var keyIndex, newIndex = 0 + destroyed = true // Pack KV pairs into the front of the underlying array + var keyIndex, newIndex = 0 while (keyIndex < capacity) { if (data(2 * keyIndex) != null) { data(newIndex) = (data(2 * keyIndex), data(2 * keyIndex + 1)) @@ -251,23 +262,36 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi } keyIndex += 1 } - assert(newIndex == curSize) + assert(curSize == newIndex + (if (haveNullValue) 1 else 0)) + // Sort by the given ordering val rawOrdering = new Comparator[AnyRef] { def compare(x: AnyRef, y: AnyRef): Int = { cmp.compare(x.asInstanceOf[(K, V)], y.asInstanceOf[(K, V)]) } } - util.Arrays.sort(data, 0, curSize, rawOrdering) + util.Arrays.sort(data, 0, newIndex, rawOrdering) new Iterator[(K, V)] { var i = 0 - def hasNext = i < curSize + var nullValueReady = haveNullValue + def hasNext: Boolean = (i < newIndex || nullValueReady) def next(): (K, V) = { - val item = data(i).asInstanceOf[(K, V)] - i += 1 - item + if (nullValueReady) { + nullValueReady = false + (null.asInstanceOf[K], nullValue) + } else { + val item = data(i).asInstanceOf[(K, V)] + i += 1 + item + } } } } + + private def checkValidityOrThrowException(): Unit = { + if (destroyed) { + throw new IllegalStateException("Map state is invalid from destructive sorting!") + } + } } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 317a6c168c..492b4fc7c6 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -112,6 +112,7 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( private var currentMap = new SizeTrackingAppendOnlyMap[K, G] private val oldMaps = new ArrayBuffer[DiskKGIterator] + private val memoryThresholdMB = { val bufferSize = System.getProperty("spark.shuffle.buffer.mb", "1024").toLong val bufferPercent = System.getProperty("spark.shuffle.buffer.fraction", "0.8").toFloat diff --git a/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala index 7e7aa7800d..71b936b0df 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.util.collection import scala.collection.mutable.HashSet import org.scalatest.FunSuite +import java.util.Comparator class AppendOnlyMapSuite extends FunSuite { test("initialization") { @@ -151,4 +152,47 @@ class AppendOnlyMapSuite extends FunSuite { assert(map("" + i) === "" + i) } } + + test("destructive sort") { + val map = new AppendOnlyMap[String, String]() + for (i <- 1 to 100) { + map("" + i) = "" + i + } + map.update(null, "happy new year!") + + try { + map.apply("1") + map.update("1", "2013") + map.changeValue("1", (hadValue, oldValue) => "2014") + map.iterator + } catch { + case e: IllegalStateException => fail() + } + + val it = map.destructiveSortedIterator(new Comparator[(String, String)] { + def compare(kv1: (String, String), kv2: (String, String)): Int = { + val x = if (kv1 != null && kv1._1 != null) kv1._1.toInt else Int.MinValue + val y = if (kv2 != null && kv2._1 != null) kv2._1.toInt else Int.MinValue + x.compareTo(y) + } + }) + + // Should be sorted by key + assert(it.hasNext) + var previous = it.next() + assert(previous == (null, "happy new year!")) + previous = it.next() + assert(previous == ("1", "2014")) + while (it.hasNext) { + val kv = it.next() + assert(kv._1.toInt > previous._1.toInt) + previous = kv + } + + // All subsequent calls to apply, update, changeValue and iterator should throw exception + intercept[IllegalStateException] { map.apply("1") } + intercept[IllegalStateException] { map.update("1", "2013") } + intercept[IllegalStateException] { map.changeValue("1", (hadValue, oldValue) => "2014") } + intercept[IllegalStateException] { map.iterator } + } } diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index 3bc88caaf3..baf94b4728 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -113,6 +113,44 @@ class ExternalAppendOnlyMapSuite extends FunSuite with BeforeAndAfter with Local assert(kv1._2 == kv2._2 && kv2._2 == kv3._2) } + test("null keys and values") { + val map = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]](createCombiner, + mergeValue, mergeCombiners) + map.insert(1, 5) + map.insert(2, 6) + map.insert(3, 7) + assert(map.size === 3) + assert(map.iterator.toSet == Set[(Int, Seq[Int])]( + (1, Seq[Int](5)), + (2, Seq[Int](6)), + (3, Seq[Int](7)) + )) + + // Null keys + val nullInt = null.asInstanceOf[Int] + map.insert(nullInt, 8) + assert(map.size === 4) + assert(map.iterator.toSet == Set[(Int, Seq[Int])]( + (1, Seq[Int](5)), + (2, Seq[Int](6)), + (3, Seq[Int](7)), + (nullInt, Seq[Int](8)) + )) + + // Null values + map.insert(4, nullInt) + map.insert(nullInt, nullInt) + assert(map.size === 5) + val result = map.iterator.toSet[(Int, ArrayBuffer[Int])].map(kv => (kv._1, kv._2.toSet)) + assert(result == Set[(Int, Set[Int])]( + (1, Set[Int](5)), + (2, Set[Int](6)), + (3, Set[Int](7)), + (4, Set[Int](nullInt)), + (nullInt, Set[Int](nullInt, 8)) + )) + } + test("simple aggregator") { // reduceByKey val rdd = sc.parallelize(1 to 10).map(i => (i%2, 1)) -- cgit v1.2.3 From 08302b113a5db773e3b8d7cfea1ab1d2b8d3695b Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Tue, 31 Dec 2013 17:42:30 -0800 Subject: Rename IntermediateBlockId to TempBlockId --- core/src/main/scala/org/apache/spark/storage/BlockId.scala | 8 ++++---- .../main/scala/org/apache/spark/storage/DiskBlockManager.scala | 6 +++--- .../org/apache/spark/util/collection/ExternalAppendOnlyMap.scala | 2 +- .../apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala | 2 +- 4 files changed, 9 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index c5dacf3fd2..bcc3101485 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -68,9 +68,9 @@ private[spark] case class StreamBlockId(streamId: Int, uniqueId: Long) extends B def name = "input-" + streamId + "-" + uniqueId } -/** Block associated with intermediate (temporary) data managed as blocks. */ -private[spark] case class IntermediateBlockId(id: String) extends BlockId { - def name = "intermediate_" + id +/** Block associated with temporary data managed as blocks. */ +private[spark] case class TempBlockId(id: String) extends BlockId { + def name = "temp_" + id } // Intended only for testing purposes @@ -85,7 +85,7 @@ private[spark] object BlockId { val BROADCAST_HELPER = "broadcast_([0-9]+)_([A-Za-z0-9]+)".r val TASKRESULT = "taskresult_([0-9]+)".r val STREAM = "input-([0-9]+)-([0-9]+)".r - val INTERMEDIATE = "intermediate_(.*)".r + val TEMP = "temp_(.*)".r val TEST = "test_(.*)".r /** Converts a BlockId "name" String back into a BlockId. */ diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 58320f254a..32da458a6f 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -91,10 +91,10 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD def getFile(blockId: BlockId): File = getFile(blockId.name) /** Produces a unique block id and File suitable for intermediate results. */ - def createIntermediateBlock: (IntermediateBlockId, File) = { - var blockId = new IntermediateBlockId(UUID.randomUUID().toString) + def createTempBlock(): (TempBlockId, File) = { + var blockId = new TempBlockId(UUID.randomUUID().toString) while (getFile(blockId).exists()) { - blockId = new IntermediateBlockId(UUID.randomUUID().toString) + blockId = new TempBlockId(UUID.randomUUID().toString) } (blockId, getFile(blockId)) } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 492b4fc7c6..96f6bb3516 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -138,7 +138,7 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( spillCount += 1 logWarning(s"In-memory KV map exceeded threshold of $memoryThresholdMB MB!") logWarning(s"Spilling to disk ($spillCount time"+(if (spillCount > 1) "s" else "")+" so far)") - val (blockId, file) = diskBlockManager.createIntermediateBlock + val (blockId, file) = diskBlockManager.createTempBlock() val writer = new DiskBlockObjectWriter(blockId, file, serializer, fileBufferSize, identity) try { val it = currentMap.destructiveSortedIterator(comparator) diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala index e6b6103d96..204330dad4 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala @@ -96,6 +96,6 @@ private[spark] class SizeTrackingAppendOnlyMap[K, V] extends AppendOnlyMap[K, V] } } -object SizeTrackingAppendOnlyMap { +private object SizeTrackingAppendOnlyMap { case class Sample(size: Long, numUpdates: Long) } -- cgit v1.2.3 From 83dfa1666487a4772c95fea21fde0d47471e063d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 31 Dec 2013 20:02:05 -0800 Subject: Address Patrick's and Reynold's comments --- .../util/collection/ExternalAppendOnlyMap.scala | 120 ++++++++++++--------- 1 file changed, 71 insertions(+), 49 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 492b4fc7c6..311405f0cf 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -20,6 +20,8 @@ package org.apache.spark.util.collection import java.io._ import java.util.Comparator +import it.unimi.dsi.fastutil.io.FastBufferedInputStream + import scala.collection.mutable.{ArrayBuffer, PriorityQueue} import scala.reflect.ClassTag @@ -53,7 +55,7 @@ private[spark] class ExternalAppendOnlyMap[K, V, C: ClassTag]( private val map: SpillableAppendOnlyMap[K, V, _, C] = { if (mergeBeforeSpill) { new SpillableAppendOnlyMap[K, V, C, C] (createCombiner, mergeValue, mergeCombiners, - Predef.identity, serializer, diskBlockManager) + identity, serializer, diskBlockManager) } else { // Use ArrayBuffer[V] as the intermediate combiner val createGroup: (V => ArrayBuffer[V]) = value => ArrayBuffer[V](value) @@ -111,9 +113,10 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( import SpillableAppendOnlyMap._ private var currentMap = new SizeTrackingAppendOnlyMap[K, G] - private val oldMaps = new ArrayBuffer[DiskKGIterator] + private val spilledMaps = new ArrayBuffer[DiskIterator] private val memoryThresholdMB = { + // TODO: Turn this into a fraction of memory per reducer val bufferSize = System.getProperty("spark.shuffle.buffer.mb", "1024").toLong val bufferPercent = System.getProperty("spark.shuffle.buffer.fraction", "0.8").toFloat bufferSize * bufferPercent @@ -152,31 +155,37 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( writer.close() } currentMap = new SizeTrackingAppendOnlyMap[K, G] - oldMaps.append(new DiskKGIterator(file)) + spilledMaps.append(new DiskIterator(file)) } override def iterator: Iterator[(K, C)] = { - if (oldMaps.isEmpty && implicitly[ClassTag[G]] == implicitly[ClassTag[C]]) { + if (spilledMaps.isEmpty && implicitly[ClassTag[G]] == implicitly[ClassTag[C]]) { currentMap.iterator.asInstanceOf[Iterator[(K, C)]] } else { new ExternalIterator() } } - // An iterator that sort-merges (K, G) pairs from memory and disk into (K, C) pairs + /** An iterator that sort-merges (K, G) pairs from memory and disk into (K, C) pairs. */ private class ExternalIterator extends Iterator[(K, C)] { - val mergeHeap = new PriorityQueue[KGITuple] - val inputStreams = oldMaps ++ Seq(currentMap.destructiveSortedIterator(comparator)) - // Invariant: size of mergeHeap == number of input streams + // A fixed-size queue that maintains a buffer for each stream we are currently merging + val mergeHeap = new PriorityQueue[StreamBuffer] + + // Input streams are derived both from the in-memory map and spilled maps on disk + // The in-memory map is sorted in place, while the spilled maps are already in sorted order + val inputStreams = Seq(currentMap.destructiveSortedIterator(comparator)) ++ spilledMaps + inputStreams.foreach{ it => - val kgPairs = readFromIterator(it) - mergeHeap.enqueue(KGITuple(it, kgPairs)) + val kgPairs = getMorePairs(it) + mergeHeap.enqueue(StreamBuffer(it, kgPairs)) } - // Read from the given iterator until a key of different hash is retrieved. - // The resulting ArrayBuffer includes this key, and is ordered by key hash. - def readFromIterator(it: Iterator[(K, G)]): ArrayBuffer[(K, G)] = { + /** + * Fetch from the given iterator until a key of different hash is retrieved. In the + * event of key hash collisions, this ensures no pairs are hidden from being merged. + */ + def getMorePairs(it: Iterator[(K, G)]): ArrayBuffer[(K, G)] = { val kgPairs = new ArrayBuffer[(K, G)] if (it.hasNext) { var kg = it.next() @@ -190,20 +199,26 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( kgPairs } - // From the given KGITuple, remove the (K, G) pair with K = key and merge it into baseGroup - def mergeIntoGroup(key: K, baseGroup: G, kgi: KGITuple): G = { - kgi.pairs.zipWithIndex.foreach { case ((k, g), i) => + /** + * If the given buffer contains a value for the given key, merge that value into + * baseGroup and remove the corresponding (K, G) pair from the buffer + */ + def mergeIfKeyExists(key: K, baseGroup: G, buffer: StreamBuffer): G = { + var i = 0 + while (i < buffer.pairs.size) { + val (k, g) = buffer.pairs(i) if (k == key) { - kgi.pairs.remove(i) + buffer.pairs.remove(i) return mergeGroups(baseGroup, g) } + i += 1 } baseGroup } override def hasNext: Boolean = { - mergeHeap.foreach{ kgi => - if (!kgi.pairs.isEmpty) { + mergeHeap.foreach{ buffer => + if (!buffer.pairs.isEmpty) { return true } } @@ -211,66 +226,74 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( } override def next(): (K, C) = { - val minKGI = mergeHeap.dequeue() - val (minPairs, minHash) = (minKGI.pairs, minKGI.minHash) + // Select a return key from the StreamBuffer that holds the lowest key hash + val minBuffer = mergeHeap.dequeue() + val (minPairs, minHash) = (minBuffer.pairs, minBuffer.minKeyHash) if (minPairs.length == 0) { - // Should only happen when hasNext is false + // Should only happen when no other stream buffers have any pairs left throw new NoSuchElementException } - - // Select a return key with the minimum hash var (minKey, minGroup) = minPairs.remove(0) assert(minKey.hashCode() == minHash) - // Merge all other KGITuple's with the same minHash - val dequeuedKGI = ArrayBuffer[KGITuple](minKGI) - while (!mergeHeap.isEmpty && mergeHeap.head.minHash == minHash) { - val newKGI = mergeHeap.dequeue() - minGroup = mergeIntoGroup(minKey, minGroup, newKGI) - dequeuedKGI += newKGI + // For all other streams that may have this key (i.e. have the same minimum key hash), + // merge in the corresponding value (if any) from that stream + val mergedBuffers = ArrayBuffer[StreamBuffer](minBuffer) + while (!mergeHeap.isEmpty && mergeHeap.head.minKeyHash == minHash) { + val newBuffer = mergeHeap.dequeue() + minGroup = mergeIfKeyExists(minKey, minGroup, newBuffer) + mergedBuffers += newBuffer } - // Repopulate and add back all dequeued KGI to mergeHeap - dequeuedKGI.foreach { kgi => - if (kgi.pairs.length == 0) { - kgi.pairs ++= readFromIterator(kgi.iterator) + // Repopulate each visited stream buffer and add it back to the merge heap + mergedBuffers.foreach { buffer => + if (buffer.pairs.length == 0) { + buffer.pairs ++= getMorePairs(buffer.iterator) } - mergeHeap.enqueue(kgi) + mergeHeap.enqueue(buffer) } (minKey, createCombiner(minGroup)) } - case class KGITuple(iterator: Iterator[(K, G)], pairs: ArrayBuffer[(K, G)]) - extends Comparable[KGITuple] { - - // Invariant: pairs are ordered by key hash - def minHash: Int = { + /** + * A buffer for streaming from a map iterator (in-memory or on-disk) sorted by key hash. + * Each buffer maintains the lowest-ordered keys in the corresponding iterator. Due to + * hash collisions, it is possible for multiple keys to be "tied" for being the lowest. + * + * StreamBuffers are ordered by the minimum key hash found across all of their own pairs. + */ + case class StreamBuffer(iterator: Iterator[(K, G)], pairs: ArrayBuffer[(K, G)]) + extends Comparable[StreamBuffer] { + + def minKeyHash: Int = { if (pairs.length > 0){ + // pairs are already sorted by key hash pairs(0)._1.hashCode() } else { Int.MaxValue } } - override def compareTo(other: KGITuple): Int = { - // mutable.PriorityQueue dequeues the max, not the min - -minHash.compareTo(other.minHash) + override def compareTo(other: StreamBuffer): Int = { + // minus sign because mutable.PriorityQueue dequeues the max, not the min + -minKeyHash.compareTo(other.minKeyHash) } } } // Iterate through (K, G) pairs in sorted order from an on-disk map - private class DiskKGIterator(file: File) extends Iterator[(K, G)] { - val fstream = new FileInputStream(file) - val dstream = ser.deserializeStream(fstream) + private class DiskIterator(file: File) extends Iterator[(K, G)] { + val fileStream = new FileInputStream(file) + val bufferedStream = new FastBufferedInputStream(fileStream) + val deserializeStream = ser.deserializeStream(bufferedStream) var nextItem: Option[(K, G)] = None var eof = false def readNextItem(): Option[(K, G)] = { if (!eof) { try { - return Some(dstream.readObject().asInstanceOf[(K, G)]) + return Some(deserializeStream.readObject().asInstanceOf[(K, G)]) } catch { case e: EOFException => eof = true @@ -302,8 +325,7 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( // TODO: Ensure this gets called even if the iterator isn't drained. def cleanup() { - fstream.close() - dstream.close() + deserializeStream.close() file.delete() } } -- cgit v1.2.3 From 92c304fd0321d77941f0b029dc7b7f61804d8bca Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 1 Jan 2014 11:42:33 -0800 Subject: Simplify ExternalAppendOnlyMap on the assumption that the mergeCombiners function is specified --- .../main/scala/org/apache/spark/Aggregator.scala | 3 +- .../util/collection/ExternalAppendOnlyMap.scala | 162 +++++++-------------- .../collection/ExternalAppendOnlyMapSuite.scala | 23 --- 3 files changed, 53 insertions(+), 135 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index 784c09ec51..c408d5f145 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -71,8 +71,7 @@ case class Aggregator[K, V, C: ClassTag] ( } combiners.iterator } else { - val combiners = - new ExternalAppendOnlyMap[K, C, C](Predef.identity, mergeCombiners, mergeCombiners) + val combiners = new ExternalAppendOnlyMap[K, C, C](identity, mergeCombiners, mergeCombiners) while (iter.hasNext) { val kc = iter.next() combiners.insert(kc._1, kc._2) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 223fae128e..9e147feec4 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -23,98 +23,40 @@ import java.util.Comparator import it.unimi.dsi.fastutil.io.FastBufferedInputStream import scala.collection.mutable.{ArrayBuffer, PriorityQueue} -import scala.reflect.ClassTag import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{DiskBlockManager, DiskBlockObjectWriter} /** - * A wrapper for SpillableAppendOnlyMap that handles two cases: + * An append-only map that spills sorted content to disk when the memory threshold is exceeded. * - * (1) If a mergeCombiners function is specified, merge values into combiners before disk - * spill, as it is possible to merge the resulting combiners later. + * This map takes two passes over the data: + * (1) Values are merged into combiners, which are sorted and spilled to disk in as necessary. + * (2) Combiners are read from disk and merged together * - * (2) Otherwise, group values of the same key together before disk spill, and merge them - * into combiners only after reading them back from disk. + * Two parameters control the memory threshold: `spark.shuffle.buffer.mb` specifies the maximum + * size of the in-memory map before a spill, and `spark.shuffle.buffer.fraction` specifies an + * additional margin of safety. The second parameter is important for the following reason: * - * In the latter case, values occupy much more space because they are not collapsed as soon - * as they are inserted. This in turn leads to more disk spills, degrading performance. - * For this reason, a mergeCombiners function should be specified if possible. + * If the spill threshold is set too high, the in-memory map may occupy more memory than is + * available, resulting in OOM. However, if the spill threshold is set too low, we spill + * frequently and incur unnecessary disk writes. This may lead to a performance regression + * compared to the normal case of using the non-spilling AppendOnlyMap. */ -private[spark] class ExternalAppendOnlyMap[K, V, C: ClassTag]( + +private[spark] class ExternalAppendOnlyMap[K, V, C]( createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, serializer: Serializer = SparkEnv.get.serializerManager.default, diskBlockManager: DiskBlockManager = SparkEnv.get.blockManager.diskBlockManager) - extends Iterable[(K, C)] with Serializable { - - private val mergeBeforeSpill: Boolean = mergeCombiners != null - - private val map: SpillableAppendOnlyMap[K, V, _, C] = { - if (mergeBeforeSpill) { - new SpillableAppendOnlyMap[K, V, C, C] (createCombiner, mergeValue, mergeCombiners, - identity, serializer, diskBlockManager) - } else { - // Use ArrayBuffer[V] as the intermediate combiner - val createGroup: (V => ArrayBuffer[V]) = value => ArrayBuffer[V](value) - val mergeValueIntoGroup: (ArrayBuffer[V], V) => ArrayBuffer[V] = (group, value) => { - group += value - } - val mergeGroups: (ArrayBuffer[V], ArrayBuffer[V]) => ArrayBuffer[V] = (group1, group2) => { - group1 ++= group2 - } - val combineGroup: (ArrayBuffer[V] => C) = group => { - var combiner : Option[C] = None - group.foreach { v => - combiner match { - case None => combiner = Some(createCombiner(v)) - case Some(c) => combiner = Some(mergeValue(c, v)) - } - } - combiner.getOrElse(null.asInstanceOf[C]) - } - new SpillableAppendOnlyMap[K, V, ArrayBuffer[V], C](createGroup, mergeValueIntoGroup, - mergeGroups, combineGroup, serializer, diskBlockManager) - } - } - - def insert(key: K, value: V): Unit = map.insert(key, value) - - override def iterator: Iterator[(K, C)] = map.iterator -} - -/** - * An append-only map that spills sorted content to disk when the memory threshold is exceeded. - * A group is an intermediate combiner, with type G equal to either C or ArrayBuffer[V]. - * - * This map takes two passes over the data: - * (1) Values are merged into groups, which are spilled to disk as necessary. - * (2) Groups are read from disk and merged into combiners, which are returned. - * - * If we never spill to disk, we avoid the second pass provided that groups G are already - * combiners C. - * - * Note that OOM is still possible with the SpillableAppendOnlyMap. This may occur if the - * collective G values do not fit into memory, or if the size estimation is not sufficiently - * accurate. To account for the latter, `spark.shuffle.buffer.fraction` specifies an additional - * margin of safety, while `spark.shuffle.buffer.mb` specifies the raw memory threshold. - */ -private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( - createGroup: V => G, - mergeValue: (G, V) => G, - mergeGroups: (G, G) => G, - createCombiner: G => C, - serializer: Serializer, - diskBlockManager: DiskBlockManager) extends Iterable[(K, C)] with Serializable with Logging { - import SpillableAppendOnlyMap._ + import ExternalAppendOnlyMap._ - private var currentMap = new SizeTrackingAppendOnlyMap[K, G] + private var currentMap = new SizeTrackingAppendOnlyMap[K, C] private val spilledMaps = new ArrayBuffer[DiskIterator] - private val memoryThresholdMB = { // TODO: Turn this into a fraction of memory per reducer val bufferSize = System.getProperty("spark.shuffle.buffer.mb", "1024").toLong @@ -123,13 +65,13 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( } private val fileBufferSize = System.getProperty("spark.shuffle.file.buffer.kb", "100").toInt * 1024 - private val comparator = new KeyGroupComparator[K, G] + private val comparator = new KCComparator[K, C] private val ser = serializer.newInstance() private var spillCount = 0 def insert(key: K, value: V): Unit = { - val update: (Boolean, G) => G = (hadVal, oldVal) => { - if (hadVal) mergeValue(oldVal, value) else createGroup(value) + val update: (Boolean, C) => C = (hadVal, oldVal) => { + if (hadVal) mergeValue(oldVal, value) else createCombiner(value) } currentMap.changeValue(key, update) if (currentMap.estimateSize() > memoryThresholdMB * 1024 * 1024) { @@ -154,19 +96,19 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( // Partial failures cannot be tolerated; do not revert partial writes writer.close() } - currentMap = new SizeTrackingAppendOnlyMap[K, G] + currentMap = new SizeTrackingAppendOnlyMap[K, C] spilledMaps.append(new DiskIterator(file)) } override def iterator: Iterator[(K, C)] = { - if (spilledMaps.isEmpty && implicitly[ClassTag[G]] == implicitly[ClassTag[C]]) { - currentMap.iterator.asInstanceOf[Iterator[(K, C)]] + if (spilledMaps.isEmpty) { + currentMap.iterator } else { new ExternalIterator() } } - /** An iterator that sort-merges (K, G) pairs from memory and disk into (K, C) pairs. */ + /** An iterator that sort-merges (K, C) pairs from the in-memory and on-disk maps */ private class ExternalIterator extends Iterator[(K, C)] { // A fixed-size queue that maintains a buffer for each stream we are currently merging @@ -177,43 +119,43 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( val inputStreams = Seq(currentMap.destructiveSortedIterator(comparator)) ++ spilledMaps inputStreams.foreach{ it => - val kgPairs = getMorePairs(it) - mergeHeap.enqueue(StreamBuffer(it, kgPairs)) + val kcPairs = getMorePairs(it) + mergeHeap.enqueue(StreamBuffer(it, kcPairs)) } /** * Fetch from the given iterator until a key of different hash is retrieved. In the * event of key hash collisions, this ensures no pairs are hidden from being merged. */ - def getMorePairs(it: Iterator[(K, G)]): ArrayBuffer[(K, G)] = { - val kgPairs = new ArrayBuffer[(K, G)] + def getMorePairs(it: Iterator[(K, C)]): ArrayBuffer[(K, C)] = { + val kcPairs = new ArrayBuffer[(K, C)] if (it.hasNext) { - var kg = it.next() - kgPairs += kg - val minHash = kg._1.hashCode() - while (it.hasNext && kg._1.hashCode() == minHash) { - kg = it.next() - kgPairs += kg + var kc = it.next() + kcPairs += kc + val minHash = kc._1.hashCode() + while (it.hasNext && kc._1.hashCode() == minHash) { + kc = it.next() + kcPairs += kc } } - kgPairs + kcPairs } /** * If the given buffer contains a value for the given key, merge that value into - * baseGroup and remove the corresponding (K, G) pair from the buffer + * baseCombiner and remove the corresponding (K, C) pair from the buffer */ - def mergeIfKeyExists(key: K, baseGroup: G, buffer: StreamBuffer): G = { + def mergeIfKeyExists(key: K, baseCombiner: C, buffer: StreamBuffer): C = { var i = 0 while (i < buffer.pairs.size) { - val (k, g) = buffer.pairs(i) + val (k, c) = buffer.pairs(i) if (k == key) { buffer.pairs.remove(i) - return mergeGroups(baseGroup, g) + return mergeCombiners(baseCombiner, c) } i += 1 } - baseGroup + baseCombiner } override def hasNext: Boolean = { @@ -233,7 +175,7 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( // Should only happen when no other stream buffers have any pairs left throw new NoSuchElementException } - var (minKey, minGroup) = minPairs.remove(0) + var (minKey, minCombiner) = minPairs.remove(0) assert(minKey.hashCode() == minHash) // For all other streams that may have this key (i.e. have the same minimum key hash), @@ -241,7 +183,7 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( val mergedBuffers = ArrayBuffer[StreamBuffer](minBuffer) while (!mergeHeap.isEmpty && mergeHeap.head.minKeyHash == minHash) { val newBuffer = mergeHeap.dequeue() - minGroup = mergeIfKeyExists(minKey, minGroup, newBuffer) + minCombiner = mergeIfKeyExists(minKey, minCombiner, newBuffer) mergedBuffers += newBuffer } @@ -253,7 +195,7 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( mergeHeap.enqueue(buffer) } - (minKey, createCombiner(minGroup)) + (minKey, minCombiner) } /** @@ -263,7 +205,7 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( * * StreamBuffers are ordered by the minimum key hash found across all of their own pairs. */ - case class StreamBuffer(iterator: Iterator[(K, G)], pairs: ArrayBuffer[(K, G)]) + case class StreamBuffer(iterator: Iterator[(K, C)], pairs: ArrayBuffer[(K, C)]) extends Comparable[StreamBuffer] { def minKeyHash: Int = { @@ -282,18 +224,18 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( } } - // Iterate through (K, G) pairs in sorted order from an on-disk map - private class DiskIterator(file: File) extends Iterator[(K, G)] { + // Iterate through (K, C) pairs in sorted order from an on-disk map + private class DiskIterator(file: File) extends Iterator[(K, C)] { val fileStream = new FileInputStream(file) val bufferedStream = new FastBufferedInputStream(fileStream) val deserializeStream = ser.deserializeStream(bufferedStream) - var nextItem: Option[(K, G)] = None + var nextItem: Option[(K, C)] = None var eof = false - def readNextItem(): Option[(K, G)] = { + def readNextItem(): Option[(K, C)] = { if (!eof) { try { - return Some(deserializeStream.readObject().asInstanceOf[(K, G)]) + return Some(deserializeStream.readObject().asInstanceOf[(K, C)]) } catch { case e: EOFException => eof = true @@ -312,7 +254,7 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( } } - override def next(): (K, G) = { + override def next(): (K, C) = { nextItem match { case Some(item) => nextItem = None @@ -331,10 +273,10 @@ private[spark] class SpillableAppendOnlyMap[K, V, G: ClassTag, C: ClassTag]( } } -private[spark] object SpillableAppendOnlyMap { - private class KeyGroupComparator[K, G] extends Comparator[(K, G)] { - def compare(kg1: (K, G), kg2: (K, G)): Int = { - kg1._1.hashCode().compareTo(kg2._1.hashCode()) +private[spark] object ExternalAppendOnlyMap { + private class KCComparator[K, C] extends Comparator[(K, C)] { + def compare(kc1: (K, C), kc2: (K, C)): Int = { + kc1._1.hashCode().compareTo(kc2._1.hashCode()) } } } diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index baf94b4728..a18d466baa 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -229,27 +229,4 @@ class ExternalAppendOnlyMapSuite extends FunSuite with BeforeAndAfter with Local } } } - - test("spilling with no mergeCombiners function") { - System.setProperty("spark.shuffle.buffer.mb", "1") - System.setProperty("spark.shuffle.buffer.fraction", "0.05") - - // combineByKey - should spill exactly 11 times - val _createCombiner: Int => ArrayBuffer[Int] = i => ArrayBuffer[Int](i) - val _mergeValue: (ArrayBuffer[Int], Int) => ArrayBuffer[Int] = (buf, i) => buf += i - val rdd = sc.parallelize(0 until 10000).map(i => (i/4, i)) - val result = rdd.combineByKey[ArrayBuffer[Int]](_createCombiner, _mergeValue, null, - new HashPartitioner(1), mapSideCombine=false).collect() - - // result should be the same as groupByKey - assert(result.length == 2500) - result.foreach { case(i, seq) => - i match { - case 0 => assert(seq.toSet == Set[Int](0, 1, 2, 3)) - case 1250 => assert(seq.toSet == Set[Int](5000, 5001, 5002, 5003)) - case 2499 => assert(seq.toSet == Set[Int](9996, 9997, 9998, 9999)) - case _ => - } - } - } } -- cgit v1.2.3 From 8831923219faf1599957056dd5f406a22d1f1128 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 2 Jan 2014 13:52:35 -0800 Subject: TempBlockId takes UUID and is explicitly non-serializable --- core/src/main/scala/org/apache/spark/storage/BlockId.scala | 7 ++++--- .../src/main/scala/org/apache/spark/storage/DiskBlockManager.scala | 4 ++-- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index bcc3101485..301d784b35 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -17,6 +17,8 @@ package org.apache.spark.storage +import java.util.UUID + /** * Identifies a particular Block of data, usually associated with a single file. * A Block can be uniquely identified by its filename, but each type of Block has a different @@ -68,8 +70,8 @@ private[spark] case class StreamBlockId(streamId: Int, uniqueId: Long) extends B def name = "input-" + streamId + "-" + uniqueId } -/** Block associated with temporary data managed as blocks. */ -private[spark] case class TempBlockId(id: String) extends BlockId { +/** Id associated with temporary data managed as blocks. Not serializable. */ +private[spark] case class TempBlockId(id: UUID) extends BlockId { def name = "temp_" + id } @@ -85,7 +87,6 @@ private[spark] object BlockId { val BROADCAST_HELPER = "broadcast_([0-9]+)_([A-Za-z0-9]+)".r val TASKRESULT = "taskresult_([0-9]+)".r val STREAM = "input-([0-9]+)-([0-9]+)".r - val TEMP = "temp_(.*)".r val TEST = "test_(.*)".r /** Converts a BlockId "name" String back into a BlockId. */ diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 32da458a6f..e25bc90c4f 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -92,9 +92,9 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD /** Produces a unique block id and File suitable for intermediate results. */ def createTempBlock(): (TempBlockId, File) = { - var blockId = new TempBlockId(UUID.randomUUID().toString) + var blockId = new TempBlockId(UUID.randomUUID()) while (getFile(blockId).exists()) { - blockId = new TempBlockId(UUID.randomUUID().toString) + blockId = new TempBlockId(UUID.randomUUID()) } (blockId, getFile(blockId)) } -- cgit v1.2.3 From 838b0e7d154699291f9915d400c59a3580173d01 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 3 Jan 2014 16:13:40 -0800 Subject: Refactor using SparkConf --- core/src/main/scala/org/apache/spark/Aggregator.scala | 5 +++-- .../main/scala/org/apache/spark/rdd/CoGroupedRDD.scala | 5 +++-- .../spark/util/collection/ExternalAppendOnlyMap.scala | 15 +++++++++------ .../util/collection/ExternalAppendOnlyMapSuite.scala | 15 ++++++--------- 4 files changed, 21 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index c408d5f145..c9e3e8ec5a 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -33,8 +33,10 @@ case class Aggregator[K, V, C: ClassTag] ( mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) { + private val sparkConf = new SparkConf() + private val externalSorting = sparkConf.get("spark.shuffle.externalSorting", "false").toBoolean + def combineValuesByKey(iter: Iterator[_ <: Product2[K, V]]) : Iterator[(K, C)] = { - val externalSorting = System.getProperty("spark.shuffle.externalSorting", "false").toBoolean if (!externalSorting) { val combiners = new AppendOnlyMap[K,C] var kv: Product2[K, V] = null @@ -58,7 +60,6 @@ case class Aggregator[K, V, C: ClassTag] ( } def combineCombinersByKey(iter: Iterator[(K, C)]) : Iterator[(K, C)] = { - val externalSorting = System.getProperty("spark.shuffle.externalSorting", "false").toBoolean if (!externalSorting) { val combiners = new AppendOnlyMap[K,C] var kc: Product2[K, C] = null diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 44494c7e0a..7dc7094aac 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -22,7 +22,7 @@ import java.io.{ObjectOutputStream, IOException} import scala.collection.mutable.ArrayBuffer import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv, TaskContext} -import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency} +import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency, SparkConf} import org.apache.spark.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap} private[spark] sealed trait CoGroupSplitDep extends Serializable @@ -66,6 +66,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: private type CoGroupValue = (Any, Int) // Int is dependency number private type CoGroupCombiner = Seq[CoGroup] + private val sparkConf = new SparkConf() private var serializerClass: String = null def setSerializer(cls: String): CoGroupedRDD[K] = { @@ -106,7 +107,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: override def compute(s: Partition, context: TaskContext): Iterator[(K, CoGroupCombiner)] = { - val externalSorting = System.getProperty("spark.shuffle.externalSorting", "false").toBoolean + val externalSorting = sparkConf.get("spark.shuffle.externalSorting", "false").toBoolean val split = s.asInstanceOf[CoGroupPartition] val numRdds = split.deps.size diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 9e147feec4..68a23192c0 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -24,7 +24,7 @@ import it.unimi.dsi.fastutil.io.FastBufferedInputStream import scala.collection.mutable.{ArrayBuffer, PriorityQueue} -import org.apache.spark.{Logging, SparkEnv} +import org.apache.spark.{SparkConf, Logging, SparkEnv} import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{DiskBlockManager, DiskBlockObjectWriter} @@ -57,14 +57,16 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( private var currentMap = new SizeTrackingAppendOnlyMap[K, C] private val spilledMaps = new ArrayBuffer[DiskIterator] + + private val sparkConf = new SparkConf() private val memoryThresholdMB = { // TODO: Turn this into a fraction of memory per reducer - val bufferSize = System.getProperty("spark.shuffle.buffer.mb", "1024").toLong - val bufferPercent = System.getProperty("spark.shuffle.buffer.fraction", "0.8").toFloat + val bufferSize = sparkConf.getLong("spark.shuffle.buffer.mb", 1024) + val bufferPercent = sparkConf.getDouble("spark.shuffle.buffer.fraction", 0.8) bufferSize * bufferPercent } - private val fileBufferSize = - System.getProperty("spark.shuffle.file.buffer.kb", "100").toInt * 1024 + private val fileBufferSize = sparkConf.getInt("spark.shuffle.file.buffer.kb", 100) * 1024 + private val syncWrites = sparkConf.get("spark.shuffle.sync", "false").toBoolean private val comparator = new KCComparator[K, C] private val ser = serializer.newInstance() private var spillCount = 0 @@ -84,7 +86,8 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( logWarning(s"In-memory KV map exceeded threshold of $memoryThresholdMB MB!") logWarning(s"Spilling to disk ($spillCount time"+(if (spillCount > 1) "s" else "")+" so far)") val (blockId, file) = diskBlockManager.createTempBlock() - val writer = new DiskBlockObjectWriter(blockId, file, serializer, fileBufferSize, identity) + val writer = + new DiskBlockObjectWriter(blockId, file, serializer, fileBufferSize, identity, syncWrites) try { val it = currentMap.destructiveSortedIterator(comparator) while (it.hasNext) { diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index a18d466baa..6c93b1f5a0 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -4,20 +4,17 @@ import scala.collection.mutable.ArrayBuffer import org.scalatest.{BeforeAndAfter, FunSuite} -import org.apache.spark.{HashPartitioner, SparkContext, SparkEnv, LocalSparkContext} +import org.apache.spark._ import org.apache.spark.SparkContext.rddToPairRDDFunctions class ExternalAppendOnlyMapSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { override def beforeEach() { - sc = new SparkContext("local", "test") - System.setProperty("spark.shuffle.externalSorting", "true") - } - - after { - System.setProperty("spark.shuffle.externalSorting", "false") - System.setProperty("spark.shuffle.buffer.mb", "1024") - System.setProperty("spark.shuffle.buffer.fraction", "0.8") + val conf = new SparkConf(false) + conf.set("spark.shuffle.externalSorting", "true") + conf.set("spark.shuffle.buffer.mb", "1024") + conf.set("spark.shuffle.buffer.fraction", "0.8") + sc = new SparkContext("local", "test", conf) } val createCombiner: (Int => ArrayBuffer[Int]) = i => ArrayBuffer[Int](i) -- cgit v1.2.3 From 333d58df8676b30adc86e479579e2659e24d01a3 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 3 Jan 2014 17:55:26 -0800 Subject: Remove unnecessary ClassTag's --- core/src/main/scala/org/apache/spark/Aggregator.scala | 4 +--- core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 7 +++---- 2 files changed, 4 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index c9e3e8ec5a..bb488f4ad8 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -17,8 +17,6 @@ package org.apache.spark -import scala.reflect.ClassTag - import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap} /** @@ -28,7 +26,7 @@ import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap} * @param mergeValue function to merge a new value into the aggregation result. * @param mergeCombiners function to merge outputs from multiple mergeValue function. */ -case class Aggregator[K, V, C: ClassTag] ( +case class Aggregator[K, V, C] ( createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) { diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 7b1759ebbc..f8cd362795 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -72,7 +72,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * 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: ClassTag](createCombiner: V => C, + def combineByKey[C](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, partitioner: Partitioner, @@ -110,7 +110,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) /** * Simplified version of combineByKey that hash-partitions the output RDD. */ - def combineByKey[C: ClassTag](createCombiner: V => C, + def combineByKey[C](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, numPartitions: Int): RDD[(K, C)] = { @@ -338,8 +338,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * Simplified version of combineByKey that hash-partitions the resulting RDD using the * existing partitioner/parallelism level. */ - def combineByKey[C: ClassTag]( - createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) + def combineByKey[C](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) : RDD[(K, C)] = { combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(self)) } -- cgit v1.2.3 From 4296d96c82881cde5832bd8f8a3b48eb9817a218 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sat, 4 Jan 2014 00:00:57 -0800 Subject: Assign spill threshold as a fraction of maximum memory Further, divide this threshold by the number of tasks running concurrently. Note that this does not guard against the following scenario: a new task quickly fills up its share of the memory before old tasks finish spilling their contents, in which case the total memory used by such maps may exceed what was specified. Currently, spark.shuffle.safetyFraction mitigates the effect of this. --- .../src/main/scala/org/apache/spark/SparkEnv.scala | 16 +++++ .../scala/org/apache/spark/executor/Executor.scala | 2 + .../spark/util/collection/AppendOnlyMap.scala | 5 +- .../util/collection/ExternalAppendOnlyMap.scala | 74 +++++++++++++++------- .../collection/ExternalAppendOnlyMapSuite.scala | 17 ++--- 5 files changed, 81 insertions(+), 33 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 634a94f0a7..224b5c1744 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -59,6 +59,9 @@ class SparkEnv private[spark] ( private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]() + // Number of tasks currently running across all threads + @volatile private var _numRunningTasks = 0 + // A general, soft-reference map for metadata needed during HadoopRDD split computation // (e.g., HadoopFileRDD uses this to cache JobConfs and InputFormats). private[spark] val hadoopJobMetadata = new MapMaker().softValues().makeMap[String, Any]() @@ -86,6 +89,19 @@ class SparkEnv private[spark] ( pythonWorkers.getOrElseUpdate(key, new PythonWorkerFactory(pythonExec, envVars)).create() } } + + /** + * Return the number of tasks currently running across all threads + */ + def numRunningTasks: Int = _numRunningTasks + + def incrementNumRunningTasks() = synchronized { + _numRunningTasks += 1 + } + + def decrementNumRunningTasks() = synchronized { + _numRunningTasks -= 1 + } } object SparkEnv extends Logging { diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index e51d274d33..bd202affa2 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -186,6 +186,7 @@ private[spark] class Executor( var taskStart: Long = 0 def gcTime = ManagementFactory.getGarbageCollectorMXBeans.map(_.getCollectionTime).sum val startGCTime = gcTime + env.incrementNumRunningTasks() try { SparkEnv.set(env) @@ -279,6 +280,7 @@ private[spark] class Executor( //System.exit(1) } } finally { + env.decrementNumRunningTasks() runningTasks.remove(taskId) } } diff --git a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala index d2a9574a71..d8fa7ed9af 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala @@ -17,8 +17,7 @@ package org.apache.spark.util.collection -import java.util -import java.util.Comparator +import java.util.{Arrays, Comparator} /** * A simple open hash table optimized for the append-only use case, where keys @@ -270,7 +269,7 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi cmp.compare(x.asInstanceOf[(K, V)], y.asInstanceOf[(K, V)]) } } - util.Arrays.sort(data, 0, newIndex, rawOrdering) + Arrays.sort(data, 0, newIndex, rawOrdering) new Iterator[(K, V)] { var i = 0 diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 68a23192c0..c348168a8b 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -32,17 +32,28 @@ import org.apache.spark.storage.{DiskBlockManager, DiskBlockObjectWriter} * An append-only map that spills sorted content to disk when the memory threshold is exceeded. * * This map takes two passes over the data: - * (1) Values are merged into combiners, which are sorted and spilled to disk in as necessary. + * + * (1) Values are merged into combiners, which are sorted and spilled to disk as necessary * (2) Combiners are read from disk and merged together * - * Two parameters control the memory threshold: `spark.shuffle.buffer.mb` specifies the maximum - * size of the in-memory map before a spill, and `spark.shuffle.buffer.fraction` specifies an - * additional margin of safety. The second parameter is important for the following reason: + * The setting of the spill threshold faces the following trade-off: If the spill threshold is + * too high, the in-memory map may occupy more memory than is available, resulting in OOM. + * However, if the spill threshold is too low, we spill frequently and incur unnecessary disk + * writes. This may lead to a performance regression compared to the normal case of using the + * non-spilling AppendOnlyMap. + * + * A few parameters control the memory threshold: + * + * `spark.shuffle.memoryFraction` specifies the collective amount of memory used for storing + * these maps as a fraction of the executor's total memory. Since each concurrently running + * task maintains one map, the actual threshold for each map is this quantity divided by the + * number of running tasks. * - * If the spill threshold is set too high, the in-memory map may occupy more memory than is - * available, resulting in OOM. However, if the spill threshold is set too low, we spill - * frequently and incur unnecessary disk writes. This may lead to a performance regression - * compared to the normal case of using the non-spilling AppendOnlyMap. + * `spark.shuffle.safetyFraction` specifies an additional margin of safety as a fraction of + * this threshold, in case map size estimation is not sufficiently accurate. + * + * `spark.shuffle.updateThresholdInterval` controls how frequently each thread checks on + * shared executor state to update its local memory threshold. */ private[spark] class ExternalAppendOnlyMap[K, V, C]( @@ -56,35 +67,54 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( import ExternalAppendOnlyMap._ private var currentMap = new SizeTrackingAppendOnlyMap[K, C] - private val spilledMaps = new ArrayBuffer[DiskIterator] - + private val spilledMaps = new ArrayBuffer[DiskMapIterator] private val sparkConf = new SparkConf() - private val memoryThresholdMB = { - // TODO: Turn this into a fraction of memory per reducer - val bufferSize = sparkConf.getLong("spark.shuffle.buffer.mb", 1024) - val bufferPercent = sparkConf.getDouble("spark.shuffle.buffer.fraction", 0.8) - bufferSize * bufferPercent + + // Collective memory threshold shared across all running tasks + private val maxMemoryThreshold = { + val memoryFraction = sparkConf.getDouble("spark.shuffle.memoryFraction", 0.75) + val safetyFraction = sparkConf.getDouble("spark.shuffle.safetyFraction", 0.8) + (Runtime.getRuntime.maxMemory * memoryFraction * safetyFraction).toLong } + + // Maximum size for this map before a spill is triggered + private var spillThreshold = maxMemoryThreshold + + // How often to update spillThreshold + private val updateThresholdInterval = + sparkConf.getInt("spark.shuffle.updateThresholdInterval", 100) + private val fileBufferSize = sparkConf.getInt("spark.shuffle.file.buffer.kb", 100) * 1024 private val syncWrites = sparkConf.get("spark.shuffle.sync", "false").toBoolean private val comparator = new KCComparator[K, C] private val ser = serializer.newInstance() + private var insertCount = 0 private var spillCount = 0 - def insert(key: K, value: V): Unit = { + def insert(key: K, value: V) { + insertCount += 1 val update: (Boolean, C) => C = (hadVal, oldVal) => { if (hadVal) mergeValue(oldVal, value) else createCombiner(value) } currentMap.changeValue(key, update) - if (currentMap.estimateSize() > memoryThresholdMB * 1024 * 1024) { + if (insertCount % updateThresholdInterval == 1) { + updateSpillThreshold() + } + if (currentMap.estimateSize() > spillThreshold) { spill() } } - private def spill(): Unit = { + // TODO: differentiate ShuffleMapTask's from ResultTask's + private def updateSpillThreshold() { + val numRunningTasks = math.max(SparkEnv.get.numRunningTasks, 1) + spillThreshold = maxMemoryThreshold / numRunningTasks + } + + private def spill() { spillCount += 1 - logWarning(s"In-memory KV map exceeded threshold of $memoryThresholdMB MB!") - logWarning(s"Spilling to disk ($spillCount time"+(if (spillCount > 1) "s" else "")+" so far)") + logWarning("In-memory map exceeded %s MB! Spilling to disk (%d time%s so far)" + .format(spillThreshold / (1024 * 1024), spillCount, if (spillCount > 1) "s" else "")) val (blockId, file) = diskBlockManager.createTempBlock() val writer = new DiskBlockObjectWriter(blockId, file, serializer, fileBufferSize, identity, syncWrites) @@ -100,7 +130,7 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( writer.close() } currentMap = new SizeTrackingAppendOnlyMap[K, C] - spilledMaps.append(new DiskIterator(file)) + spilledMaps.append(new DiskMapIterator(file)) } override def iterator: Iterator[(K, C)] = { @@ -228,7 +258,7 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( } // Iterate through (K, C) pairs in sorted order from an on-disk map - private class DiskIterator(file: File) extends Iterator[(K, C)] { + private class DiskMapIterator(file: File) extends Iterator[(K, C)] { val fileStream = new FileInputStream(file) val bufferedStream = new FastBufferedInputStream(fileStream) val deserializeStream = ser.deserializeStream(bufferedStream) diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index 6c93b1f5a0..ef957bb0e5 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -5,15 +5,13 @@ import scala.collection.mutable.ArrayBuffer import org.scalatest.{BeforeAndAfter, FunSuite} import org.apache.spark._ -import org.apache.spark.SparkContext.rddToPairRDDFunctions +import org.apache.spark.SparkContext._ class ExternalAppendOnlyMapSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { override def beforeEach() { val conf = new SparkConf(false) conf.set("spark.shuffle.externalSorting", "true") - conf.set("spark.shuffle.buffer.mb", "1024") - conf.set("spark.shuffle.buffer.fraction", "0.8") sc = new SparkContext("local", "test", conf) } @@ -27,14 +25,14 @@ class ExternalAppendOnlyMapSuite extends FunSuite with BeforeAndAfter with Local } test("simple insert") { - var map = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]](createCombiner, + val map = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]](createCombiner, mergeValue, mergeCombiners) // Single insert map.insert(1, 10) var it = map.iterator assert(it.hasNext) - var kv = it.next() + val kv = it.next() assert(kv._1 == 1 && kv._2 == ArrayBuffer[Int](10)) assert(!it.hasNext) @@ -59,7 +57,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with BeforeAndAfter with Local map.insert(1, 100) map.insert(2, 200) map.insert(1, 1000) - var it = map.iterator + val it = map.iterator assert(it.hasNext) val result = it.toSet[(Int, ArrayBuffer[Int])].map(kv => (kv._1, kv._2.toSet)) assert(result == Set[(Int, Set[Int])]( @@ -177,8 +175,9 @@ class ExternalAppendOnlyMapSuite extends FunSuite with BeforeAndAfter with Local } test("spilling") { - System.setProperty("spark.shuffle.buffer.mb", "1") - System.setProperty("spark.shuffle.buffer.fraction", "0.05") + // TODO: Figure out correct memory parameters to actually induce spilling + // System.setProperty("spark.shuffle.buffer.mb", "1") + // System.setProperty("spark.shuffle.buffer.fraction", "0.05") // reduceByKey - should spill exactly 6 times val rddA = sc.parallelize(0 until 10000).map(i => (i/2, i)) @@ -226,4 +225,6 @@ class ExternalAppendOnlyMapSuite extends FunSuite with BeforeAndAfter with Local } } } + + // TODO: Test memory allocation for multiple concurrently running tasks } -- cgit v1.2.3 From 2db7884f6f1939d2a62fb71279a3ad80706308e1 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sat, 4 Jan 2014 01:20:09 -0800 Subject: Address Mark's comments --- core/src/main/scala/org/apache/spark/Aggregator.scala | 8 ++++---- .../org/apache/spark/util/collection/AppendOnlyMap.scala | 15 +++++---------- .../apache/spark/util/collection/AppendOnlyMapSuite.scala | 8 ++++---- 3 files changed, 13 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index bb488f4ad8..292e32e7c8 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -50,8 +50,8 @@ case class Aggregator[K, V, C] ( val combiners = new ExternalAppendOnlyMap[K, V, C](createCombiner, mergeValue, mergeCombiners) while (iter.hasNext) { - val kv = iter.next() - combiners.insert(kv._1, kv._2) + val (k, v) = iter.next() + combiners.insert(k, v) } combiners.iterator } @@ -72,8 +72,8 @@ case class Aggregator[K, V, C] ( } else { val combiners = new ExternalAppendOnlyMap[K, C, C](identity, mergeCombiners, mergeCombiners) while (iter.hasNext) { - val kc = iter.next() - combiners.insert(kc._1, kc._2) + val (k, c) = iter.next() + combiners.insert(k, c) } combiners.iterator } diff --git a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala index d8fa7ed9af..6faaa3197f 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala @@ -49,12 +49,13 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi // Triggered by destructiveSortedIterator; the underlying data array may no longer be used private var destroyed = false + private val destructionMessage = "Map state is invalid from destructive sorting!" private val LOAD_FACTOR = 0.7 /** Get the value for a given key */ def apply(key: K): V = { - checkValidityOrThrowException() + assert(!destroyed, destructionMessage) val k = key.asInstanceOf[AnyRef] if (k.eq(null)) { return nullValue @@ -78,7 +79,7 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi /** Set the value for a key */ def update(key: K, value: V): Unit = { - checkValidityOrThrowException() + assert(!destroyed, destructionMessage) val k = key.asInstanceOf[AnyRef] if (k.eq(null)) { if (!haveNullValue) { @@ -113,7 +114,7 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi * for key, if any, or null otherwise. Returns the newly updated value. */ def changeValue(key: K, updateFunc: (Boolean, V) => V): V = { - checkValidityOrThrowException() + assert(!destroyed, destructionMessage) val k = key.asInstanceOf[AnyRef] if (k.eq(null)) { if (!haveNullValue) { @@ -148,7 +149,7 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi /** Iterator method from Iterable */ override def iterator: Iterator[(K, V)] = { - checkValidityOrThrowException() + assert(!destroyed, destructionMessage) new Iterator[(K, V)] { var pos = -1 @@ -287,10 +288,4 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi } } } - - private def checkValidityOrThrowException(): Unit = { - if (destroyed) { - throw new IllegalStateException("Map state is invalid from destructive sorting!") - } - } } diff --git a/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala index 71b936b0df..f44442f1a5 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala @@ -190,9 +190,9 @@ class AppendOnlyMapSuite extends FunSuite { } // All subsequent calls to apply, update, changeValue and iterator should throw exception - intercept[IllegalStateException] { map.apply("1") } - intercept[IllegalStateException] { map.update("1", "2013") } - intercept[IllegalStateException] { map.changeValue("1", (hadValue, oldValue) => "2014") } - intercept[IllegalStateException] { map.iterator } + intercept[AssertionError] { map.apply("1") } + intercept[AssertionError] { map.update("1", "2013") } + intercept[AssertionError] { map.changeValue("1", (hadValue, oldValue) => "2014") } + intercept[AssertionError] { map.iterator } } } -- cgit v1.2.3 From 4de9c9554ca6464b806496dbffe0ba99c0ae6b45 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sat, 4 Jan 2014 11:16:30 -0800 Subject: Use AtomicInteger for numRunningTasks --- core/src/main/scala/org/apache/spark/SparkEnv.scala | 19 +++++++------------ 1 file changed, 7 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 224b5c1744..b581c7b074 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -17,8 +17,9 @@ package org.apache.spark -import collection.mutable -import serializer.Serializer +import java.util.concurrent.atomic.AtomicInteger + +import scala.collection.mutable import akka.actor._ import akka.remote.RemoteActorRefProvider @@ -60,7 +61,7 @@ class SparkEnv private[spark] ( private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]() // Number of tasks currently running across all threads - @volatile private var _numRunningTasks = 0 + private val _numRunningTasks = new AtomicInteger(0) // A general, soft-reference map for metadata needed during HadoopRDD split computation // (e.g., HadoopFileRDD uses this to cache JobConfs and InputFormats). @@ -93,15 +94,9 @@ class SparkEnv private[spark] ( /** * Return the number of tasks currently running across all threads */ - def numRunningTasks: Int = _numRunningTasks - - def incrementNumRunningTasks() = synchronized { - _numRunningTasks += 1 - } - - def decrementNumRunningTasks() = synchronized { - _numRunningTasks -= 1 - } + def numRunningTasks: Int = _numRunningTasks.intValue() + def incrementNumRunningTasks(): Int = _numRunningTasks.incrementAndGet() + def decrementNumRunningTasks(): Int = _numRunningTasks.decrementAndGet() } object SparkEnv extends Logging { -- cgit v1.2.3 From 80ba9f8ba06e623600469ddb3e59dffcbedee1d0 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 7 Jan 2014 12:44:22 -0800 Subject: Get SparkConf from SparkEnv, rather than creating new ones --- core/src/main/scala/org/apache/spark/Aggregator.scala | 2 +- core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala | 6 +++--- .../org/apache/spark/util/collection/ExternalAppendOnlyMap.scala | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index 292e32e7c8..08a96b0c34 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -31,7 +31,7 @@ case class Aggregator[K, V, C] ( mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) { - private val sparkConf = new SparkConf() + private val sparkConf = SparkEnv.get.conf private val externalSorting = sparkConf.get("spark.shuffle.externalSorting", "false").toBoolean def combineValuesByKey(iter: Iterator[_ <: Product2[K, V]]) : Iterator[(K, C)] = { diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 7dc7094aac..b7c7773e58 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -22,7 +22,7 @@ import java.io.{ObjectOutputStream, IOException} import scala.collection.mutable.ArrayBuffer import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv, TaskContext} -import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency, SparkConf} +import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency} import org.apache.spark.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap} private[spark] sealed trait CoGroupSplitDep extends Serializable @@ -66,7 +66,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: private type CoGroupValue = (Any, Int) // Int is dependency number private type CoGroupCombiner = Seq[CoGroup] - private val sparkConf = new SparkConf() + private val sparkConf = SparkEnv.get.conf private var serializerClass: String = null def setSerializer(cls: String): CoGroupedRDD[K] = { @@ -122,7 +122,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: case ShuffleCoGroupSplitDep(shuffleId) => { // Read map outputs of shuffle val fetcher = SparkEnv.get.shuffleFetcher - val ser = SparkEnv.get.serializerManager.get(serializerClass, SparkEnv.get.conf) + val ser = SparkEnv.get.serializerManager.get(serializerClass, sparkConf) val it = fetcher.fetch[Product2[K, Any]](shuffleId, split.index, context, ser) rddIterators += ((it, depNum)) } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index c348168a8b..a5897e8066 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -24,7 +24,7 @@ import it.unimi.dsi.fastutil.io.FastBufferedInputStream import scala.collection.mutable.{ArrayBuffer, PriorityQueue} -import org.apache.spark.{SparkConf, Logging, SparkEnv} +import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{DiskBlockManager, DiskBlockObjectWriter} @@ -68,7 +68,7 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( private var currentMap = new SizeTrackingAppendOnlyMap[K, C] private val spilledMaps = new ArrayBuffer[DiskMapIterator] - private val sparkConf = new SparkConf() + private val sparkConf = SparkEnv.get.conf // Collective memory threshold shared across all running tasks private val maxMemoryThreshold = { -- cgit v1.2.3 From aa5002bb9682ae6271db1fedb2c2658ed04fd4a1 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 9 Jan 2014 21:43:58 -0800 Subject: Defensively allocate memory from global pool This is an alternative to the existing approach, which evenly distributes the collective shuffle memory among all running tasks. In the new approach, each thread requests a chunk of memory whenever its map is about to multiplicatively grow. If there is sufficient memory in the global pool, the thread allocates it and grows its map. Otherwise, it spills. A danger with the previous approach is that a new task may quickly fill up its map before old tasks finish spilling, potentially causing an OOM. This approach prevents this scenario as it favors existing tasks over new tasks; any thread that may step over the boundary of other threads defensively backs off and starts spilling. Testing through spark-perf reveals: (1) When no spills have occured, the performance of external sorting using this memory management approach is essentially the same as without external sorting. (2) When one or more spills have occured, the performance of external sorting is a small multiple (3x) worse --- .../src/main/scala/org/apache/spark/SparkEnv.scala | 18 ++--- .../scala/org/apache/spark/executor/Executor.scala | 7 +- .../apache/spark/storage/BlockObjectWriter.scala | 4 + .../spark/util/collection/AppendOnlyMap.scala | 12 ++- .../util/collection/ExternalAppendOnlyMap.scala | 86 ++++++++++++++-------- 5 files changed, 80 insertions(+), 47 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 584261df04..08b592df71 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -17,8 +17,6 @@ package org.apache.spark -import java.util.concurrent.atomic.AtomicInteger - import scala.collection.mutable import scala.concurrent.Await @@ -56,12 +54,13 @@ class SparkEnv private[spark] ( val httpFileServer: HttpFileServer, val sparkFilesDir: String, val metricsSystem: MetricsSystem, - val conf: SparkConf) { + val conf: SparkConf) extends Logging { - private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]() + // A mapping of thread ID to amount of memory used for shuffle in bytes + // All accesses should be manually synchronized + val shuffleMemoryMap = mutable.HashMap[Long, Long]() - // Number of tasks currently running across all threads - private val _numRunningTasks = new AtomicInteger(0) + private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]() // A general, soft-reference map for metadata needed during HadoopRDD split computation // (e.g., HadoopFileRDD uses this to cache JobConfs and InputFormats). @@ -90,13 +89,6 @@ class SparkEnv private[spark] ( pythonWorkers.getOrElseUpdate(key, new PythonWorkerFactory(pythonExec, envVars)).create() } } - - /** - * Return the number of tasks currently running across all threads - */ - def numRunningTasks: Int = _numRunningTasks.intValue() - def incrementNumRunningTasks(): Int = _numRunningTasks.incrementAndGet() - def decrementNumRunningTasks(): Int = _numRunningTasks.decrementAndGet() } object SparkEnv extends Logging { diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index bd202affa2..a7b2328a02 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -186,7 +186,6 @@ private[spark] class Executor( var taskStart: Long = 0 def gcTime = ManagementFactory.getGarbageCollectorMXBeans.map(_.getCollectionTime).sum val startGCTime = gcTime - env.incrementNumRunningTasks() try { SparkEnv.set(env) @@ -280,7 +279,11 @@ private[spark] class Executor( //System.exit(1) } } finally { - env.decrementNumRunningTasks() + // TODO: Unregister shuffle memory only for ShuffleMapTask + val shuffleMemoryMap = env.shuffleMemoryMap + shuffleMemoryMap.synchronized { + shuffleMemoryMap.remove(Thread.currentThread().getId) + } runningTasks.remove(taskId) } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala index 61e63c60d5..369a277232 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala @@ -181,4 +181,8 @@ class DiskBlockObjectWriter( // Only valid if called after close() override def timeWriting() = _timeWriting + + def bytesWritten: Long = { + lastValidPosition - initialPosition + } } diff --git a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala index 6faaa3197f..d98c7aa3d7 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala @@ -30,14 +30,15 @@ import java.util.{Arrays, Comparator} * TODO: Cache the hash values of each key? java.util.HashMap does that. */ private[spark] -class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] with Serializable { +class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, + V)] with Serializable { require(initialCapacity <= (1 << 29), "Can't make capacity bigger than 2^29 elements") require(initialCapacity >= 1, "Invalid initial capacity") private var capacity = nextPowerOf2(initialCapacity) private var mask = capacity - 1 private var curSize = 0 - private var growThreshold = LOAD_FACTOR * capacity + private var growThreshold = (LOAD_FACTOR * capacity).toInt // Holds keys and values in the same array for memory locality; specifically, the order of // elements is key0, value0, key1, value1, key2, value2, etc. @@ -239,7 +240,7 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi data = newData capacity = newCapacity mask = newMask - growThreshold = LOAD_FACTOR * newCapacity + growThreshold = (LOAD_FACTOR * newCapacity).toInt } private def nextPowerOf2(n: Int): Int = { @@ -288,4 +289,9 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi } } } + + /** + * Return whether the next insert will cause the map to grow + */ + def atGrowThreshold: Boolean = curSize == growThreshold } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index a5897e8066..50f05351eb 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -22,14 +22,16 @@ import java.util.Comparator import it.unimi.dsi.fastutil.io.FastBufferedInputStream -import scala.collection.mutable.{ArrayBuffer, PriorityQueue} +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{DiskBlockManager, DiskBlockObjectWriter} /** - * An append-only map that spills sorted content to disk when the memory threshold is exceeded. + * An append-only map that spills sorted content to disk when there is insufficient space for it + * to grow. * * This map takes two passes over the data: * @@ -42,7 +44,7 @@ import org.apache.spark.storage.{DiskBlockManager, DiskBlockObjectWriter} * writes. This may lead to a performance regression compared to the normal case of using the * non-spilling AppendOnlyMap. * - * A few parameters control the memory threshold: + * Two parameters control the memory threshold: * * `spark.shuffle.memoryFraction` specifies the collective amount of memory used for storing * these maps as a fraction of the executor's total memory. Since each concurrently running @@ -51,9 +53,6 @@ import org.apache.spark.storage.{DiskBlockManager, DiskBlockObjectWriter} * * `spark.shuffle.safetyFraction` specifies an additional margin of safety as a fraction of * this threshold, in case map size estimation is not sufficiently accurate. - * - * `spark.shuffle.updateThresholdInterval` controls how frequently each thread checks on - * shared executor state to update its local memory threshold. */ private[spark] class ExternalAppendOnlyMap[K, V, C]( @@ -77,12 +76,9 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( (Runtime.getRuntime.maxMemory * memoryFraction * safetyFraction).toLong } - // Maximum size for this map before a spill is triggered - private var spillThreshold = maxMemoryThreshold - - // How often to update spillThreshold - private val updateThresholdInterval = - sparkConf.getInt("spark.shuffle.updateThresholdInterval", 100) + // How many inserts into this map before tracking its shuffle memory usage + private val initialInsertThreshold = + sparkConf.getLong("spark.shuffle.initialInsertThreshold", 1000) private val fileBufferSize = sparkConf.getInt("spark.shuffle.file.buffer.kb", 100) * 1024 private val syncWrites = sparkConf.get("spark.shuffle.sync", "false").toBoolean @@ -91,30 +87,54 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( private var insertCount = 0 private var spillCount = 0 + /** + * Insert the given key and value into the map. + * + * If the underlying map is about to grow, check if the global pool of shuffle memory has + * enough room for this to happen. If so, allocate the memory required to grow the map; + * otherwise, spill the in-memory map to disk. + * + * The shuffle memory usage of the first initialInsertThreshold entries is not tracked. + */ def insert(key: K, value: V) { insertCount += 1 val update: (Boolean, C) => C = (hadVal, oldVal) => { if (hadVal) mergeValue(oldVal, value) else createCombiner(value) } - currentMap.changeValue(key, update) - if (insertCount % updateThresholdInterval == 1) { - updateSpillThreshold() - } - if (currentMap.estimateSize() > spillThreshold) { - spill() + if (insertCount > initialInsertThreshold && currentMap.atGrowThreshold) { + val mapSize = currentMap.estimateSize() + var shouldSpill = false + val shuffleMemoryMap = SparkEnv.get.shuffleMemoryMap + + // Atomically check whether there is sufficient memory in the global pool for + // this map to grow and, if possible, allocate the required amount + shuffleMemoryMap.synchronized { + val threadId = Thread.currentThread().getId + val previouslyOccupiedMemory = shuffleMemoryMap.get(threadId) + val availableMemory = maxMemoryThreshold - + (shuffleMemoryMap.values.sum - previouslyOccupiedMemory.getOrElse(0L)) + + // Assume map grow factor is 2x + shouldSpill = availableMemory < mapSize * 2 + if (!shouldSpill) { + shuffleMemoryMap(threadId) = mapSize * 2 + } + } + // Do not synchronize spills + if (shouldSpill) { + spill(mapSize) + } } + currentMap.changeValue(key, update) } - // TODO: differentiate ShuffleMapTask's from ResultTask's - private def updateSpillThreshold() { - val numRunningTasks = math.max(SparkEnv.get.numRunningTasks, 1) - spillThreshold = maxMemoryThreshold / numRunningTasks - } - - private def spill() { + /** + * Sort the existing contents of the in-memory map and spill them to a temporary file on disk + */ + private def spill(mapSize: Long) { spillCount += 1 - logWarning("In-memory map exceeded %s MB! Spilling to disk (%d time%s so far)" - .format(spillThreshold / (1024 * 1024), spillCount, if (spillCount > 1) "s" else "")) + logWarning("* Spilling in-memory map of %d MB to disk (%d time%s so far)" + .format(mapSize / (1024 * 1024), spillCount, if (spillCount > 1) "s" else "")) val (blockId, file) = diskBlockManager.createTempBlock() val writer = new DiskBlockObjectWriter(blockId, file, serializer, fileBufferSize, identity, syncWrites) @@ -131,6 +151,13 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( } currentMap = new SizeTrackingAppendOnlyMap[K, C] spilledMaps.append(new DiskMapIterator(file)) + + // Reset the amount of shuffle memory used by this map in the global pool + val shuffleMemoryMap = SparkEnv.get.shuffleMemoryMap + shuffleMemoryMap.synchronized { + shuffleMemoryMap(Thread.currentThread().getId) = 0 + } + insertCount = 0 } override def iterator: Iterator[(K, C)] = { @@ -145,11 +172,12 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( private class ExternalIterator extends Iterator[(K, C)] { // A fixed-size queue that maintains a buffer for each stream we are currently merging - val mergeHeap = new PriorityQueue[StreamBuffer] + val mergeHeap = new mutable.PriorityQueue[StreamBuffer] // Input streams are derived both from the in-memory map and spilled maps on disk // The in-memory map is sorted in place, while the spilled maps are already in sorted order - val inputStreams = Seq(currentMap.destructiveSortedIterator(comparator)) ++ spilledMaps + val sortedMap = currentMap.destructiveSortedIterator(comparator) + val inputStreams = Seq(sortedMap) ++ spilledMaps inputStreams.foreach{ it => val kcPairs = getMorePairs(it) -- cgit v1.2.3 From 372a533a6c091361115f0f0712e93ef3af376b30 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 9 Jan 2014 21:47:49 -0800 Subject: Fix wonky imports from merge --- .../src/test/java/org/apache/spark/streaming/JavaAPISuite.java | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index be93799a2a..8b7d7709bf 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -17,14 +17,6 @@ package org.apache.spark.streaming; -import com.google.common.base.Optional; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.collect.Sets; -import com.google.common.io.Files; - -import kafka.serializer.StringDecoder; - import scala.Tuple2; import org.junit.After; @@ -36,6 +28,7 @@ import java.util.*; import com.google.common.base.Optional; import com.google.common.collect.Lists; import com.google.common.io.Files; +import com.google.common.collect.Sets; import org.apache.spark.SparkConf; import org.apache.spark.HashPartitioner; -- cgit v1.2.3 From e4c51d21135978908f7f4a46683f70ef98b720ec Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 10 Jan 2014 15:09:51 -0800 Subject: Address Patrick's and Reynold's comments Aside from trivial formatting changes, use nulls instead of Options for DiskMapIterator, and add documentation for spark.shuffle.externalSorting and spark.shuffle.memoryFraction. Also, set spark.shuffle.memoryFraction to 0.3, and spark.storage.memoryFraction = 0.6. --- .../main/scala/org/apache/spark/Aggregator.scala | 2 +- .../scala/org/apache/spark/rdd/CoGroupedRDD.scala | 3 +- .../org/apache/spark/storage/BlockManager.scala | 2 +- .../util/collection/ExternalAppendOnlyMap.scala | 89 ++++++++++++---------- docs/configuration.md | 24 +++++- 5 files changed, 73 insertions(+), 47 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index 08a96b0c34..8b30cd4bfe 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -32,7 +32,7 @@ case class Aggregator[K, V, C] ( mergeCombiners: (C, C) => C) { private val sparkConf = SparkEnv.get.conf - private val externalSorting = sparkConf.get("spark.shuffle.externalSorting", "false").toBoolean + private val externalSorting = sparkConf.getBoolean("spark.shuffle.externalSorting", true) def combineValuesByKey(iter: Iterator[_ <: Product2[K, V]]) : Iterator[(K, C)] = { if (!externalSorting) { diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index b7c7773e58..a73714abca 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -106,8 +106,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: override val partitioner = Some(part) override def compute(s: Partition, context: TaskContext): Iterator[(K, CoGroupCombiner)] = { - - val externalSorting = sparkConf.get("spark.shuffle.externalSorting", "false").toBoolean + val externalSorting = sparkConf.getBoolean("spark.shuffle.externalSorting", true) val split = s.asInstanceOf[CoGroupPartition] val numRdds = split.deps.size diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index c56e2ca2df..56cae6f6b9 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -864,7 +864,7 @@ private[spark] object BlockManager extends Logging { val ID_GENERATOR = new IdGenerator def getMaxMemory(conf: SparkConf): Long = { - val memoryFraction = conf.getDouble("spark.storage.memoryFraction", 0.66) + val memoryFraction = conf.getDouble("spark.storage.memoryFraction", 0.6) (Runtime.getRuntime.maxMemory * memoryFraction).toLong } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 50f05351eb..e3bcd895aa 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -71,21 +71,24 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( // Collective memory threshold shared across all running tasks private val maxMemoryThreshold = { - val memoryFraction = sparkConf.getDouble("spark.shuffle.memoryFraction", 0.75) + val memoryFraction = sparkConf.getDouble("spark.shuffle.memoryFraction", 0.3) val safetyFraction = sparkConf.getDouble("spark.shuffle.safetyFraction", 0.8) (Runtime.getRuntime.maxMemory * memoryFraction * safetyFraction).toLong } - // How many inserts into this map before tracking its shuffle memory usage - private val initialInsertThreshold = - sparkConf.getLong("spark.shuffle.initialInsertThreshold", 1000) + // Number of pairs in the in-memory map + private var numPairsInMemory = 0 + + // Number of in-memory pairs inserted before tracking the map's shuffle memory usage + private val trackMemoryThreshold = 1000 + + // How many times we have spilled so far + private var spillCount = 0 private val fileBufferSize = sparkConf.getInt("spark.shuffle.file.buffer.kb", 100) * 1024 - private val syncWrites = sparkConf.get("spark.shuffle.sync", "false").toBoolean + private val syncWrites = sparkConf.getBoolean("spark.shuffle.sync", false) private val comparator = new KCComparator[K, C] private val ser = serializer.newInstance() - private var insertCount = 0 - private var spillCount = 0 /** * Insert the given key and value into the map. @@ -94,14 +97,13 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( * enough room for this to happen. If so, allocate the memory required to grow the map; * otherwise, spill the in-memory map to disk. * - * The shuffle memory usage of the first initialInsertThreshold entries is not tracked. + * The shuffle memory usage of the first trackMemoryThreshold entries is not tracked. */ def insert(key: K, value: V) { - insertCount += 1 val update: (Boolean, C) => C = (hadVal, oldVal) => { if (hadVal) mergeValue(oldVal, value) else createCombiner(value) } - if (insertCount > initialInsertThreshold && currentMap.atGrowThreshold) { + if (numPairsInMemory > trackMemoryThreshold && currentMap.atGrowThreshold) { val mapSize = currentMap.estimateSize() var shouldSpill = false val shuffleMemoryMap = SparkEnv.get.shuffleMemoryMap @@ -114,7 +116,7 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( val availableMemory = maxMemoryThreshold - (shuffleMemoryMap.values.sum - previouslyOccupiedMemory.getOrElse(0L)) - // Assume map grow factor is 2x + // Assume map growth factor is 2x shouldSpill = availableMemory < mapSize * 2 if (!shouldSpill) { shuffleMemoryMap(threadId) = mapSize * 2 @@ -126,6 +128,7 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( } } currentMap.changeValue(key, update) + numPairsInMemory += 1 } /** @@ -133,7 +136,7 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( */ private def spill(mapSize: Long) { spillCount += 1 - logWarning("* Spilling in-memory map of %d MB to disk (%d time%s so far)" + logWarning("Spilling in-memory map of %d MB to disk (%d time%s so far)" .format(mapSize / (1024 * 1024), spillCount, if (spillCount > 1) "s" else "")) val (blockId, file) = diskBlockManager.createTempBlock() val writer = @@ -157,9 +160,13 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( shuffleMemoryMap.synchronized { shuffleMemoryMap(Thread.currentThread().getId) = 0 } - insertCount = 0 + numPairsInMemory = 0 } + /** + * Return an iterator that merges the in-memory map with the spilled maps. + * If no spill has occurred, simply return the in-memory map's iterator. + */ override def iterator: Iterator[(K, C)] = { if (spilledMaps.isEmpty) { currentMap.iterator @@ -168,7 +175,9 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( } } - /** An iterator that sort-merges (K, C) pairs from the in-memory and on-disk maps */ + /** + * An iterator that sort-merges (K, C) pairs from the in-memory map and the spilled maps + */ private class ExternalIterator extends Iterator[(K, C)] { // A fixed-size queue that maintains a buffer for each stream we are currently merging @@ -179,7 +188,7 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( val sortedMap = currentMap.destructiveSortedIterator(comparator) val inputStreams = Seq(sortedMap) ++ spilledMaps - inputStreams.foreach{ it => + inputStreams.foreach { it => val kcPairs = getMorePairs(it) mergeHeap.enqueue(StreamBuffer(it, kcPairs)) } @@ -187,6 +196,7 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( /** * Fetch from the given iterator until a key of different hash is retrieved. In the * event of key hash collisions, this ensures no pairs are hidden from being merged. + * Assume the given iterator is in sorted order. */ def getMorePairs(it: Iterator[(K, C)]): ArrayBuffer[(K, C)] = { val kcPairs = new ArrayBuffer[(K, C)] @@ -219,17 +229,16 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( baseCombiner } - override def hasNext: Boolean = { - mergeHeap.foreach{ buffer => - if (!buffer.pairs.isEmpty) { - return true - } - } - false - } + /** + * Return true if there exists an input stream that still has unvisited pairs + */ + override def hasNext: Boolean = mergeHeap.exists(!_.pairs.isEmpty) + /** + * Select a key with the minimum hash, then combine all values with the same key from all input streams. + */ override def next(): (K, C) = { - // Select a return key from the StreamBuffer that holds the lowest key hash + // Select a key from the StreamBuffer that holds the lowest key hash val minBuffer = mergeHeap.dequeue() val (minPairs, minHash) = (minBuffer.pairs, minBuffer.minKeyHash) if (minPairs.length == 0) { @@ -285,45 +294,43 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( } } - // Iterate through (K, C) pairs in sorted order from an on-disk map + /** + * An iterator that returns (K, C) pairs in sorted order from an on-disk map + */ private class DiskMapIterator(file: File) extends Iterator[(K, C)] { val fileStream = new FileInputStream(file) val bufferedStream = new FastBufferedInputStream(fileStream) val deserializeStream = ser.deserializeStream(bufferedStream) - var nextItem: Option[(K, C)] = None + var nextItem: (K, C) = null var eof = false - def readNextItem(): Option[(K, C)] = { + def readNextItem(): (K, C) = { if (!eof) { try { - return Some(deserializeStream.readObject().asInstanceOf[(K, C)]) + return deserializeStream.readObject().asInstanceOf[(K, C)] } catch { case e: EOFException => eof = true cleanup() } } - None + null } override def hasNext: Boolean = { - nextItem match { - case Some(item) => true - case None => - nextItem = readNextItem() - nextItem.isDefined + if (nextItem == null) { + nextItem = readNextItem() } + nextItem != null } override def next(): (K, C) = { - nextItem match { - case Some(item) => - nextItem = None - item - case None => - val item = readNextItem() - item.getOrElse(throw new NoSuchElementException) + val item = if (nextItem == null) readNextItem() else nextItem + if (item == null) { + throw new NoSuchElementException } + nextItem = null + item } // TODO: Ensure this gets called even if the iterator isn't drained. diff --git a/docs/configuration.md b/docs/configuration.md index 6717757781..c1158491f0 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -104,13 +104,24 @@ Apart from these, the following properties are also available, and may be useful spark.storage.memoryFraction - 0.66 + 0.6 Fraction of Java heap to use for Spark's memory cache. This should not be larger than the "old" - generation of objects in the JVM, which by default is given 2/3 of the heap, but you can increase + generation of objects in the JVM, which by default is given 0.6 of the heap, but you can increase it if you configure your own old generation size. + + spark.shuffle.memoryFraction + 0.3 + + Fraction of Java heap to use for aggregation and cogroups during shuffles, if + spark.shuffle.externalSorting is enabled. At any given time, the collective size of + all in-memory maps used for shuffles is bounded by this limit, beyond which the contents will + begin to spill to disk. If spills are often, consider increasing this value at the expense of + spark.storage.memoryFraction. + + spark.mesos.coarse false @@ -376,6 +387,15 @@ Apart from these, the following properties are also available, and may be useful If set to "true", consolidates intermediate files created during a shuffle. Creating fewer files can improve filesystem performance for shuffles with large numbers of reduce tasks. It is recommended to set this to "true" when using ext4 or xfs filesystems. On ext3, this option might degrade performance on machines with many (>8) cores due to filesystem limitations. + + spark.shuffle.externalSorting + true + + If set to "true", spills in-memory maps used for shuffles to disk when a memory threshold is reached. This + threshold is specified by spark.shuffle.memoryFraction. Enable this especially for memory-intensive + applications. + + spark.speculation false -- cgit v1.2.3 From 2e393cd5fdfbf3a85fced370b5c42315e86dad49 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 10 Jan 2014 15:45:38 -0800 Subject: Update documentation for externalSorting --- docs/configuration.md | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index c1158491f0..40a57c4bc6 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -391,9 +391,8 @@ Apart from these, the following properties are also available, and may be useful spark.shuffle.externalSorting true - If set to "true", spills in-memory maps used for shuffles to disk when a memory threshold is reached. This - threshold is specified by spark.shuffle.memoryFraction. Enable this especially for memory-intensive - applications. + If set to "true", limits the amount of memory used during reduces by spilling data out to disk. This spilling + threshold is specified by spark.shuffle.memoryFraction. -- cgit v1.2.3