Github user aarondav commented on a diff in the pull request:
https://github.com/apache/spark/pull/1499#discussion_r15152282
--- Diff:
core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala ---
@@ -0,0 +1,390 @@
+/*
+ * 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 java.util.Comparator
+
+import scala.collection.mutable.ArrayBuffer
+
+import com.google.common.io.ByteStreams
+
+import org.apache.spark.{Aggregator, SparkEnv, Logging, Partitioner}
+import org.apache.spark.serializer.Serializer
+import org.apache.spark.storage.BlockId
+
+/**
+ * Sorts and potentially merges a number of key-value pairs of type (K, V)
to produce key-combiner
+ * pairs of type (K, C). Uses a Partitioner to first group the keys into
partitions, and then
+ * optionally sorts keys within each partition using a custom Comparator.
Can output a single
+ * partitioned file with a different byte range for each partition,
suitable for shuffle fetches.
+ *
+ * If combining is disabled, the type C must equal V -- we'll cast the
objects at the end.
+ *
+ * @param aggregator optional Aggregator with combine functions to use for
merging data
+ * @param partitioner optional partitioner; if given, sort by partition ID
and then key
+ * @param ordering optional ordering to sort keys within each partition
+ * @param serializer serializer to use
+ */
+private[spark] class ExternalSorter[K, V, C](
+ aggregator: Option[Aggregator[K, V, C]] = None,
+ partitioner: Option[Partitioner] = None,
+ ordering: Option[Ordering[K]] = None,
+ serializer: Option[Serializer] = None) extends Logging {
+
+ private val numPartitions = partitioner.map(_.numPartitions).getOrElse(1)
+ private val shouldPartition = numPartitions > 1
+
+ private val blockManager = SparkEnv.get.blockManager
+ private val diskBlockManager = blockManager.diskBlockManager
+ private val ser = Serializer.getSerializer(serializer.getOrElse(null))
+ private val serInstance = ser.newInstance()
+
+ private val conf = SparkEnv.get.conf
+ private val fileBufferSize = conf.getInt("spark.shuffle.file.buffer.kb",
100) * 1024
+ private val serializerBatchSize =
conf.getLong("spark.shuffle.spill.batchSize", 10000)
+
+ private def getPartition(key: K): Int = {
+ if (shouldPartition) partitioner.get.getPartition(key) else 0
+ }
+
+ // Data structures to store in-memory objects before we spill. Depending
on whether we have an
+ // Aggregator set, we either put objects into an AppendOnlyMap where we
combine them, or we
+ // store them in an array buffer.
+ var map = new SizeTrackingAppendOnlyMap[(Int, K), C]
+ var buffer = new SizeTrackingBuffer[((Int, K), C)]
+
+ // Track how many elements we've read before we try to estimate memory.
Ideally we'd use
+ // map.size or buffer.size for this, but because users' Aggregators can
potentially increase
+ // the size of a merged element when we add values with the same key,
it's safer to track
+ // elements read from the input iterator.
+ private var elementsRead = 0L
+ private val trackMemoryThreshold = 1000
+
+ // Spilling statistics
+ private var spillCount = 0
+ private var _memoryBytesSpilled = 0L
+ private var _diskBytesSpilled = 0L
+
+ // Collective memory threshold shared across all running tasks
+ private val maxMemoryThreshold = {
+ val memoryFraction = conf.getDouble("spark.shuffle.memoryFraction",
0.3)
+ val safetyFraction = conf.getDouble("spark.shuffle.safetyFraction",
0.8)
+ (Runtime.getRuntime.maxMemory * memoryFraction * safetyFraction).toLong
+ }
+
+ // For now, just compare them by partition; later we can compare by key
as well
+ private val comparator = new Comparator[((Int, K), C)] {
+ override def compare(a: ((Int, K), C), b: ((Int, K), C)): Int = {
+ a._1._1 - b._1._1
+ }
+ }
+
+ // Information about a spilled file. Includes sizes in bytes of
"batches" written by the
+ // serializer as we periodically reset its stream, as well as number of
elements in each
+ // partition, used to efficiently keep track of partitions when merging.
+ private case class SpilledFile(
+ file: File,
+ blockId: BlockId,
+ serializerBatchSizes: ArrayBuffer[Long],
+ elementsPerPartition: Array[Long])
+ private val spills = new ArrayBuffer[SpilledFile]
+
+ def write(records: Iterator[_ <: Product2[K, V]]): Unit = {
+ // TODO: stop combining if we find that the reduction factor isn't high
+ val shouldCombine = aggregator.isDefined
+
+ if (shouldCombine) {
+ // Combine values in-memory first using our AppendOnlyMap
+ val mergeValue = aggregator.get.mergeValue
+ val createCombiner = aggregator.get.createCombiner
+ var kv: Product2[K, V] = null
+ val update = (hadValue: Boolean, oldValue: C) => {
+ if (hadValue) mergeValue(oldValue, kv._2) else
createCombiner(kv._2)
+ }
+ while (records.hasNext) {
+ elementsRead += 1
+ kv = records.next()
+ map.changeValue((getPartition(kv._1), kv._1), update)
+ maybeSpill(usingMap = true)
+ }
+ } else {
+ // Stick values into our buffer
+ while (records.hasNext) {
+ elementsRead += 1
+ val kv = records.next()
+ buffer += (((getPartition(kv._1), kv._1), kv._2.asInstanceOf[C]))
+ maybeSpill(usingMap = false)
+ }
+ }
+ }
+
+ private def maybeSpill(usingMap: Boolean): Unit = {
+ val collection: SizeTrackingCollection[((Int, K), C)] = if (usingMap)
map else buffer
+
+ if (elementsRead > trackMemoryThreshold && collection.atGrowThreshold)
{
--- End diff --
Doesn't this "collection.atGrowThreshold" thing assume that we're not just
combining into a few large keys? collection.estimateSize() is very fast, and is
intended to be possible to use it in a per-element manner.
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---