Skip to content

Commit

Permalink
Create central manager to track memory for all spilling collections
Browse files Browse the repository at this point in the history
This tracks memory properly if there are multiple spilling collections
in the same task (which was a problem before), and also implements an
algorithm that lets each thread grow up to 1 / 2N of the memory pool
(where N is the number of threads) before spilling, which avoids an
inefficiency with small spills we had before (some threads would spill
many times at 0-1 MB because the pool was allocated elsewhere).
  • Loading branch information
mateiz committed Aug 4, 2014
1 parent 8e7d5ba commit b810120
Show file tree
Hide file tree
Showing 6 changed files with 358 additions and 78 deletions.
7 changes: 3 additions & 4 deletions core/src/main/scala/org/apache/spark/SparkEnv.scala
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ import org.apache.spark.metrics.MetricsSystem
import org.apache.spark.network.ConnectionManager
import org.apache.spark.scheduler.LiveListenerBus
import org.apache.spark.serializer.Serializer
import org.apache.spark.shuffle.ShuffleManager
import org.apache.spark.shuffle.{ShuffleMemoryManager, ShuffleManager}
import org.apache.spark.storage._
import org.apache.spark.util.{AkkaUtils, Utils}

Expand Down Expand Up @@ -68,9 +68,8 @@ class SparkEnv (
val metricsSystem: MetricsSystem,
val conf: SparkConf) extends Logging {

// A mapping of thread ID to amount of memory, in bytes, used for shuffle aggregations
// All accesses should be manually synchronized
val shuffleMemoryMap = mutable.HashMap[Long, Long]()
// Manages the memory used by externally spilling collections in shuffle operations
val shuffleMemoryManager = new ShuffleMemoryManager(conf)

private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]()

