Skip to content

Commit

Permalink
SPARK-2711. Create a ShuffleMemoryManager to track memory for all spi…
Browse files Browse the repository at this point in the history
…lling collections

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).

Author: Matei Zaharia <matei@databricks.com>

Closes #1707 from mateiz/spark-2711 and squashes the following commits:

debf75b [Matei Zaharia] Review comments
24f28f3 [Matei Zaharia] Small rename
c8f3a8b [Matei Zaharia] Update ShuffleMemoryManager to be able to partially grant requests
315e3a5 [Matei Zaharia] Some review comments
b810120 [Matei Zaharia] Create central manager to track memory for all spilling collections

(cherry picked from commit 4fde28c)
Signed-off-by: Matei Zaharia <matei@databricks.com>
  • Loading branch information
mateiz committed Aug 5, 2014
1 parent a092285 commit d13d253
Show file tree
Hide file tree
Showing 6 changed files with 450 additions and 81 deletions.
10 changes: 5 additions & 5 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 @@ -66,12 +66,9 @@ class SparkEnv (
val httpFileServer: HttpFileServer,
val sparkFilesDir: String,
val metricsSystem: MetricsSystem,
val shuffleMemoryManager: ShuffleMemoryManager,
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]()

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

// A general, soft-reference map for metadata needed during HadoopRDD split computation
Expand Down Expand Up @@ -252,6 +249,8 @@ object SparkEnv extends Logging {
val shuffleManager = instantiateClass[ShuffleManager](
"spark.shuffle.manager", "org.apache.spark.shuffle.hash.HashShuffleManager")

val shuffleMemoryManager = new ShuffleMemoryManager(conf)

// Warn about deprecated spark.cache.class property
if (conf.contains("spark.cache.class")) {
logWarning("The spark.cache.class property is no longer being used! Specify storage " +
Expand All @@ -273,6 +272,7 @@ object SparkEnv extends Logging {
httpFileServer,
sparkFilesDir,
metricsSystem,
shuffleMemoryManager,
conf)
}

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,125 @@
/*
* 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 scala.collection.mutable

import org.apache.spark.{Logging, SparkException, SparkConf}

/**
* 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) extends Logging {
private val threadMemory = new mutable.HashMap[Long, Long]() // threadId -> memory bytes

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

/**
* Try to acquire up to numBytes memory for the current thread, and return the number of bytes
* obtained, or 0 if none can be allocated. 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 at least 1 / 2N of the
* total memory pool (where N is the # of active threads) before it is forced to spill. This can
* happen if the number of threads increases but an older thread had a lot of memory already.
*/
def tryToAcquire(numBytes: Long): Long = synchronized {
val threadId = Thread.currentThread().getId
assert(numBytes > 0, "invalid number of bytes requested: " + numBytes)

// 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
if (!threadMemory.contains(threadId)) {
threadMemory(threadId) = 0L
notifyAll() // Will later cause waiting threads to wake up and check numThreads again
}

// 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)
val freeMemory = maxMemory - threadMemory.values.sum

// How much we can grant this thread; don't let it grow to more than 1 / numActiveThreads
val maxToGrant = math.min(numBytes, (maxMemory / numActiveThreads) - curMem)

if (curMem < maxMemory / (2 * numActiveThreads)) {
// We want to let each thread get at least 1 / (2 * numActiveThreads) before blocking;
// if we can't give it this much now, wait for other threads to free up memory
// (this happens if older threads allocated lots of memory before N grew)
if (freeMemory >= math.min(maxToGrant, maxMemory / (2 * numActiveThreads) - curMem)) {
val toGrant = math.min(maxToGrant, freeMemory)
threadMemory(threadId) += toGrant
return toGrant
} else {
logInfo(s"Thread $threadId waiting for at least 1/2N of shuffle memory pool to be free")
wait()
}
} else {
// Only give it as much memory as is free, which might be none if it reached 1 / numThreads
val toGrant = math.min(maxToGrant, freeMemory)
threadMemory(threadId) += toGrant
return toGrant
}
}
0L // 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 been 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 been freed
}
}

private object ShuffleMemoryManager {
/**
* Figure out the shuffle memory limit from a SparkConf. We currently have both a fraction
* of the memory pool and a safety factor since collections can sometimes grow bigger than
* the size we target before we estimate their sizes again.
*/
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 @@ -140,28 +134,15 @@ class ExternalAppendOnlyMap[K, V, C](
if (elementsRead > trackMemoryThreshold && elementsRead % 32 == 0 &&
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)
// Claim up to double our current memory from the shuffle memory pool
val currentMemory = currentMap.estimateSize()
val amountToRequest = 2 * currentMemory - myMemoryThreshold
val granted = shuffleMemoryManager.tryToAcquire(amountToRequest)
myMemoryThreshold += granted
if (myMemoryThreshold <= currentMemory) {
// We were granted too little memory to grow further (either tryToAcquire returned 0,
// or we already had more memory than myMemoryThreshold); spill the current collection
spill(currentMemory) // Will also release memory back to ShuffleMemoryManager
}
}
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)
// Claim up to double our current memory from the shuffle memory pool
val currentMemory = collection.estimateSize()
val amountToRequest = 2 * currentMemory - myMemoryThreshold
val granted = shuffleMemoryManager.tryToAcquire(amountToRequest)
myMemoryThreshold += granted
if (myMemoryThreshold <= currentMemory) {
// We were granted too little memory to grow further (either tryToAcquire returned 0,
// or we already had more memory than myMemoryThreshold); spill the current collection
spill(currentMemory, usingMap) // Will also release memory back to 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 d13d253

Please sign in to comment.