Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1499#discussion_r15192915
  
    --- Diff: 
core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala ---
    @@ -0,0 +1,573 @@
    +/*
    + * 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 scala.collection.mutable
    +
    +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 when spilling to disk
    + */
    +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)
    +  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
    +  }
    +
    +  // A comparator for keys K that orders them within a partition to allow 
partial aggregation.
    +  // Can be a partial ordering by hash code if a total ordering is not 
provided through by the
    +  // user. (A partial ordering means that equal keys have 
comparator.compare(k, k) = 0, but some
    +  // non-equal keys also have this, so we need to do a later pass to find 
truly equal keys).
    +  // Note that we ignore this if no aggregator is given.
    +  private val keyComparator: Comparator[K] = ordering.getOrElse(new 
Comparator[K] {
    +    override def compare(a: K, b: K): Int = {
    +      val h1 = if (a == null) 0 else a.hashCode()
    +      val h2 = if (b == null) 0 else b.hashCode()
    +      h1 - h2
    +    }
    +  })
    +
    +  private val sortWithinPartitions = ordering.isDefined || 
aggregator.isDefined
    +
    +  // A comparator for ((Int, K), C) elements that orders them by partition 
and then possibly by key
    +  private val partitionKeyComparator: Comparator[((Int, K), C)] = {
    +    if (sortWithinPartitions) {
    +      // Sort by partition ID then key comparator
    +      new Comparator[((Int, K), C)] {
    +        override def compare(a: ((Int, K), C), b: ((Int, K), C)): Int = {
    +          val partitionDiff = a._1._1 - b._1._1
    +          if (partitionDiff != 0) {
    +            partitionDiff
    +          } else {
    +            keyComparator.compare(a._1._2, b._1._2)
    +          }
    +        }
    +      }
    +    } else {
    +      // Just sort it by partition ID
    +      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[this] case class SpilledFile(
    +    file: File,
    +    blockId: BlockId,
    +    serializerBatchSizes: Array[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) 
{
    +      // TODO: This is code from ExternalAppendOnlyMap that doesn't work 
if there are two external
    +      // collections being used in the same task. However we'll just copy 
it for now.
    +
    +      val currentSize = collection.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 growth factor is 2x
    +        shouldSpill = availableMemory < currentSize * 2
    +        if (!shouldSpill) {
    +          shuffleMemoryMap(threadId) = currentSize * 2
    +        }
    +      }
    +      // Do not synchronize spills
    +      if (shouldSpill) {
    +        spill(currentSize, usingMap)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Spill the current in-memory collection to disk, adding a new file to 
spills, and clear it.
    +   *
    +   * @param usingMap whether we're using a map or buffer as our current 
in-memory collection
    +   */
    +  private def spill(memorySize: Long, usingMap: Boolean): Unit = {
    +    val collection: SizeTrackingCollection[((Int, K), C)] = if (usingMap) 
map else buffer
    +    val memorySize = collection.estimateSize()
    +
    +    spillCount += 1
    +    logWarning("Spilling in-memory batch of %d MB to disk (%d spill%s so 
far)"
    +      .format(memorySize / (1024 * 1024), spillCount, if (spillCount > 1) 
"s" else ""))
    +    val (blockId, file) = diskBlockManager.createTempBlock()
    +    var writer = blockManager.getDiskWriter(blockId, file, ser, 
fileBufferSize)
    +    var objectsWritten = 0
    +
    +    // List of batch sizes (bytes) in the order they are written to disk
    +    val batchSizes = new ArrayBuffer[Long]
    +
    +    // How many elements we have in each partition
    +    val elementsPerPartition = new Array[Long](numPartitions)
    +
    +    // Flush the disk writer's contents to disk, and update relevant 
variables
    +    def flush() = {
    +      writer.commit()
    +      val bytesWritten = writer.bytesWritten
    +      batchSizes.append(bytesWritten)
    +      _diskBytesSpilled += bytesWritten
    +      objectsWritten = 0
    +    }
    +
    +    try {
    +      val it = collection.destructiveSortedIterator(partitionKeyComparator)
    +      while (it.hasNext) {
    +        val elem = it.next()
    +        val partitionId = elem._1._1
    +        val key = elem._1._2
    +        val value = elem._2
    +        writer.write(key)
    +        writer.write(value)
    +        elementsPerPartition(partitionId) += 1
    +        objectsWritten += 1
    +
    +        if (objectsWritten == serializerBatchSize) {
    +          flush()
    +          writer.close()
    +          writer = blockManager.getDiskWriter(blockId, file, ser, 
fileBufferSize)
    +        }
    +      }
    +      if (objectsWritten > 0) {
    +        flush()
    +      }
    +      writer.close()
    +    } catch {
    +      case e: Exception =>
    +        writer.close()
    +        file.delete()
    +    }
    +
    +    if (usingMap) {
    +      map = new SizeTrackingAppendOnlyMap[(Int, K), C]
    +    } else {
    +      buffer = new SizeTrackingBuffer[((Int, K), C)]
    +    }
    +
    +    spills.append(SpilledFile(file, blockId, batchSizes.toArray, 
elementsPerPartition))
    +    _memoryBytesSpilled += memorySize
    +  }
    +
    +  /**
    +   * Merge a sequence of sorted files, giving an iterator over partitions 
and then over elements
    +   * inside each partition. This can be used to either write out a new 
file or return data to
    +   * the user.
    +   *
    +   * Returns an iterator over all the data written to this object, grouped 
by partition. For each
    +   * partition we then have an iterator over its contents, and these are 
expected to be accessed
    +   * in order (you can't "skip ahead" to one partition without reading the 
previous one).
    +   * Guaranteed to return a key-value pair for each partition, in order of 
partition ID.
    +   */
    +  private def merge(spills: Seq[SpilledFile], inMemory: Iterator[((Int, 
K), C)])
    +      : Iterator[(Int, Iterator[Product2[K, C]])] = {
    +    // TODO: merge intermediate results if they are sorted by the 
comparator
    +    val readers = spills.map(new SpillReader(_))
    +    val inMemBuffered = inMemory.buffered
    +    (0 until numPartitions).iterator.map { p =>
    +      val inMemIterator = new Iterator[Product2[K, C]] {
    +        override def hasNext: Boolean = {
    +          inMemBuffered.hasNext && inMemBuffered.head._1._1 == p
    +        }
    +        override def next(): Product2[K, C] = {
    +          val elem = inMemBuffered.next()
    +          (elem._1._2, elem._2)
    +        }
    +      }
    +      val iterators = readers.map(_.readNextPartition()) ++ 
Seq(inMemIterator)
    +      if (aggregator.isDefined) {
    +        // Perform partial aggregation across partitions
    +        (p, mergeWithAggregation(
    +          iterators, aggregator.get.mergeCombiners, keyComparator, 
ordering.isDefined))
    +      } else if (ordering.isDefined) {
    +        // No aggregator given, but we have an ordering (e.g. used by 
reduce tasks in sortByKey);
    +        // sort the elements without trying to merge them
    +        (p, mergeSort(iterators, ordering.get))
    +      } else {
    +        (p, iterators.iterator.flatten)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Merge-sort a sequence of (K, C) iterators using a given a comparator 
for the keys.
    +   */
    +  private def mergeSort(iterators: Seq[Iterator[Product2[K, C]]], 
comparator: Comparator[K])
    +    : Iterator[Product2[K, C]] =
    +  {
    +    val bufferedIters = iterators.map(_.buffered)
    +    type Iter = BufferedIterator[Product2[K, C]]
    +    val heap = new mutable.PriorityQueue[Iter]()(new Ordering[Iter] {
    +      override def compare(x: Iter, y: Iter): Int = 
-comparator.compare(x.head._1, y.head._1)
    +    })
    +    heap.enqueue(bufferedIters: _*)
    +    new Iterator[Product2[K, C]] {
    +      override def hasNext: Boolean = !heap.isEmpty
    +
    +      override def next(): Product2[K, C] = {
    +        if (!hasNext) {
    +          throw new NoSuchElementException
    +        }
    +        val firstBuf = heap.dequeue()
    +        val firstPair = firstBuf.next()
    +        if (firstBuf.hasNext) {
    +          heap.enqueue(firstBuf)
    +        }
    +        firstPair
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Merge a sequence of (K, C) iterators by aggregating values for each 
key, assuming that each
    +   * iterator is sorted by key with a given comparator. If the comparator 
is not a total ordering
    +   * (e.g. when we sort objects by hash code and different keys may 
compare as equal although
    +   * they're not), we still merge them by doing equality tests for all 
keys that compare as equal.
    +   */
    +  private def mergeWithAggregation(
    +      iterators: Seq[Iterator[Product2[K, C]]],
    +      mergeCombiners: (C, C) => C,
    +      comparator: Comparator[K],
    +      totalOrder: Boolean)
    +      : Iterator[Product2[K, C]] =
    --- End diff --
    
    Yeah I like to put the bracket that way, otherwise it sometimes gets lost


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to