Expand Down
5 changes: 1 addition & 4 deletions core/src/main/scala/org/apache/spark/executor/Executor.scala
Original file line number Diff line number Diff line change
Expand Up @@ -276,10 +276,7 @@ private[spark] class Executor(
}
} finally {
// Release memory used by this thread for shuffles
val shuffleMemoryMap = env.shuffleMemoryMap
shuffleMemoryMap.synchronized {
shuffleMemoryMap.remove(Thread.currentThread().getId)
}
env.shuffleMemoryManager.releaseMemoryForThisThread()
// Release memory used by this thread for unrolling blocks
env.blockManager.memoryStore.releaseUnrollMemoryForThisThread()
runningTasks.remove(taskId)
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,114 @@
/*
* 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.shuffle

import org.apache.spark.{SparkException, SparkConf}
import scala.collection.mutable

/**
* Allocates a pool of memory to task threads for use in shuffle operations. Each disk-spilling
* collection (ExternalAppendOnlyMap or ExternalSorter) used by these tasks can acquire memory
* from this pool and release it as it spills data out. When a task ends, all its memory will be
* released by the Executor.
*
* This class tries to ensure that each thread gets a reasonable share of memory, instead of some
* thread ramping up to a large amount first and then causing others to spill to disk repeatedly.
* If there are N threads, it ensures that each thread can acquire at least 1 / 2N of the memory
* before it has to spill, and at most 1 / N. Because N varies dynamically, we keep track of the
* set of active threads and redo the calculations of 1 / 2N and 1 / N in waiting threads whenever
* this set changes. This is all done by synchronizing access on "this" to mutate state and using
* wait() and notifyAll() to signal changes.
*/
private[spark] class ShuffleMemoryManager(maxMemory: Long) {
private val threadMemory = new mutable.HashMap[Long, Long]() // threadId -> memory bytes

def this(conf: SparkConf) = this(ShuffleMemoryManager.getMaxMemory(conf))

/**
* Try to acquire numBytes memory for the current thread, or return false if the pool cannot
* allocate this much memory to it. This call may block until there is enough free memory in
* some situations, to make sure each thread has a chance to ramp up to a reasonable share of
* the available memory before being forced to spill.
*/
def tryToAcquire(numBytes: Long): Boolean = synchronized {
val threadId = Thread.currentThread().getId

// Add this thread to the threadMemory map just so we can keep an accurate count of the number
// of active threads, to let other threads ramp down their memory in calls to tryToAcquire
threadMemory.getOrElseUpdate(threadId, 0L)

// Keep looping until we're either sure that we don't want to grant this request (because this
// thread would have more than 1 / numActiveThreads of the memory) or we have enough free
// memory to give it (we always let each thread get at least 1 / (2 * numActiveThreads)).
while (true) {
val numActiveThreads = threadMemory.keys.size
val curMem = threadMemory(threadId)
if (curMem + numBytes > maxMemory / numActiveThreads) {
// We'd get more than 1 / numActiveThreads of the total memory; don't allow that
return false
}
val bytesFree = maxMemory - threadMemory.values.sum
if (bytesFree >= numBytes) {
// Grant the request
threadMemory(threadId) = curMem + numBytes
// Notify other waiting threads because the # active of threads may have increased, so
// they may cancel their current waits
notifyAll()
return true
} else if (curMem + numBytes <= maxMemory / (2 * numActiveThreads)) {
// This thread has so little memory that we want it to block and acquire a bigger
// amount instead of cancelling the request. Wait on "this" for a thread to call notify.
// Before doing the wait, however, also notify other current waiters in case our thread
// becoming active just pushed them over the limit to give up their own waits.
notifyAll()
wait()
} else {
// Thread would have between 1 / (2 * numActiveThreads) and 1 / numActiveThreads memory
return false
}
}
false // Never reached
}

/** Release numBytes bytes for the current thread. */
def release(numBytes: Long): Unit = synchronized {
val threadId = Thread.currentThread().getId
val curMem = threadMemory.getOrElse(threadId, 0L)
if (curMem < numBytes) {
throw new SparkException(
s"Internal error: release called on ${numBytes} bytes but thread only has ${curMem}")
}
threadMemory(threadId) -= numBytes
notifyAll() // Notify waiters who locked "this" in tryToAcquire that memory has freed
}

/** Release all memory for the current thread and mark it as inactive (e.g. when a task ends). */
def releaseMemoryForThisThread(): Unit = synchronized {
val threadId = Thread.currentThread().getId
threadMemory.remove(threadId)
notifyAll() // Notify waiters who locked "this" in tryToAcquire that memory has freed
}
}

private object ShuffleMemoryManager {
def getMaxMemory(conf: SparkConf): Long = {
val memoryFraction = conf.getDouble("spark.shuffle.memoryFraction", 0.2)
val safetyFraction = conf.getDouble("spark.shuffle.safetyFraction", 0.8)
(Runtime.getRuntime.maxMemory * memoryFraction * safetyFraction).toLong
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -71,13 +71,7 @@ class ExternalAppendOnlyMap[K, V, C](
private val spilledMaps = new ArrayBuffer[DiskMapIterator]
private val sparkConf = SparkEnv.get.conf
private val diskBlockManager = blockManager.diskBlockManager

// Collective memory threshold shared across all running tasks
private val maxMemoryThreshold = {
val memoryFraction = sparkConf.getDouble("spark.shuffle.memoryFraction", 0.2)
val safetyFraction = sparkConf.getDouble("spark.shuffle.safetyFraction", 0.8)
(Runtime.getRuntime.maxMemory * memoryFraction * safetyFraction).toLong
}
private val shuffleMemoryManager = SparkEnv.get.shuffleMemoryManager

// Number of pairs inserted since last spill; note that we count them even if a value is merged
// with a previous key in case we're doing something like groupBy where the result grows
Expand Down Expand Up @@ -141,27 +135,14 @@ class ExternalAppendOnlyMap[K, V, C](
currentMap.estimateSize() >= myMemoryThreshold)
{
val currentSize = 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))

// Try to allocate at least 2x more memory, otherwise spill
shouldSpill = availableMemory < currentSize * 2
if (!shouldSpill) {
shuffleMemoryMap(threadId) = currentSize * 2
myMemoryThreshold = currentSize * 2
}
}
// Do not synchronize spills
if (shouldSpill) {
spill(currentSize)

// Try to grab double our currently used memory from the shuffle pool; if that fails,
// spill and release our claimed memory
val amountToGrab = 2 * currentSize - myMemoryThreshold
if (shuffleMemoryManager.tryToAcquire(amountToGrab)) {
myMemoryThreshold += amountToGrab
} else {
spill(currentSize) // Also releases memory back to the ShuffleMemoryManager at the end
}
}
currentMap.changeValue(curEntry._1, update)
Expand Down Expand Up @@ -245,12 +226,9 @@ class ExternalAppendOnlyMap[K, V, C](
currentMap = new SizeTrackingAppendOnlyMap[K, C]
spilledMaps.append(new DiskMapIterator(file, blockId, batchSizes))

// 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
}
myMemoryThreshold = 0
// Release our memory back to the shuffle pool so that other threads can grab it
shuffleMemoryManager.release(myMemoryThreshold)
myMemoryThreshold = 0L

elementsRead = 0
_memoryBytesSpilled += mapSize
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,7 @@ private[spark] class ExternalSorter[K, V, C](

private val blockManager = SparkEnv.get.blockManager
private val diskBlockManager = blockManager.diskBlockManager
private val shuffleMemoryManager = SparkEnv.get.shuffleMemoryManager
private val ser = Serializer.getSerializer(serializer)
private val serInstance = ser.newInstance()

Expand Down Expand Up @@ -116,13 +117,6 @@ private[spark] class ExternalSorter[K, V, C](
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.2)
val safetyFraction = conf.getDouble("spark.shuffle.safetyFraction", 0.8)
(Runtime.getRuntime.maxMemory * memoryFraction * safetyFraction).toLong
}

// How much of the shared memory pool this collection has claimed
private var myMemoryThreshold = 0L

Expand Down Expand Up @@ -218,31 +212,15 @@ private[spark] class ExternalSorter[K, V, C](
if (elementsRead > trackMemoryThreshold && elementsRead % 32 == 0 &&
collection.estimateSize() >= myMemoryThreshold)
{
// TODO: This logic doesn't work if there are two external collections being used in the same
// task (e.g. to read shuffle output and write it out into another shuffle) [SPARK-2711]

val currentSize = collection.estimateSize()
var shouldSpill = false
val shuffleMemoryMap = SparkEnv.get.shuffleMemoryMap

// Atomically check whether there is sufficient memory in the global pool for
// us to double our threshold
shuffleMemoryMap.synchronized {
val threadId = Thread.currentThread().getId
val previouslyClaimedMemory = shuffleMemoryMap.get(threadId)
val availableMemory = maxMemoryThreshold -
(shuffleMemoryMap.values.sum - previouslyClaimedMemory.getOrElse(0L))

// Try to allocate at least 2x more memory, otherwise spill
shouldSpill = availableMemory < currentSize * 2
if (!shouldSpill) {
shuffleMemoryMap(threadId) = currentSize * 2
myMemoryThreshold = currentSize * 2
}
}
// Do not hold lock during spills
if (shouldSpill) {
spill(currentSize, usingMap)

// Try to grab double our currently used memory from the shuffle pool; if that fails,
// spill and release our claimed memory
val amountToGrab = 2 * currentSize - myMemoryThreshold
if (shuffleMemoryManager.tryToAcquire(amountToGrab)) {
myMemoryThreshold += amountToGrab
} else {
spill(currentSize, usingMap) // Also releases memory back to the ShuffleMemoryManager
}
}
}
Expand Down Expand Up @@ -327,11 +305,8 @@ private[spark] class ExternalSorter[K, V, C](
buffer = new SizeTrackingPairBuffer[(Int, K), C]
}

// 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
}
// Release our memory back to the shuffle pool so that other threads can grab it
shuffleMemoryManager.release(myMemoryThreshold)
myMemoryThreshold = 0

spills.append(SpilledFile(file, blockId, batchSizes.toArray, elementsPerPartition))
Expand Down
Loading

0 comments on commit b810120

Please sign in to comment.