aboutsummaryrefslogtreecommitdiff
path: root/core/src/main/scala/org/apache/spark/Aggregator.scala
diff options
context:
space:
mode:
authorPatrick Wendell <pwendell@gmail.com>2014-01-14 13:52:23 -0800
committerPatrick Wendell <pwendell@gmail.com>2014-01-14 13:52:23 -0800
commit8ea2cd56e4a243a834214d04e29502a5fdb539df (patch)
treebf0e1807b56c6d6eb1afa86c3d14e6467c8c24c1 /core/src/main/scala/org/apache/spark/Aggregator.scala
parentb683608c9f5db0b320c91c4d027fff2e8ce3bd04 (diff)
downloadspark-8ea2cd56e4a243a834214d04e29502a5fdb539df.tar.gz
spark-8ea2cd56e4a243a834214d04e29502a5fdb539df.tar.bz2
spark-8ea2cd56e4a243a834214d04e29502a5fdb539df.zip
Adding fix covering combineCombinersByKey as well
Diffstat (limited to 'core/src/main/scala/org/apache/spark/Aggregator.scala')
-rw-r--r--core/src/main/scala/org/apache/spark/Aggregator.scala16
1 files changed, 12 insertions, 4 deletions
diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala
index d712927a21..edbea6ea56 100644
--- a/core/src/main/scala/org/apache/spark/Aggregator.scala
+++ b/core/src/main/scala/org/apache/spark/Aggregator.scala
@@ -17,6 +17,8 @@
package org.apache.spark
+import scala.{Option, deprecated}
+
import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap}
/**
@@ -35,10 +37,11 @@ case class Aggregator[K, V, C] (
private val externalSorting = sparkConf.getBoolean("spark.shuffle.spill", true)
@deprecated("use combineValuesByKey with TaskContext argument", "0.9.0")
- def combineValuesByKey(iter: Iterator[_ <: Product2[K, V]]) = combineValuesByKey(iter, null)
+ def combineValuesByKey(iter: Iterator[_ <: Product2[K, V]]): Iterator[(K, C)] =
+ combineValuesByKey(iter, null)
def combineValuesByKey(iter: Iterator[_ <: Product2[K, V]],
- context: TaskContext) : Iterator[(K, C)] = {
+ context: TaskContext): Iterator[(K, C)] = {
if (!externalSorting) {
val combiners = new AppendOnlyMap[K,C]
var kv: Product2[K, V] = null
@@ -63,6 +66,10 @@ case class Aggregator[K, V, C] (
}
}
+ @deprecated("use combineCombinersByKey with TaskContext argument", "0.9.0")
+ def combineCombinersByKey(iter: Iterator[(K, C)]) : Iterator[(K, C)] =
+ combineCombinersByKey(iter, null)
+
def combineCombinersByKey(iter: Iterator[(K, C)], context: TaskContext) : Iterator[(K, C)] = {
if (!externalSorting) {
val combiners = new AppendOnlyMap[K,C]
@@ -81,8 +88,9 @@ case class Aggregator[K, V, C] (
val (k, c) = iter.next()
combiners.insert(k, c)
}
- context.taskMetrics.memoryBytesSpilled = combiners.memoryBytesSpilled
- context.taskMetrics.diskBytesSpilled = combiners.diskBytesSpilled
+ // TODO: Make this non optional in a future release
+ Option(context).foreach(c => c.taskMetrics.memoryBytesSpilled = combiners.memoryBytesSpilled)
+ Option(context).foreach(c => c.taskMetrics.diskBytesSpilled = combiners.diskBytesSpilled)
combiners.iterator
}
}